001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.rest.model;
019
020import com.fasterxml.jackson.annotation.JsonInclude;
021import java.io.IOException;
022import java.io.Serializable;
023import java.util.ArrayList;
024import java.util.Arrays;
025import java.util.Base64;
026import java.util.List;
027import java.util.Map;
028import java.util.NavigableSet;
029import java.util.Objects;
030import javax.xml.bind.annotation.XmlAttribute;
031import javax.xml.bind.annotation.XmlElement;
032import javax.xml.bind.annotation.XmlRootElement;
033import org.apache.hadoop.hbase.CompareOperator;
034import org.apache.hadoop.hbase.HConstants;
035import org.apache.hadoop.hbase.client.Scan;
036import org.apache.hadoop.hbase.filter.BinaryComparator;
037import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
038import org.apache.hadoop.hbase.filter.BitComparator;
039import org.apache.hadoop.hbase.filter.ByteArrayComparable;
040import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
041import org.apache.hadoop.hbase.filter.ColumnPaginationFilter;
042import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
043import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
044import org.apache.hadoop.hbase.filter.CompareFilter;
045import org.apache.hadoop.hbase.filter.DependentColumnFilter;
046import org.apache.hadoop.hbase.filter.FamilyFilter;
047import org.apache.hadoop.hbase.filter.Filter;
048import org.apache.hadoop.hbase.filter.FilterList;
049import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
050import org.apache.hadoop.hbase.filter.FuzzyRowFilter;
051import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
052import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
053import org.apache.hadoop.hbase.filter.MultiRowRangeFilter;
054import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange;
055import org.apache.hadoop.hbase.filter.MultipleColumnPrefixFilter;
056import org.apache.hadoop.hbase.filter.NullComparator;
057import org.apache.hadoop.hbase.filter.PageFilter;
058import org.apache.hadoop.hbase.filter.PrefixFilter;
059import org.apache.hadoop.hbase.filter.QualifierFilter;
060import org.apache.hadoop.hbase.filter.RandomRowFilter;
061import org.apache.hadoop.hbase.filter.RegexStringComparator;
062import org.apache.hadoop.hbase.filter.RowFilter;
063import org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter;
064import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
065import org.apache.hadoop.hbase.filter.SkipFilter;
066import org.apache.hadoop.hbase.filter.SubstringComparator;
067import org.apache.hadoop.hbase.filter.TimestampsFilter;
068import org.apache.hadoop.hbase.filter.ValueFilter;
069import org.apache.hadoop.hbase.filter.WhileMatchFilter;
070import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
071import org.apache.hadoop.hbase.rest.RestUtil;
072import org.apache.hadoop.hbase.security.visibility.Authorizations;
073import org.apache.hadoop.hbase.util.Bytes;
074import org.apache.hadoop.hbase.util.Pair;
075import org.apache.yetus.audience.InterfaceAudience;
076
077import org.apache.hbase.thirdparty.com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider;
078import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
079import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream;
080import org.apache.hbase.thirdparty.com.google.protobuf.Message;
081import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
082import org.apache.hbase.thirdparty.javax.ws.rs.core.MediaType;
083
084import org.apache.hadoop.hbase.shaded.rest.protobuf.generated.ScannerMessage.Scanner;
085
086/**
087 * A representation of Scanner parameters.
088 *
089 * <pre>
090 * &lt;complexType name="Scanner"&gt;
091 *   &lt;sequence&gt;
092 *     &lt;element name="column" type="base64Binary" minOccurs="0" maxOccurs="unbounded"/&gt;
093 *     &lt;element name="filter" type="string" minOccurs="0" maxOccurs="1"&gt;&lt;/element&gt;
094 *   &lt;/sequence&gt;
095 *   &lt;attribute name="startRow" type="base64Binary"&gt;&lt;/attribute&gt;
096 *   &lt;attribute name="endRow" type="base64Binary"&gt;&lt;/attribute&gt;
097 *   &lt;attribute name="batch" type="int"&gt;&lt;/attribute&gt;
098 *   &lt;attribute name="caching" type="int"&gt;&lt;/attribute&gt;
099 *   &lt;attribute name="startTime" type="int"&gt;&lt;/attribute&gt;
100 *   &lt;attribute name="endTime" type="int"&gt;&lt;/attribute&gt;
101 *   &lt;attribute name="maxVersions" type="int"&gt;&lt;/attribute&gt;
102 * &lt;/complexType&gt;
103 * </pre>
104 */
105@XmlRootElement(name = "Scanner")
106@JsonInclude(JsonInclude.Include.NON_NULL)
107@InterfaceAudience.Private
108public class ScannerModel implements ProtobufMessageHandler, Serializable {
109
110  private static final long serialVersionUID = 1L;
111
112  private byte[] startRow = HConstants.EMPTY_START_ROW;
113  private byte[] endRow = HConstants.EMPTY_END_ROW;
114  private List<byte[]> columns = new ArrayList<>();
115  private int batch = Integer.MAX_VALUE;
116  private long startTime = 0;
117  private long endTime = Long.MAX_VALUE;
118  private String filter = null;
119  private int maxVersions = Integer.MAX_VALUE;
120  private int caching = -1;
121  private List<String> labels = new ArrayList<>();
122  private boolean cacheBlocks = true;
123  private int limit = -1;
124
125  /**
126   * Implement lazily-instantiated singleton as per recipe here:
127   * http://literatejava.com/jvm/fastest-threadsafe-singleton-jvm/
128   */
129  private static class JaxbJsonProviderHolder {
130    static final JacksonJaxbJsonProvider INSTANCE = new JacksonJaxbJsonProvider();
131  }
132
133  @XmlRootElement
134  static class FilterModel {
135
136    @XmlRootElement
137    static class ByteArrayComparableModel {
138      @XmlAttribute
139      public String type;
140      @XmlAttribute
141      public String value;
142      @XmlAttribute
143      public String op;
144
145      static enum ComparatorType {
146        BinaryComparator,
147        BinaryPrefixComparator,
148        BitComparator,
149        NullComparator,
150        RegexStringComparator,
151        SubstringComparator
152      }
153
154      public ByteArrayComparableModel() {
155      }
156
157      public ByteArrayComparableModel(ByteArrayComparable comparator) {
158        String typeName = comparator.getClass().getSimpleName();
159        ComparatorType type = ComparatorType.valueOf(typeName);
160        this.type = typeName;
161        switch (type) {
162          case BinaryComparator:
163          case BinaryPrefixComparator:
164            this.value = Bytes.toString(Base64.getEncoder().encode(comparator.getValue()));
165            break;
166          case BitComparator:
167            this.value = Bytes.toString(Base64.getEncoder().encode(comparator.getValue()));
168            this.op = ((BitComparator) comparator).getOperator().toString();
169            break;
170          case NullComparator:
171            break;
172          case RegexStringComparator:
173          case SubstringComparator:
174            this.value = Bytes.toString(comparator.getValue());
175            break;
176          default:
177            throw new RuntimeException("unhandled filter type: " + type);
178        }
179      }
180
181      public ByteArrayComparable build() {
182        ByteArrayComparable comparator;
183        switch (ComparatorType.valueOf(type)) {
184          case BinaryComparator:
185            comparator = new BinaryComparator(Base64.getDecoder().decode(value));
186            break;
187          case BinaryPrefixComparator:
188            comparator = new BinaryPrefixComparator(Base64.getDecoder().decode(value));
189            break;
190          case BitComparator:
191            comparator = new BitComparator(Base64.getDecoder().decode(value),
192              BitComparator.BitwiseOp.valueOf(op));
193            break;
194          case NullComparator:
195            comparator = new NullComparator();
196            break;
197          case RegexStringComparator:
198            comparator = new RegexStringComparator(value);
199            break;
200          case SubstringComparator:
201            comparator = new SubstringComparator(value);
202            break;
203          default:
204            throw new RuntimeException("unhandled comparator type: " + type);
205        }
206        return comparator;
207      }
208
209    }
210
211    /**
212     * This DTO omits the pseudo-getters in MultiRowRangeFilter.RowRange which break Jackson
213     * deserialization. It also avoids adding those as dummy JSON elements.
214     */
215    static class RowRangeModel {
216
217      protected byte[] startRow;
218
219      protected boolean startRowInclusive = true;
220
221      protected byte[] stopRow;
222
223      protected boolean stopRowInclusive = false;
224
225      public RowRangeModel() {
226      }
227
228      public RowRangeModel(MultiRowRangeFilter.RowRange rr) {
229        this.startRow = rr.getStartRow();
230        this.startRowInclusive = rr.isStartRowInclusive();
231        this.stopRow = rr.getStopRow();
232        this.stopRowInclusive = rr.isStopRowInclusive();
233      }
234
235      public MultiRowRangeFilter.RowRange build() {
236        return new MultiRowRangeFilter.RowRange(startRow, startRowInclusive, stopRow,
237          stopRowInclusive);
238      }
239
240      public byte[] getStartRow() {
241        return startRow;
242      }
243
244      public byte[] getStopRow() {
245        return stopRow;
246      }
247
248      /** Returns if start row is inclusive. */
249      public boolean isStartRowInclusive() {
250        return startRowInclusive;
251      }
252
253      /** Returns if stop row is inclusive. */
254      public boolean isStopRowInclusive() {
255        return stopRowInclusive;
256      }
257
258      @Override
259      public int hashCode() {
260        final int prime = 31;
261        int result = 1;
262        result = prime * result + Arrays.hashCode(startRow);
263        result = prime * result + Arrays.hashCode(stopRow);
264        result = prime * result + Objects.hash(startRowInclusive, stopRowInclusive);
265        return result;
266      }
267
268      @Override
269      public boolean equals(Object obj) {
270        if (this == obj) {
271          return true;
272        }
273        if (!(obj instanceof RowRangeModel)) {
274          return false;
275        }
276        RowRangeModel other = (RowRangeModel) obj;
277        return Arrays.equals(startRow, other.startRow)
278          && startRowInclusive == other.startRowInclusive && Arrays.equals(stopRow, other.stopRow)
279          && stopRowInclusive == other.stopRowInclusive;
280      }
281
282    }
283
284    static class FuzzyKeyModel {
285
286      protected byte[] key;
287
288      protected byte[] mask;
289
290      public FuzzyKeyModel() {
291      }
292
293      public FuzzyKeyModel(Pair<byte[], byte[]> keyWithMask) {
294        this.key = keyWithMask.getFirst();
295        this.mask = keyWithMask.getSecond();
296      }
297
298      public Pair<byte[], byte[]> build() {
299        return new Pair<>(key, mask);
300      }
301
302      public byte[] getKey() {
303        return key;
304      }
305
306      public void setKey(byte[] key) {
307        this.key = key;
308      }
309
310      public byte[] getMask() {
311        return mask;
312      }
313
314      public void setMask(byte[] mask) {
315        this.mask = mask;
316      }
317
318      @Override
319      public int hashCode() {
320        final int prime = 31;
321        int result = 1;
322        result = prime * result + Arrays.hashCode(key);
323        result = prime * result + Arrays.hashCode(mask);
324        return result;
325      }
326
327      @Override
328      public boolean equals(Object obj) {
329        if (this == obj) {
330          return true;
331        }
332        if (!(obj instanceof FuzzyKeyModel)) {
333          return false;
334        }
335        FuzzyKeyModel other = (FuzzyKeyModel) obj;
336        return Arrays.equals(key, other.key) && Arrays.equals(mask, other.mask);
337      }
338
339    }
340
341    // A grab bag of fields, would have been a union if this were C.
342    // These are null by default and will only be serialized if set (non null).
343    @XmlAttribute
344    public String type;
345    @XmlAttribute
346    public String op;
347    @XmlElement
348    ByteArrayComparableModel comparator;
349    @XmlAttribute
350    public String value;
351    @XmlElement
352    public List<FilterModel> filters;
353    @XmlAttribute
354    public Integer limit;
355    @XmlAttribute
356    public Integer offset;
357    @XmlAttribute
358    public String family;
359    @XmlAttribute
360    public String qualifier;
361    @XmlAttribute
362    public Boolean ifMissing;
363    @XmlAttribute
364    public Boolean latestVersion;
365    @XmlAttribute
366    public String minColumn;
367    @XmlAttribute
368    public Boolean minColumnInclusive;
369    @XmlAttribute
370    public String maxColumn;
371    @XmlAttribute
372    public Boolean maxColumnInclusive;
373    @XmlAttribute
374    public Boolean dropDependentColumn;
375    @XmlAttribute
376    public Float chance;
377    @XmlElement
378    public List<String> prefixes;
379    @XmlElement
380    private List<RowRangeModel> ranges;
381    @XmlElement
382    public List<Long> timestamps;
383    @XmlElement
384    private List<FuzzyKeyModel> fuzzyKeys;
385
386    static enum FilterType {
387      ColumnCountGetFilter,
388      ColumnPaginationFilter,
389      ColumnPrefixFilter,
390      ColumnRangeFilter,
391      DependentColumnFilter,
392      FamilyFilter,
393      FilterList,
394      FirstKeyOnlyFilter,
395      InclusiveStopFilter,
396      KeyOnlyFilter,
397      MultipleColumnPrefixFilter,
398      MultiRowRangeFilter,
399      PageFilter,
400      PrefixFilter,
401      QualifierFilter,
402      RandomRowFilter,
403      RowFilter,
404      SingleColumnValueExcludeFilter,
405      SingleColumnValueFilter,
406      SkipFilter,
407      TimestampsFilter,
408      ValueFilter,
409      WhileMatchFilter,
410      FuzzyRowFilter
411    }
412
413    public FilterModel() {
414    }
415
416    public FilterModel(Filter filter) {
417      String typeName = filter.getClass().getSimpleName();
418      FilterType type = FilterType.valueOf(typeName);
419      this.type = typeName;
420      switch (type) {
421        case ColumnCountGetFilter:
422          this.limit = ((ColumnCountGetFilter) filter).getLimit();
423          break;
424        case ColumnPaginationFilter:
425          this.limit = ((ColumnPaginationFilter) filter).getLimit();
426          this.offset = ((ColumnPaginationFilter) filter).getOffset();
427          break;
428        case ColumnPrefixFilter:
429          byte[] src = ((ColumnPrefixFilter) filter).getPrefix();
430          this.value = Bytes.toString(Base64.getEncoder().encode(src));
431          break;
432        case ColumnRangeFilter:
433          ColumnRangeFilter crf = (ColumnRangeFilter) filter;
434          this.minColumn = Bytes.toString(Base64.getEncoder().encode(crf.getMinColumn()));
435          this.minColumnInclusive = crf.getMinColumnInclusive();
436          this.maxColumn = Bytes.toString(Base64.getEncoder().encode(crf.getMaxColumn()));
437          this.maxColumnInclusive = crf.getMaxColumnInclusive();
438          break;
439        case DependentColumnFilter: {
440          DependentColumnFilter dcf = (DependentColumnFilter) filter;
441          this.family = Bytes.toString(Base64.getEncoder().encode(dcf.getFamily()));
442          byte[] qualifier = dcf.getQualifier();
443          if (qualifier != null) {
444            this.qualifier = Bytes.toString(Base64.getEncoder().encode(qualifier));
445          }
446          this.op = dcf.getCompareOperator().toString();
447          this.comparator = new ByteArrayComparableModel(dcf.getComparator());
448          this.dropDependentColumn = dcf.dropDependentColumn();
449        }
450          break;
451        case FilterList:
452          this.op = ((FilterList) filter).getOperator().toString();
453          this.filters = new ArrayList<>();
454          for (Filter child : ((FilterList) filter).getFilters()) {
455            this.filters.add(new FilterModel(child));
456          }
457          break;
458        case FirstKeyOnlyFilter:
459        case KeyOnlyFilter:
460          break;
461        case InclusiveStopFilter:
462          this.value = Bytes
463            .toString(Base64.getEncoder().encode(((InclusiveStopFilter) filter).getStopRowKey()));
464          break;
465        case MultipleColumnPrefixFilter:
466          this.prefixes = new ArrayList<>();
467          for (byte[] prefix : ((MultipleColumnPrefixFilter) filter).getPrefix()) {
468            this.prefixes.add(Bytes.toString(Base64.getEncoder().encode(prefix)));
469          }
470          break;
471        case MultiRowRangeFilter:
472          this.ranges = new ArrayList<>();
473          for (RowRange range : ((MultiRowRangeFilter) filter).getRowRanges()) {
474            this.ranges.add(new RowRangeModel(range));
475          }
476          break;
477        case PageFilter:
478          this.value = Long.toString(((PageFilter) filter).getPageSize());
479          break;
480        case PrefixFilter:
481          this.value =
482            Bytes.toString(Base64.getEncoder().encode(((PrefixFilter) filter).getPrefix()));
483          break;
484        case FamilyFilter:
485        case QualifierFilter:
486        case RowFilter:
487        case ValueFilter:
488          this.op = ((CompareFilter) filter).getCompareOperator().toString();
489          this.comparator = new ByteArrayComparableModel(((CompareFilter) filter).getComparator());
490          break;
491        case RandomRowFilter:
492          this.chance = ((RandomRowFilter) filter).getChance();
493          break;
494        case SingleColumnValueExcludeFilter:
495        case SingleColumnValueFilter: {
496          SingleColumnValueFilter scvf = (SingleColumnValueFilter) filter;
497          this.family = Bytes.toString(Base64.getEncoder().encode(scvf.getFamily()));
498          byte[] qualifier = scvf.getQualifier();
499          if (qualifier != null) {
500            this.qualifier = Bytes.toString(Base64.getEncoder().encode(qualifier));
501          }
502          this.op = scvf.getCompareOperator().toString();
503          this.comparator = new ByteArrayComparableModel(scvf.getComparator());
504          if (scvf.getFilterIfMissing()) {
505            this.ifMissing = true;
506          }
507          if (scvf.getLatestVersionOnly()) {
508            this.latestVersion = true;
509          }
510        }
511          break;
512        case SkipFilter:
513          this.filters = new ArrayList<>();
514          this.filters.add(new FilterModel(((SkipFilter) filter).getFilter()));
515          break;
516        case TimestampsFilter:
517          this.timestamps = ((TimestampsFilter) filter).getTimestamps();
518          break;
519        case WhileMatchFilter:
520          this.filters = new ArrayList<>();
521          this.filters.add(new FilterModel(((WhileMatchFilter) filter).getFilter()));
522          break;
523        case FuzzyRowFilter:
524          this.fuzzyKeys = new ArrayList<>(((FuzzyRowFilter) filter).getFuzzyKeys().size());
525          for (Pair<byte[], byte[]> keyWithMask : ((FuzzyRowFilter) filter).getFuzzyKeys()) {
526            this.fuzzyKeys.add(new FuzzyKeyModel(keyWithMask));
527          }
528          break;
529        default:
530          throw new RuntimeException("unhandled filter type " + type);
531      }
532    }
533
534    public Filter build() {
535      Filter filter;
536      switch (FilterType.valueOf(type)) {
537        case ColumnCountGetFilter:
538          filter = new ColumnCountGetFilter(limit);
539          break;
540        case ColumnPaginationFilter:
541          filter = new ColumnPaginationFilter(limit, offset);
542          break;
543        case ColumnPrefixFilter:
544          filter = new ColumnPrefixFilter(Base64.getDecoder().decode(value));
545          break;
546        case ColumnRangeFilter:
547          filter = new ColumnRangeFilter(Base64.getDecoder().decode(minColumn), minColumnInclusive,
548            Base64.getDecoder().decode(maxColumn), maxColumnInclusive);
549          break;
550        case DependentColumnFilter:
551          filter = new DependentColumnFilter(Base64.getDecoder().decode(family),
552            qualifier != null ? Base64.getDecoder().decode(qualifier) : null, dropDependentColumn,
553            CompareOperator.valueOf(op), comparator.build());
554          break;
555        case FamilyFilter:
556          filter = new FamilyFilter(CompareOperator.valueOf(op), comparator.build());
557          break;
558        case FilterList: {
559          List<Filter> list = new ArrayList<>(filters.size());
560          for (FilterModel model : filters) {
561            list.add(model.build());
562          }
563          filter = new FilterList(FilterList.Operator.valueOf(op), list);
564        }
565          break;
566        case FirstKeyOnlyFilter:
567          filter = new FirstKeyOnlyFilter();
568          break;
569        case InclusiveStopFilter:
570          filter = new InclusiveStopFilter(Base64.getDecoder().decode(value));
571          break;
572        case KeyOnlyFilter:
573          filter = new KeyOnlyFilter();
574          break;
575        case MultipleColumnPrefixFilter: {
576          byte[][] values = new byte[prefixes.size()][];
577          for (int i = 0; i < prefixes.size(); i++) {
578            values[i] = Base64.getDecoder().decode(prefixes.get(i));
579          }
580          filter = new MultipleColumnPrefixFilter(values);
581        }
582          break;
583        case MultiRowRangeFilter: {
584          ArrayList<MultiRowRangeFilter.RowRange> rowRanges = new ArrayList<>(ranges.size());
585          for (RowRangeModel rangeModel : ranges) {
586            rowRanges.add(rangeModel.build());
587          }
588          filter = new MultiRowRangeFilter(rowRanges);
589        }
590          break;
591        case PageFilter:
592          filter = new PageFilter(Long.parseLong(value));
593          break;
594        case PrefixFilter:
595          filter = new PrefixFilter(Base64.getDecoder().decode(value));
596          break;
597        case QualifierFilter:
598          filter = new QualifierFilter(CompareOperator.valueOf(op), comparator.build());
599          break;
600        case RandomRowFilter:
601          filter = new RandomRowFilter(chance);
602          break;
603        case RowFilter:
604          filter = new RowFilter(CompareOperator.valueOf(op), comparator.build());
605          break;
606        case SingleColumnValueFilter:
607          filter = new SingleColumnValueFilter(Base64.getDecoder().decode(family),
608            qualifier != null ? Base64.getDecoder().decode(qualifier) : null,
609            CompareOperator.valueOf(op), comparator.build());
610          if (ifMissing != null) {
611            ((SingleColumnValueFilter) filter).setFilterIfMissing(ifMissing);
612          }
613          if (latestVersion != null) {
614            ((SingleColumnValueFilter) filter).setLatestVersionOnly(latestVersion);
615          }
616          break;
617        case SingleColumnValueExcludeFilter:
618          filter = new SingleColumnValueExcludeFilter(Base64.getDecoder().decode(family),
619            qualifier != null ? Base64.getDecoder().decode(qualifier) : null,
620            CompareOperator.valueOf(op), comparator.build());
621          if (ifMissing != null) {
622            ((SingleColumnValueExcludeFilter) filter).setFilterIfMissing(ifMissing);
623          }
624          if (latestVersion != null) {
625            ((SingleColumnValueExcludeFilter) filter).setLatestVersionOnly(latestVersion);
626          }
627          break;
628        case SkipFilter:
629          filter = new SkipFilter(filters.get(0).build());
630          break;
631        case TimestampsFilter:
632          filter = new TimestampsFilter(timestamps);
633          break;
634        case ValueFilter:
635          filter = new ValueFilter(CompareOperator.valueOf(op), comparator.build());
636          break;
637        case WhileMatchFilter:
638          filter = new WhileMatchFilter(filters.get(0).build());
639          break;
640        case FuzzyRowFilter: {
641          ArrayList<Pair<byte[], byte[]>> fuzzyKeyArgs = new ArrayList<>(fuzzyKeys.size());
642          for (FuzzyKeyModel keyModel : fuzzyKeys) {
643            fuzzyKeyArgs.add(keyModel.build());
644          }
645          filter = new FuzzyRowFilter(fuzzyKeyArgs);
646        }
647          break;
648        default:
649          throw new RuntimeException("unhandled filter type: " + type);
650      }
651      return filter;
652    }
653
654  }
655
656  /**
657   * Get the <code>JacksonJaxbJsonProvider</code> instance;
658   * @return A <code>JacksonJaxbJsonProvider</code>.
659   */
660  private static JacksonJaxbJsonProvider getJasonProvider() {
661    return JaxbJsonProviderHolder.INSTANCE;
662  }
663
664  /**
665   * @param s the JSON representation of the filter
666   * @return the filter
667   */
668  public static Filter buildFilter(String s) throws Exception {
669    FilterModel model =
670      getJasonProvider().locateMapper(FilterModel.class, MediaType.APPLICATION_JSON_TYPE)
671        .readValue(s, FilterModel.class);
672    return model.build();
673  }
674
675  /**
676   * @param filter the filter
677   * @return the JSON representation of the filter
678   */
679  public static String stringifyFilter(final Filter filter) throws Exception {
680    return getJasonProvider().locateMapper(FilterModel.class, MediaType.APPLICATION_JSON_TYPE)
681      .writeValueAsString(new FilterModel(filter));
682  }
683
684  private static final byte[] COLUMN_DIVIDER = Bytes.toBytes(":");
685
686  /**
687   * @param scan the scan specification
688   */
689  public static ScannerModel fromScan(Scan scan) throws Exception {
690    ScannerModel model = new ScannerModel();
691    model.setStartRow(scan.getStartRow());
692    model.setEndRow(scan.getStopRow());
693    Map<byte[], NavigableSet<byte[]>> families = scan.getFamilyMap();
694    if (families != null) {
695      for (Map.Entry<byte[], NavigableSet<byte[]>> entry : families.entrySet()) {
696        if (entry.getValue() != null) {
697          for (byte[] qualifier : entry.getValue()) {
698            model.addColumn(Bytes.add(entry.getKey(), COLUMN_DIVIDER, qualifier));
699          }
700        } else {
701          model.addColumn(entry.getKey());
702        }
703      }
704    }
705    model.setStartTime(scan.getTimeRange().getMin());
706    model.setEndTime(scan.getTimeRange().getMax());
707    int caching = scan.getCaching();
708    if (caching > 0) {
709      model.setCaching(caching);
710    }
711    int batch = scan.getBatch();
712    if (batch > 0) {
713      model.setBatch(batch);
714    }
715    int maxVersions = scan.getMaxVersions();
716    if (maxVersions > 0) {
717      model.setMaxVersions(maxVersions);
718    }
719    if (scan.getLimit() > 0) {
720      model.setLimit(scan.getLimit());
721    }
722    Filter filter = scan.getFilter();
723    if (filter != null) {
724      model.setFilter(stringifyFilter(filter));
725    }
726    // Add the visbility labels if found in the attributes
727    Authorizations authorizations = scan.getAuthorizations();
728    if (authorizations != null) {
729      List<String> labels = authorizations.getLabels();
730      for (String label : labels) {
731        model.addLabel(label);
732      }
733    }
734    return model;
735  }
736
737  /**
738   * Default constructor
739   */
740  public ScannerModel() {
741  }
742
743  /**
744   * Constructor
745   * @param startRow    the start key of the row-range
746   * @param endRow      the end key of the row-range
747   * @param columns     the columns to scan
748   * @param batch       the number of values to return in batch
749   * @param caching     the number of rows that the scanner will fetch at once
750   * @param endTime     the upper bound on timestamps of values of interest
751   * @param maxVersions the maximum number of versions to return
752   * @param filter      a filter specification (values with timestamps later than this are excluded)
753   */
754  public ScannerModel(byte[] startRow, byte[] endRow, List<byte[]> columns, int batch, int caching,
755    long endTime, int maxVersions, String filter) {
756    super();
757    this.startRow = startRow;
758    this.endRow = endRow;
759    this.columns = columns;
760    this.batch = batch;
761    this.caching = caching;
762    this.endTime = endTime;
763    this.maxVersions = maxVersions;
764    this.filter = filter;
765  }
766
767  /**
768   * Constructor
769   * @param startRow  the start key of the row-range
770   * @param endRow    the end key of the row-range
771   * @param columns   the columns to scan
772   * @param batch     the number of values to return in batch
773   * @param caching   the number of rows that the scanner will fetch at once
774   * @param startTime the lower bound on timestamps of values of interest (values with timestamps
775   *                  earlier than this are excluded)
776   * @param endTime   the upper bound on timestamps of values of interest (values with timestamps
777   *                  later than this are excluded)
778   * @param filter    a filter specification
779   */
780  public ScannerModel(byte[] startRow, byte[] endRow, List<byte[]> columns, int batch, int caching,
781    long startTime, long endTime, String filter) {
782    super();
783    this.startRow = startRow;
784    this.endRow = endRow;
785    this.columns = columns;
786    this.batch = batch;
787    this.caching = caching;
788    this.startTime = startTime;
789    this.endTime = endTime;
790    this.filter = filter;
791  }
792
793  /**
794   * Add a column to the column set
795   * @param column the column name, as &lt;column&gt;(:&lt;qualifier&gt;)?
796   */
797  public void addColumn(byte[] column) {
798    columns.add(column);
799  }
800
801  /**
802   * Add a visibility label to the scan
803   */
804  public void addLabel(String label) {
805    labels.add(label);
806  }
807
808  /** Returns true if a start row was specified */
809  public boolean hasStartRow() {
810    return !Bytes.equals(startRow, HConstants.EMPTY_START_ROW);
811  }
812
813  /** Returns start row */
814  @XmlAttribute
815  public byte[] getStartRow() {
816    return startRow;
817  }
818
819  /** Returns true if an end row was specified */
820  public boolean hasEndRow() {
821    return !Bytes.equals(endRow, HConstants.EMPTY_END_ROW);
822  }
823
824  /** Returns end row */
825  @XmlAttribute
826  public byte[] getEndRow() {
827    return endRow;
828  }
829
830  /** Returns list of columns of interest in column:qualifier format, or empty for all */
831  @XmlElement(name = "column")
832  public List<byte[]> getColumns() {
833    return columns;
834  }
835
836  @XmlElement(name = "labels")
837  public List<String> getLabels() {
838    return labels;
839  }
840
841  /** Returns the number of cells to return in batch */
842  @XmlAttribute
843  public int getBatch() {
844    return batch;
845  }
846
847  /** Returns the number of rows that the scanner to fetch at once */
848  @XmlAttribute
849  public int getCaching() {
850    return caching;
851  }
852
853  /** Returns the limit specification */
854  @XmlAttribute
855  public int getLimit() {
856    return limit;
857  }
858
859  /** Returns true if HFile blocks should be cached on the servers for this scan, false otherwise */
860  @XmlAttribute
861  public boolean getCacheBlocks() {
862    return cacheBlocks;
863  }
864
865  /** Returns the lower bound on timestamps of items of interest */
866  @XmlAttribute
867  public long getStartTime() {
868    return startTime;
869  }
870
871  /** Returns the upper bound on timestamps of items of interest */
872  @XmlAttribute
873  public long getEndTime() {
874    return endTime;
875  }
876
877  /** Returns maximum number of versions to return */
878  @XmlAttribute
879  public int getMaxVersions() {
880    return maxVersions;
881  }
882
883  /** Returns the filter specification */
884  @XmlElement
885  public String getFilter() {
886    return filter;
887  }
888
889  /**
890   * @param startRow start row
891   */
892  public void setStartRow(byte[] startRow) {
893    this.startRow = startRow;
894  }
895
896  /**
897   * @param endRow end row
898   */
899  public void setEndRow(byte[] endRow) {
900    this.endRow = endRow;
901  }
902
903  /**
904   * @param columns list of columns of interest in column:qualifier format, or empty for all
905   */
906  public void setColumns(List<byte[]> columns) {
907    this.columns = columns;
908  }
909
910  /**
911   * @param batch the number of cells to return in batch
912   */
913  public void setBatch(int batch) {
914    this.batch = batch;
915  }
916
917  /**
918   * @param caching the number of rows to fetch at once
919   */
920  public void setCaching(int caching) {
921    this.caching = caching;
922  }
923
924  /**
925   * @param value true if HFile blocks should be cached on the servers for this scan, false
926   *              otherwise
927   */
928  public void setCacheBlocks(boolean value) {
929    this.cacheBlocks = value;
930  }
931
932  /**
933   * @param limit the number of rows can fetch of each scanner at lifetime
934   */
935  public void setLimit(int limit) {
936    this.limit = limit;
937  }
938
939  /**
940   * @param maxVersions maximum number of versions to return
941   */
942  public void setMaxVersions(int maxVersions) {
943    this.maxVersions = maxVersions;
944  }
945
946  /**
947   * @param startTime the lower bound on timestamps of values of interest
948   */
949  public void setStartTime(long startTime) {
950    this.startTime = startTime;
951  }
952
953  /**
954   * @param endTime the upper bound on timestamps of values of interest
955   */
956  public void setEndTime(long endTime) {
957    this.endTime = endTime;
958  }
959
960  /**
961   * @param filter the filter specification
962   */
963  public void setFilter(String filter) {
964    this.filter = filter;
965  }
966
967  @Override
968  public Message messageFromObject() {
969    Scanner.Builder builder = Scanner.newBuilder();
970    if (!Bytes.equals(startRow, HConstants.EMPTY_START_ROW)) {
971      builder.setStartRow(UnsafeByteOperations.unsafeWrap(startRow));
972    }
973    if (!Bytes.equals(endRow, HConstants.EMPTY_START_ROW)) {
974      builder.setEndRow(UnsafeByteOperations.unsafeWrap(endRow));
975    }
976    for (byte[] column : columns) {
977      builder.addColumns(UnsafeByteOperations.unsafeWrap(column));
978    }
979    if (startTime != 0) {
980      builder.setStartTime(startTime);
981    }
982    if (endTime != 0) {
983      builder.setEndTime(endTime);
984    }
985    builder.setBatch(getBatch());
986    if (caching > 0) {
987      builder.setCaching(caching);
988    }
989    if (limit > 0) {
990      builder.setLimit(limit);
991    }
992    builder.setMaxVersions(maxVersions);
993    if (filter != null) {
994      builder.setFilter(filter);
995    }
996    if (labels != null && labels.size() > 0) {
997      for (String label : labels)
998        builder.addLabels(label);
999    }
1000    builder.setCacheBlocks(cacheBlocks);
1001    return builder.build();
1002  }
1003
1004  @Override
1005  public ProtobufMessageHandler getObjectFromMessage(CodedInputStream cis) throws IOException {
1006    Scanner.Builder builder = Scanner.newBuilder();
1007    RestUtil.mergeFrom(builder, cis);
1008    if (builder.hasStartRow()) {
1009      startRow = builder.getStartRow().toByteArray();
1010    }
1011    if (builder.hasEndRow()) {
1012      endRow = builder.getEndRow().toByteArray();
1013    }
1014    for (ByteString column : builder.getColumnsList()) {
1015      addColumn(column.toByteArray());
1016    }
1017    if (builder.hasBatch()) {
1018      batch = builder.getBatch();
1019    }
1020    if (builder.hasCaching()) {
1021      caching = builder.getCaching();
1022    }
1023    if (builder.hasLimit()) {
1024      limit = builder.getLimit();
1025    }
1026    if (builder.hasStartTime()) {
1027      startTime = builder.getStartTime();
1028    }
1029    if (builder.hasEndTime()) {
1030      endTime = builder.getEndTime();
1031    }
1032    if (builder.hasMaxVersions()) {
1033      maxVersions = builder.getMaxVersions();
1034    }
1035    if (builder.hasFilter()) {
1036      filter = builder.getFilter();
1037    }
1038    if (builder.getLabelsList() != null) {
1039      List<String> labels = builder.getLabelsList();
1040      for (String label : labels) {
1041        addLabel(label);
1042      }
1043    }
1044    if (builder.hasCacheBlocks()) {
1045      this.cacheBlocks = builder.getCacheBlocks();
1046    }
1047    return this;
1048  }
1049
1050}