public Filter build() {
   Filter filter;
   switch (FilterType.valueOf(type)) {
     case ColumnCountGetFilter:
       filter = new ColumnCountGetFilter(limit);
       break;
     case ColumnPaginationFilter:
       filter = new ColumnPaginationFilter(limit, offset);
       break;
     case ColumnPrefixFilter:
       filter = new ColumnPrefixFilter(Base64.decode(value));
       break;
     case ColumnRangeFilter:
       filter =
           new ColumnRangeFilter(
               Base64.decode(minColumn),
               minColumnInclusive,
               Base64.decode(maxColumn),
               maxColumnInclusive);
       break;
     case DependentColumnFilter:
       filter =
           new DependentColumnFilter(
               Base64.decode(family),
               qualifier != null ? Base64.decode(qualifier) : null,
               dropDependentColumn,
               CompareOp.valueOf(op),
               comparator.build());
       break;
     case FamilyFilter:
       filter = new FamilyFilter(CompareOp.valueOf(op), comparator.build());
       break;
     case FilterList:
       {
         List<Filter> list = new ArrayList<Filter>();
         for (FilterModel model : filters) {
           list.add(model.build());
         }
         filter = new FilterList(FilterList.Operator.valueOf(op), list);
       }
       break;
     case FirstKeyOnlyFilter:
       filter = new FirstKeyOnlyFilter();
       break;
     case InclusiveStopFilter:
       filter = new InclusiveStopFilter(Base64.decode(value));
       break;
     case KeyOnlyFilter:
       filter = new KeyOnlyFilter();
       break;
     case MultipleColumnPrefixFilter:
       {
         byte[][] values = new byte[prefixes.size()][];
         for (int i = 0; i < prefixes.size(); i++) {
           values[i] = Base64.decode(prefixes.get(i));
         }
         filter = new MultipleColumnPrefixFilter(values);
       }
       break;
     case PageFilter:
       filter = new PageFilter(Long.valueOf(value));
       break;
     case PrefixFilter:
       filter = new PrefixFilter(Base64.decode(value));
       break;
     case QualifierFilter:
       filter = new QualifierFilter(CompareOp.valueOf(op), comparator.build());
       break;
     case RandomRowFilter:
       filter = new RandomRowFilter(chance);
       break;
     case RowFilter:
       filter = new RowFilter(CompareOp.valueOf(op), comparator.build());
       break;
     case SingleColumnValueFilter:
       filter =
           new SingleColumnValueFilter(
               Base64.decode(family),
               qualifier != null ? Base64.decode(qualifier) : null,
               CompareOp.valueOf(op),
               comparator.build());
       if (ifMissing != null) {
         ((SingleColumnValueFilter) filter).setFilterIfMissing(ifMissing);
       }
       if (latestVersion != null) {
         ((SingleColumnValueFilter) filter).setLatestVersionOnly(latestVersion);
       }
       break;
     case SingleColumnValueExcludeFilter:
       filter =
           new SingleColumnValueExcludeFilter(
               Base64.decode(family),
               qualifier != null ? Base64.decode(qualifier) : null,
               CompareOp.valueOf(op),
               comparator.build());
       if (ifMissing != null) {
         ((SingleColumnValueExcludeFilter) filter).setFilterIfMissing(ifMissing);
       }
       if (latestVersion != null) {
         ((SingleColumnValueExcludeFilter) filter).setLatestVersionOnly(latestVersion);
       }
       break;
     case SkipFilter:
       filter = new SkipFilter(filters.get(0).build());
       break;
     case TimestampsFilter:
       filter = new TimestampsFilter(timestamps);
       break;
     case ValueFilter:
       filter = new ValueFilter(CompareOp.valueOf(op), comparator.build());
       break;
     case WhileMatchFilter:
       filter = new WhileMatchFilter(filters.get(0).build());
       break;
     default:
       throw new RuntimeException("unhandled filter type: " + type);
   }
   return filter;
 }
 public FilterModel(Filter filter) {
   String typeName = filter.getClass().getSimpleName();
   FilterType type = FilterType.valueOf(typeName);
   this.type = typeName;
   switch (type) {
     case ColumnCountGetFilter:
       this.limit = ((ColumnCountGetFilter) filter).getLimit();
       break;
     case ColumnPaginationFilter:
       this.limit = ((ColumnPaginationFilter) filter).getLimit();
       this.offset = ((ColumnPaginationFilter) filter).getOffset();
       break;
     case ColumnPrefixFilter:
       this.value = Base64.encodeBytes(((ColumnPrefixFilter) filter).getPrefix());
       break;
     case ColumnRangeFilter:
       this.minColumn = Base64.encodeBytes(((ColumnRangeFilter) filter).getMinColumn());
       this.minColumnInclusive = ((ColumnRangeFilter) filter).getMinColumnInclusive();
       this.maxColumn = Base64.encodeBytes(((ColumnRangeFilter) filter).getMaxColumn());
       this.maxColumnInclusive = ((ColumnRangeFilter) filter).getMaxColumnInclusive();
       break;
     case DependentColumnFilter:
       {
         DependentColumnFilter dcf = (DependentColumnFilter) filter;
         this.family = Base64.encodeBytes(dcf.getFamily());
         byte[] qualifier = dcf.getQualifier();
         if (qualifier != null) {
           this.qualifier = Base64.encodeBytes(qualifier);
         }
         this.op = dcf.getOperator().toString();
         this.comparator = new WritableByteArrayComparableModel(dcf.getComparator());
         this.dropDependentColumn = dcf.dropDependentColumn();
       }
       break;
     case FilterList:
       this.op = ((FilterList) filter).getOperator().toString();
       this.filters = new ArrayList<FilterModel>();
       for (Filter child : ((FilterList) filter).getFilters()) {
         this.filters.add(new FilterModel(child));
       }
       break;
     case FirstKeyOnlyFilter:
     case KeyOnlyFilter:
       break;
     case InclusiveStopFilter:
       this.value = Base64.encodeBytes(((InclusiveStopFilter) filter).getStopRowKey());
       break;
     case MultipleColumnPrefixFilter:
       this.prefixes = new ArrayList<String>();
       for (byte[] prefix : ((MultipleColumnPrefixFilter) filter).getPrefix()) {
         this.prefixes.add(Base64.encodeBytes(prefix));
       }
       break;
     case PageFilter:
       this.value = Long.toString(((PageFilter) filter).getPageSize());
       break;
     case PrefixFilter:
       this.value = Base64.encodeBytes(((PrefixFilter) filter).getPrefix());
       break;
     case FamilyFilter:
     case QualifierFilter:
     case RowFilter:
     case ValueFilter:
       this.op = ((CompareFilter) filter).getOperator().toString();
       this.comparator =
           new WritableByteArrayComparableModel(((CompareFilter) filter).getComparator());
       break;
     case RandomRowFilter:
       this.chance = ((RandomRowFilter) filter).getChance();
       break;
     case SingleColumnValueExcludeFilter:
     case SingleColumnValueFilter:
       {
         SingleColumnValueFilter scvf = (SingleColumnValueFilter) filter;
         this.family = Base64.encodeBytes(scvf.getFamily());
         byte[] qualifier = scvf.getQualifier();
         if (qualifier != null) {
           this.qualifier = Base64.encodeBytes(qualifier);
         }
         this.op = scvf.getOperator().toString();
         this.comparator = new WritableByteArrayComparableModel(scvf.getComparator());
         if (scvf.getFilterIfMissing()) {
           this.ifMissing = true;
         }
         if (scvf.getLatestVersionOnly()) {
           this.latestVersion = true;
         }
       }
       break;
     case SkipFilter:
       this.filters = new ArrayList<FilterModel>();
       this.filters.add(new FilterModel(((SkipFilter) filter).getFilter()));
       break;
     case TimestampsFilter:
       this.timestamps = ((TimestampsFilter) filter).getTimestamps();
       break;
     case WhileMatchFilter:
       this.filters = new ArrayList<FilterModel>();
       this.filters.add(new FilterModel(((WhileMatchFilter) filter).getFilter()));
       break;
     default:
       throw new RuntimeException("unhandled filter type " + type);
   }
 }