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