You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2017/03/20 14:48:57 UTC

[1/9] accumulo git commit: ACCUMULO-4501 ACCUMULO-96 Added Summarization

Repository: accumulo
Updated Branches:
  refs/heads/master 68ba2ef11 -> 94cdcc4d3


http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java
new file mode 100644
index 0000000..919307b
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java
@@ -0,0 +1,820 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.test.functional;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.counting;
+import static java.util.stream.Collectors.groupingBy;
+import static org.apache.accumulo.test.functional.BasicSummarizer.DELETES_STAT;
+import static org.apache.accumulo.test.functional.BasicSummarizer.MAX_TIMESTAMP_STAT;
+import static org.apache.accumulo.test.functional.BasicSummarizer.MIN_TIMESTAMP_STAT;
+import static org.apache.accumulo.test.functional.BasicSummarizer.TOTAL_STAT;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.DELETES_IGNORED_STAT;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.EMITTED_STAT;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.SEEN_STAT;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.TOO_LONG_STAT;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.TOO_MANY_STAT;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.LongSummaryStatistics;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.regex.PatternSyntaxException;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.admin.CompactionConfig;
+import org.apache.accumulo.core.client.admin.CompactionStrategyConfig;
+import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.client.impl.AccumuloServerException;
+import org.apache.accumulo.core.client.security.SecurityErrorCode;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.client.summary.CounterSummary;
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.Summary;
+import org.apache.accumulo.core.client.summary.Summary.FileStatistics;
+import org.apache.accumulo.core.client.summary.summarizers.FamilySummarizer;
+import org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.Filter;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.tserver.compaction.CompactionPlan;
+import org.apache.accumulo.tserver.compaction.CompactionStrategy;
+import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMap.Builder;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+public class SummaryIT extends AccumuloClusterHarness {
+
+  private LongSummaryStatistics getTimestampStats(final String table, Connector c) throws TableNotFoundException {
+    try (Scanner scanner = c.createScanner(table, Authorizations.EMPTY)) {
+      Stream<Entry<Key,Value>> stream = StreamSupport.stream(scanner.spliterator(), false);
+      LongSummaryStatistics stats = stream.mapToLong(e -> e.getKey().getTimestamp()).summaryStatistics();
+      return stats;
+    }
+  }
+
+  private LongSummaryStatistics getTimestampStats(final String table, Connector c, String startRow, String endRow) throws TableNotFoundException {
+    try (Scanner scanner = c.createScanner(table, Authorizations.EMPTY)) {
+      scanner.setRange(new Range(startRow, false, endRow, true));
+      Stream<Entry<Key,Value>> stream = StreamSupport.stream(scanner.spliterator(), false);
+      LongSummaryStatistics stats = stream.mapToLong(e -> e.getKey().getTimestamp()).summaryStatistics();
+      return stats;
+    }
+  }
+
+  private void checkSummaries(Collection<Summary> summaries, SummarizerConfiguration sc, int total, int missing, int extra, Object... kvs) {
+    Summary summary = Iterables.getOnlyElement(summaries);
+    Assert.assertEquals("total wrong", total, summary.getFileStatistics().getTotal());
+    Assert.assertEquals("missing wrong", missing, summary.getFileStatistics().getMissing());
+    Assert.assertEquals("extra wrong", extra, summary.getFileStatistics().getExtra());
+    Assert.assertEquals("deleted wrong", 0, summary.getFileStatistics().getDeleted());
+    Assert.assertEquals(sc, summary.getSummarizerConfiguration());
+    Map<String,Long> expected = new HashMap<>();
+    for (int i = 0; i < kvs.length; i += 2) {
+      expected.put((String) kvs[i], (Long) kvs[i + 1]);
+    }
+    Assert.assertEquals(expected, summary.getStatistics());
+  }
+
+  private void addSplits(final String table, Connector c, String... splits) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    c.tableOperations().addSplits(table, new TreeSet<Text>(Lists.transform(Arrays.asList(splits), Text::new)));
+  }
+
+  @Test
+  public void basicSummaryTest() throws Exception {
+    final String table = getUniqueNames(1)[0];
+    Connector c = getConnector();
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(BasicSummarizer.class.getName()).build();
+    ntc.enableSummarization(sc1);
+    c.tableOperations().create(table, ntc);
+
+    BatchWriter bw = writeData(table, c);
+
+    Collection<Summary> summaries = c.tableOperations().summaries(table).flush(false).retrieve();
+    Assert.assertEquals(0, summaries.size());
+
+    LongSummaryStatistics stats = getTimestampStats(table, c);
+
+    summaries = c.tableOperations().summaries(table).flush(true).retrieve();
+    checkSummaries(summaries, sc1, 1, 0, 0, TOTAL_STAT, 100_000l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 0l);
+
+    Mutation m = new Mutation(String.format("r%09x", 999));
+    m.put("f1", "q1", "999-0");
+    m.putDelete("f1", "q2");
+    bw.addMutation(m);
+    bw.flush();
+
+    c.tableOperations().flush(table, null, null, true);
+
+    stats = getTimestampStats(table, c);
+
+    summaries = c.tableOperations().summaries(table).retrieve();
+
+    checkSummaries(summaries, sc1, 2, 0, 0, TOTAL_STAT, 100_002l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 1l);
+
+    bw.close();
+
+    c.tableOperations().compact(table, new CompactionConfig().setWait(true));
+
+    summaries = c.tableOperations().summaries(table).retrieve();
+    checkSummaries(summaries, sc1, 1, 0, 0, TOTAL_STAT, 100_000l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 0l);
+
+    // split tablet into two
+    String sp1 = String.format("r%09x", 50_000);
+    addSplits(table, c, sp1);
+
+    summaries = c.tableOperations().summaries(table).retrieve();
+
+    checkSummaries(summaries, sc1, 1, 0, 0, TOTAL_STAT, 100_000l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 0l);
+
+    // compact 2nd tablet
+    c.tableOperations().compact(table, new CompactionConfig().setStartRow(new Text(sp1)).setWait(true));
+
+    summaries = c.tableOperations().summaries(table).retrieve();
+    checkSummaries(summaries, sc1, 2, 0, 1, TOTAL_STAT, 113_999l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 0l);
+
+    // get summaries for first tablet
+    stats = getTimestampStats(table, c, sp1, null);
+    summaries = c.tableOperations().summaries(table).startRow(sp1).retrieve();
+    checkSummaries(summaries, sc1, 1, 0, 0, TOTAL_STAT, 49_999l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 0l);
+
+    // compact all tablets and regenerate all summaries
+    c.tableOperations().compact(table, new CompactionConfig());
+
+    summaries = c.tableOperations().summaries(table).retrieve();
+    stats = getTimestampStats(table, c);
+    checkSummaries(summaries, sc1, 2, 0, 0, TOTAL_STAT, 100_000l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 0l);
+
+    summaries = c.tableOperations().summaries(table).startRow(String.format("r%09x", 75_000)).endRow(String.format("r%09x", 80_000)).retrieve();
+    Summary summary = Iterables.getOnlyElement(summaries);
+    Assert.assertEquals(1, summary.getFileStatistics().getTotal());
+    Assert.assertEquals(1, summary.getFileStatistics().getExtra());
+    long total = summary.getStatistics().get(TOTAL_STAT);
+    Assert.assertTrue("Total " + total + " out of expected range", total > 0 && total <= 10_000);
+
+    // test adding and removing
+    c.tableOperations().removeSummarizers(table, sc -> sc.getClassName().contains("foo"));
+
+    List<SummarizerConfiguration> summarizers = c.tableOperations().listSummarizers(table);
+    Assert.assertEquals(1, summarizers.size());
+    Assert.assertTrue(summarizers.contains(sc1));
+
+    c.tableOperations().removeSummarizers(table, sc -> sc.getClassName().equals(BasicSummarizer.class.getName()));
+    summarizers = c.tableOperations().listSummarizers(table);
+    Assert.assertEquals(0, summarizers.size());
+
+    c.tableOperations().compact(table, new CompactionConfig().setWait(true));
+
+    summaries = c.tableOperations().summaries(table).retrieve();
+    Assert.assertEquals(0, summaries.size());
+
+    c.tableOperations().addSummarizers(table, sc1);
+    c.tableOperations().compact(table, new CompactionConfig().setWait(true));
+    summaries = c.tableOperations().summaries(table).retrieve();
+    checkSummaries(summaries, sc1, 2, 0, 0, TOTAL_STAT, 100_000l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 0l);
+  }
+
+  private BatchWriter writeData(final String table, Connector c) throws TableNotFoundException, MutationsRejectedException {
+    BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig());
+    for (int i = 0; i < 100_000; i++) {
+      Mutation m = new Mutation(String.format("r%09x", i));
+      m.put("f1", "q1", "" + i);
+      bw.addMutation(m);
+    }
+    bw.flush();
+    return bw;
+  }
+
+  public static class KeySizeSummarizer implements Summarizer {
+
+    @Override
+    public Collector collector(SummarizerConfiguration sc) {
+      return new Collector() {
+        private int maxLen = Integer.parseInt(sc.getOptions().getOrDefault("maxLen", "1024"));
+        private long[] lengths = new long[maxLen];
+        private long overMax = 0;
+
+        @Override
+        public void accept(Key k, Value v) {
+          int size = k.getSize();
+          if (size >= maxLen) {
+            overMax++;
+          } else {
+            lengths[size]++;
+          }
+        }
+
+        @Override
+        public void summarize(StatisticConsumer sc) {
+          if (overMax > 0) {
+            sc.accept("len>=" + maxLen, overMax);
+          }
+          for (int i = 0; i < lengths.length; i++) {
+            if (lengths[i] > 0) {
+              sc.accept("len=" + i, lengths[i]);
+            }
+          }
+        }
+
+      };
+    }
+
+    @Override
+    public Combiner combiner(SummarizerConfiguration sc) {
+      return (m1, m2) -> m2.forEach((k, v) -> m1.merge(k, v, Long::sum));
+    }
+  }
+
+  private static void checkSummary(Collection<Summary> summaries, SummarizerConfiguration sc, Object... stats) {
+    Map<String,Long> expected = new HashMap<>();
+    for (int i = 0; i < stats.length; i += 2) {
+      expected.put((String) stats[i], (Long) stats[i + 1]);
+    }
+
+    for (Summary summary : summaries) {
+      if (summary.getSummarizerConfiguration().equals(sc)) {
+        Assert.assertEquals(expected, summary.getStatistics());
+        return;
+      }
+    }
+
+    Assert.fail("Did not find summary with config : " + sc);
+  }
+
+  @Test
+  public void selectionTest() throws Exception {
+    final String table = getUniqueNames(1)[0];
+    Connector c = getConnector();
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(BasicSummarizer.class).build();
+    SummarizerConfiguration sc2 = SummarizerConfiguration.builder(KeySizeSummarizer.class).addOption("maxLen", "512").build();
+    ntc.enableSummarization(sc1, sc2);
+    c.tableOperations().create(table, ntc);
+
+    BatchWriter bw = writeData(table, c);
+    bw.close();
+
+    c.tableOperations().flush(table, null, null, true);
+
+    LongSummaryStatistics stats = getTimestampStats(table, c);
+
+    Collection<Summary> summaries = c.tableOperations().summaries(table).withConfiguration(sc2).retrieve();
+    Assert.assertEquals(1, summaries.size());
+    checkSummary(summaries, sc2, "len=14", 100_000l);
+
+    summaries = c.tableOperations().summaries(table).withConfiguration(sc1).retrieve();
+    Assert.assertEquals(1, summaries.size());
+    checkSummary(summaries, sc1, TOTAL_STAT, 100_000l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 0l);
+
+    // retrieve a non-existant summary
+    SummarizerConfiguration sc3 = SummarizerConfiguration.builder(KeySizeSummarizer.class.getName()).addOption("maxLen", "256").build();
+    summaries = c.tableOperations().summaries(table).withConfiguration(sc3).retrieve();
+    Assert.assertEquals(0, summaries.size());
+
+    summaries = c.tableOperations().summaries(table).withConfiguration(sc1, sc2).retrieve();
+    Assert.assertEquals(2, summaries.size());
+    checkSummary(summaries, sc1, TOTAL_STAT, 100_000l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 0l);
+    checkSummary(summaries, sc2, "len=14", 100_000l);
+
+    summaries = c.tableOperations().summaries(table).retrieve();
+    Assert.assertEquals(2, summaries.size());
+    checkSummary(summaries, sc1, TOTAL_STAT, 100_000l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 0l);
+    checkSummary(summaries, sc2, "len=14", 100_000l);
+
+    summaries = c.tableOperations().summaries(table).withMatchingConfiguration(".*BasicSummarizer \\{\\}.*").retrieve();
+    Assert.assertEquals(1, summaries.size());
+    checkSummary(summaries, sc1, TOTAL_STAT, 100_000l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 0l);
+
+    summaries = c.tableOperations().summaries(table).withMatchingConfiguration(".*KeySizeSummarizer \\{maxLen=512\\}.*").retrieve();
+    Assert.assertEquals(1, summaries.size());
+    checkSummary(summaries, sc2, "len=14", 100_000l);
+
+    summaries = c.tableOperations().summaries(table).withMatchingConfiguration(".*KeySizeSummarizer \\{maxLen=256\\}.*").retrieve();
+    Assert.assertEquals(0, summaries.size());
+
+    summaries = c.tableOperations().summaries(table).withMatchingConfiguration(".*BasicSummarizer \\{\\}.*").withConfiguration(sc2).retrieve();
+    Assert.assertEquals(2, summaries.size());
+    checkSummary(summaries, sc1, TOTAL_STAT, 100_000l, MIN_TIMESTAMP_STAT, stats.getMin(), MAX_TIMESTAMP_STAT, stats.getMax(), DELETES_STAT, 0l);
+    checkSummary(summaries, sc2, "len=14", 100_000l);
+
+    // Ensure a bad regex fails fast.
+    try {
+      summaries = c.tableOperations().summaries(table).withMatchingConfiguration(".*KeySizeSummarizer {maxLen=256}.*").retrieve();
+      Assert.fail("Bad regex should have caused exception");
+    } catch (PatternSyntaxException e) {}
+  }
+
+  /**
+   * A summarizer that counts the number of times {@code foo} and {@code bar} occur in the row.
+   */
+  public static class FooCounter implements Summarizer {
+
+    @Override
+    public Collector collector(SummarizerConfiguration sc) {
+      return new Collector() {
+
+        long foos = 0;
+        long bars = 0;
+
+        @Override
+        public void accept(Key k, Value v) {
+          String row = k.getRowData().toString();
+
+          if (row.contains("foo")) {
+            foos++;
+          } else if (row.contains("bar")) {
+            bars++;
+          }
+        }
+
+        @Override
+        public void summarize(StatisticConsumer sc) {
+          sc.accept("foos", foos);
+          sc.accept("bars", bars);
+        }
+
+      };
+    }
+
+    @Override
+    public Combiner combiner(SummarizerConfiguration sc) {
+      return (m1, m2) -> m2.forEach((k, v) -> m1.merge(k, v, Long::sum));
+    }
+  }
+
+  /**
+   * An Accumulo iterator that filters out entries where the row contains {@code foo}.
+   */
+  public static class FooFilter extends Filter {
+    @Override
+    public boolean accept(Key k, Value v) {
+      return !k.getRowData().toString().contains("foo");
+    }
+  }
+
+  /**
+   * A compaction strategy that intitiates a compaction when {@code foo} occurs more than {@code bar} in the data. The {@link FooCounter} summary data is used
+   * to make the determination.
+   */
+  public static class FooCS extends CompactionStrategy {
+
+    private boolean compact = false;
+
+    @Override
+    public boolean shouldCompact(MajorCompactionRequest request) throws IOException {
+      return true;
+    }
+
+    public void gatherInformation(MajorCompactionRequest request) throws IOException {
+      List<Summary> summaries = request.getSummaries(request.getFiles().keySet(), conf -> conf.getClassName().contains("FooCounter"));
+      if (summaries.size() == 1) {
+        Summary summary = summaries.get(0);
+        Long foos = summary.getStatistics().getOrDefault("foos", 0l);
+        Long bars = summary.getStatistics().getOrDefault("bars", 0l);
+
+        compact = foos > bars;
+      }
+    }
+
+    @Override
+    public CompactionPlan getCompactionPlan(MajorCompactionRequest request) throws IOException {
+      if (compact) {
+        CompactionPlan cp = new CompactionPlan();
+        cp.inputFiles.addAll(request.getFiles().keySet());
+        return cp;
+      }
+      return null;
+    }
+
+  }
+
+  @Test
+  public void compactionTest() throws Exception {
+    final String table = getUniqueNames(1)[0];
+    Connector c = getConnector();
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(FooCounter.class.getName()).build();
+    ntc.enableSummarization(sc1);
+    c.tableOperations().create(table, ntc);
+
+    try (BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig())) {
+      write(bw, "bar1", "f1", "q1", "v1");
+      write(bw, "bar2", "f1", "q1", "v2");
+      write(bw, "foo1", "f1", "q1", "v3");
+    }
+
+    // Create a compaction config that will filter out foos if there are too many. Uses summary data to know if there are too many foos.
+    CompactionStrategyConfig csc = new CompactionStrategyConfig(FooCS.class.getName());
+    List<IteratorSetting> iterators = Collections.singletonList(new IteratorSetting(100, FooFilter.class));
+    CompactionConfig compactConfig = new CompactionConfig().setFlush(true).setCompactionStrategy(csc).setIterators(iterators).setWait(true);
+
+    // this compaction should make no changes because there are less foos than bars
+    c.tableOperations().compact(table, compactConfig);
+
+    try (Scanner scanner = c.createScanner(table, Authorizations.EMPTY)) {
+      Stream<Entry<Key,Value>> stream = StreamSupport.stream(scanner.spliterator(), false);
+      Map<String,Long> counts = stream.map(e -> e.getKey().getRowData().toString()) // convert to row
+          .map(r -> r.replaceAll("[0-9]+", "")) // strip numbers off row
+          .collect(groupingBy(identity(), counting())); // count different row types
+      Assert.assertEquals(1l, (long) counts.getOrDefault("foo", 0l));
+      Assert.assertEquals(2l, (long) counts.getOrDefault("bar", 0l));
+      Assert.assertEquals(2, counts.size());
+    }
+
+    try (BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig())) {
+      write(bw, "foo2", "f1", "q1", "v4");
+      write(bw, "foo3", "f1", "q1", "v5");
+      write(bw, "foo4", "f1", "q1", "v6");
+    }
+
+    // this compaction should remove all foos because there are more foos than bars
+    c.tableOperations().compact(table, compactConfig);
+
+    try (Scanner scanner = c.createScanner(table, Authorizations.EMPTY)) {
+      Stream<Entry<Key,Value>> stream = StreamSupport.stream(scanner.spliterator(), false);
+      Map<String,Long> counts = stream.map(e -> e.getKey().getRowData().toString()) // convert to row
+          .map(r -> r.replaceAll("[0-9]+", "")) // strip numbers off row
+          .collect(groupingBy(identity(), counting())); // count different row types
+      Assert.assertEquals(0l, (long) counts.getOrDefault("foo", 0l));
+      Assert.assertEquals(2l, (long) counts.getOrDefault("bar", 0l));
+      Assert.assertEquals(1, counts.size());
+    }
+  }
+
+  public static class BuggySummarizer extends FooCounter {
+    @Override
+    public Combiner combiner(SummarizerConfiguration sc) {
+      return (m1, m2) -> {
+        throw new NullPointerException();
+      };
+    }
+  }
+
+  @Test
+  public void testBuggySummarizer() throws Exception {
+    final String table = getUniqueNames(1)[0];
+    Connector c = getConnector();
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(BuggySummarizer.class).build();
+    ntc.enableSummarization(sc1);
+    c.tableOperations().create(table, ntc);
+
+    // add a single split so that summary stats merge is forced
+    c.tableOperations().addSplits(table, new TreeSet<>(Collections.singleton(new Text("g"))));
+
+    try (BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig())) {
+      write(bw, "bar1", "f1", "q1", "v1");
+      write(bw, "bar2", "f1", "q1", "v2");
+      write(bw, "foo1", "f1", "q1", "v3");
+    }
+
+    c.tableOperations().flush(table, null, null, true);
+    try {
+      c.tableOperations().summaries(table).retrieve();
+      Assert.fail("Expected server side failure and did not see it");
+    } catch (AccumuloServerException ase) {}
+
+  }
+
+  @Test
+  public void testPermissions() throws Exception {
+    final String table = getUniqueNames(1)[0];
+    Connector c = getConnector();
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(FooCounter.class).build();
+    ntc.enableSummarization(sc1);
+    c.tableOperations().create(table, ntc);
+
+    try (BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig())) {
+      write(bw, "bar1", "f1", "q1", "v1");
+      write(bw, "bar2", "f1", "q1", "v2");
+      write(bw, "foo1", "f1", "q1", "v3");
+    }
+
+    c.tableOperations().flush(table, null, null, true);
+
+    PasswordToken passTok = new PasswordToken("letmesee");
+    c.securityOperations().createLocalUser("user1", passTok);
+
+    String instanceName = c.getInstance().getInstanceName();
+    String zookeepers = c.getInstance().getZooKeepers();
+    Connector c2 = new ZooKeeperInstance(instanceName, zookeepers).getConnector("user1", passTok);
+    try {
+      c2.tableOperations().summaries(table).retrieve();
+      Assert.fail("Expected operation to fail because user does not have permssion to get summaries");
+    } catch (AccumuloSecurityException ase) {
+      Assert.assertEquals(SecurityErrorCode.PERMISSION_DENIED, ase.getSecurityErrorCode());
+    }
+
+    c.securityOperations().grantTablePermission("user1", table, TablePermission.GET_SUMMARIES);
+
+    int tries = 0;
+    while (tries < 10) {
+      try {
+        Summary summary = c2.tableOperations().summaries(table).retrieve().get(0);
+        Assert.assertEquals(2, summary.getStatistics().size());
+        Assert.assertEquals(2l, (long) summary.getStatistics().getOrDefault("bars", 0l));
+        Assert.assertEquals(1l, (long) summary.getStatistics().getOrDefault("foos", 0l));
+        break;
+      } catch (AccumuloSecurityException ase) {
+        UtilWaitThread.sleep(500);
+        tries++;
+      }
+    }
+  }
+
+  public static class BigSummarizer implements Summarizer {
+    @Override
+    public Collector collector(SummarizerConfiguration sc) {
+      return new Collector() {
+        private int num = 10;
+
+        @Override
+        public void accept(Key k, Value v) {
+          if (k.getRowData().toString().contains("large")) {
+            num = 100_000;
+          }
+        }
+
+        @Override
+        public void summarize(StatisticConsumer sc) {
+          for (int i = 0; i < num; i++) {
+            sc.accept(String.format("%09x", i), i * 19);
+          }
+        }
+      };
+    }
+
+    @Override
+    public Combiner combiner(SummarizerConfiguration sc) {
+      return (m1, m2) -> m2.forEach((k, v) -> m1.merge(k, v, Long::sum));
+    }
+  }
+
+  @Test
+  public void tooLargeTest() throws Exception {
+    final String table = getUniqueNames(1)[0];
+    Connector c = getConnector();
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(BigSummarizer.class).build();
+    ntc.enableSummarization(sc1);
+    c.tableOperations().create(table, ntc);
+
+    try (BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig())) {
+      write(bw, "a_large", "f1", "q1", "v1");
+      write(bw, "v_small", "f1", "q1", "v2");
+    }
+
+    c.tableOperations().flush(table, null, null, true);
+    Summary summary = c.tableOperations().summaries(table).retrieve().get(0);
+    Assert.assertEquals(1, summary.getFileStatistics().getLarge());
+    Assert.assertEquals(0, summary.getFileStatistics().getMissing());
+    Assert.assertEquals(0, summary.getFileStatistics().getExtra());
+    Assert.assertEquals(0, summary.getFileStatistics().getDeleted());
+    Assert.assertEquals(1, summary.getFileStatistics().getInaccurate());
+    Assert.assertEquals(1, summary.getFileStatistics().getTotal());
+    Assert.assertEquals(Collections.emptyMap(), summary.getStatistics());
+
+    // create situation where one tablet has summary data and one does not because the summary data was too large
+    c.tableOperations().addSplits(table, new TreeSet<>(Collections.singleton(new Text("m"))));
+    c.tableOperations().compact(table, new CompactionConfig().setWait(true));
+
+    summary = c.tableOperations().summaries(table).retrieve().get(0);
+    Assert.assertEquals(1, summary.getFileStatistics().getLarge());
+    Assert.assertEquals(0, summary.getFileStatistics().getMissing());
+    Assert.assertEquals(0, summary.getFileStatistics().getExtra());
+    Assert.assertEquals(0, summary.getFileStatistics().getDeleted());
+    Assert.assertEquals(1, summary.getFileStatistics().getInaccurate());
+    Assert.assertEquals(2, summary.getFileStatistics().getTotal());
+
+    HashMap<String,Long> expected = new HashMap<>();
+    for (int i = 0; i < 10; i++) {
+      expected.put(String.format("%09x", i), i * 19l);
+    }
+
+    Assert.assertEquals(expected, summary.getStatistics());
+  }
+
+  private void write(BatchWriter bw, String row, String family, String qualifier, String value) throws MutationsRejectedException {
+    Mutation m1 = new Mutation(row);
+    m1.put(family, qualifier, value);
+    bw.addMutation(m1);
+  }
+
+  private void write(BatchWriter bw, Map<Key,Value> expected, String row, String family, String qualifier, long ts, String value)
+      throws MutationsRejectedException {
+    Mutation m1 = new Mutation(row);
+    m1.put(family, qualifier, ts, value);
+    bw.addMutation(m1);
+    expected.put(Key.builder().row(row).family(family).qualifier(qualifier).timestamp(ts).build(), new Value(value));
+  }
+
+  private Map<String,Long> nm(Object... entries) {
+    Builder<String,Long> imb = ImmutableMap.builder();
+    for (int i = 0; i < entries.length; i += 2) {
+      imb.put((String) entries[i], (Long) entries[i + 1]);
+    }
+    return imb.build();
+  }
+
+  @Test
+  public void testLocalityGroups() throws Exception {
+    final String table = getUniqueNames(1)[0];
+    Connector c = getConnector();
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(FamilySummarizer.class).build();
+    SummarizerConfiguration sc2 = SummarizerConfiguration.builder(BasicSummarizer.class).build();
+    ntc.enableSummarization(sc1, sc2);
+    c.tableOperations().create(table, ntc);
+
+    Map<String,Set<Text>> lgroups = new HashMap<>();
+    lgroups.put("lg1", ImmutableSet.of(new Text("chocolate"), new Text("coffee")));
+    lgroups.put("lg2", ImmutableSet.of(new Text(" broccoli "), new Text("cabbage")));
+
+    c.tableOperations().setLocalityGroups(table, lgroups);
+
+    Map<Key,Value> expected = new HashMap<>();
+    try (BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig())) {
+      write(bw, expected, "order:001", "chocolate", "dark", 3l, "99kg");
+      write(bw, expected, "order:001", "chocolate", "light", 4l, "94kg");
+      write(bw, expected, "order:001", "coffee", "dark", 5l, "33kg");
+      write(bw, expected, "order:001", "broccoli", "crowns", 6l, "2kg");
+      write(bw, expected, "order:001", "cheddar", "canadian", 7l, "40kg");
+
+      write(bw, expected, "order:653", "chocolate", "dark", 3l, "3kg");
+      write(bw, expected, "order:653", "chocolate", "light", 4l, "4kg");
+      write(bw, expected, "order:653", "coffee", "dark", 5l, "2kg");
+      write(bw, expected, "order:653", "broccoli", "crowns", 6l, "105kg");
+      write(bw, expected, "order:653", "cabbage", "heads", 7l, "199kg");
+      write(bw, expected, "order:653", "cheddar", "canadian", 8l, "43kg");
+    }
+
+    List<Summary> summaries = c.tableOperations().summaries(table).flush(true).retrieve();
+    Assert.assertEquals(2, summaries.stream().map(Summary::getSummarizerConfiguration).distinct().count());
+    for (Summary summary : summaries) {
+      if (summary.getSummarizerConfiguration().equals(sc1)) {
+        Map<String,Long> expectedStats = nm("c:chocolate", 4l, "c:coffee", 2l, "c:broccoli", 2l, "c:cheddar", 2l, "c:cabbage", 1l, TOO_LONG_STAT, 0l,
+            TOO_MANY_STAT, 0l, SEEN_STAT, 11l, EMITTED_STAT, 11l, DELETES_IGNORED_STAT, 0l);
+        Assert.assertEquals(expectedStats, summary.getStatistics());
+        Assert.assertEquals(0, summary.getFileStatistics().getInaccurate());
+        Assert.assertEquals(1, summary.getFileStatistics().getTotal());
+      } else if (summary.getSummarizerConfiguration().equals(sc2)) {
+        Map<String,Long> expectedStats = nm(DELETES_STAT, 0l, TOTAL_STAT, 11l, MIN_TIMESTAMP_STAT, 3l, MAX_TIMESTAMP_STAT, 8l);
+        Assert.assertEquals(expectedStats, summary.getStatistics());
+        Assert.assertEquals(0, summary.getFileStatistics().getInaccurate());
+        Assert.assertEquals(1, summary.getFileStatistics().getTotal());
+      } else {
+        Assert.fail("unexpected summary config " + summary.getSummarizerConfiguration());
+      }
+    }
+
+    Map<Key,Value> actual = new HashMap<>();
+    c.createScanner(table, Authorizations.EMPTY).forEach(e -> actual.put(e.getKey(), e.getValue()));
+
+    Assert.assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testExceptions() throws Exception {
+    Connector c = getConnector();
+
+    try {
+      c.tableOperations().summaries("foo").retrieve();
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+
+    try {
+      c.tableOperations().addSummarizers("foo", SummarizerConfiguration.builder(VisibilitySummarizer.class).build());
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+
+    try {
+      c.tableOperations().listSummarizers("foo");
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+
+    try {
+      c.tableOperations().removeSummarizers("foo", sc -> true);
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+
+    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(FamilySummarizer.class).setPropertyId("p1").build();
+    SummarizerConfiguration sc2 = SummarizerConfiguration.builder(VisibilitySummarizer.class).setPropertyId("p1").build();
+
+    c.tableOperations().create("foo");
+    c.tableOperations().addSummarizers("foo", sc1);
+    c.tableOperations().addSummarizers("foo", sc1);
+    try {
+      // adding second summarizer with same id should fail
+      c.tableOperations().addSummarizers("foo", sc2);
+      Assert.fail();
+    } catch (IllegalArgumentException e) {}
+
+    c.tableOperations().removeSummarizers("foo", sc -> true);
+    Assert.assertEquals(0, c.tableOperations().listSummarizers("foo").size());
+
+    try {
+      // adding two summarizers at the same time with same id should fail
+      c.tableOperations().addSummarizers("foo", sc1, sc2);
+      Assert.fail();
+    } catch (IllegalArgumentException e) {}
+    Assert.assertEquals(0, c.tableOperations().listSummarizers("foo").size());
+
+    c.tableOperations().offline("foo", true);
+    try {
+      c.tableOperations().summaries("foo").retrieve();
+      Assert.fail();
+    } catch (TableOfflineException e) {}
+  }
+
+  @Test
+  public void testManyFiles() throws Exception {
+    final String table = getUniqueNames(1)[0];
+    Connector c = getConnector();
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    ntc.enableSummarization(SummarizerConfiguration.builder(FamilySummarizer.class).build());
+    c.tableOperations().create(table, ntc);
+
+    Random rand = new Random(42);
+    int q = 0;
+
+    SortedSet<Text> partitionKeys = new TreeSet<>();
+    for (int split = 100_000; split < 1_000_000; split += 100_000) {
+      partitionKeys.add(new Text(String.format("%06d", split)));
+    }
+    c.tableOperations().addSplits(table, partitionKeys);
+    Map<String,Long> famCounts = new HashMap<>();
+
+    for (int t = 0; t < 20; t++) {
+      // this loop should cause a varying number of files and compactions
+      try (BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig())) {
+        for (int i = 0; i < 10000; i++) {
+          String row = String.format("%06d", rand.nextInt(1_000_000));
+          String fam = String.format("%03d", rand.nextInt(100));
+          String qual = String.format("%06d", q++);
+          write(bw, row, fam, qual, "val");
+          famCounts.merge(fam, 1L, Long::sum);
+        }
+      }
+
+      List<Summary> summaries = c.tableOperations().summaries(table).flush(true).retrieve();
+      Assert.assertEquals(1, summaries.size());
+      CounterSummary cs = new CounterSummary(summaries.get(0));
+      Assert.assertEquals(famCounts, cs.getCounters());
+      FileStatistics fileStats = summaries.get(0).getFileStatistics();
+      Assert.assertEquals(0, fileStats.getInaccurate());
+      Assert.assertTrue("Saw " + fileStats.getTotal() + " files expected >=10", fileStats.getTotal() >= 10);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/test/src/main/java/org/apache/accumulo/test/functional/TooManyDeletesIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TooManyDeletesIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TooManyDeletesIT.java
new file mode 100644
index 0000000..f0e6aa3
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TooManyDeletesIT.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.test.functional;
+
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.Summary;
+import org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.tserver.compaction.strategies.TooManyDeletesCompactionStrategy;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TooManyDeletesIT extends AccumuloClusterHarness {
+  @Test
+  public void tooManyDeletesCompactionStrategyIT() throws Exception {
+    Connector c = getConnector();
+
+    String table = getUniqueNames(1)[0];
+
+    SummarizerConfiguration sc = SummarizerConfiguration.builder(DeletesSummarizer.class).build();
+
+    // TODO open issue about programatic config of compaction strategies
+
+    NewTableConfiguration ntc = new NewTableConfiguration().enableSummarization(sc);
+    HashMap<String,String> props = new HashMap<>();
+    props.put(Property.TABLE_COMPACTION_STRATEGY.getKey(), TooManyDeletesCompactionStrategy.class.getName());
+    props.put(Property.TABLE_COMPACTION_STRATEGY_PREFIX.getKey() + TooManyDeletesCompactionStrategy.THRESHOLD_OPT, ".25");
+    // ensure compaction does not happen because of the number of files
+    props.put(Property.TABLE_MAJC_RATIO.getKey(), "10");
+    ntc.setProperties(props);
+
+    c.tableOperations().create(table, ntc);
+
+    try (BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig())) {
+      for (int i = 0; i < 1000; i++) {
+        Mutation m = new Mutation("row" + i);
+        m.put("f", "q", "v" + i);
+        bw.addMutation(m);
+      }
+    }
+
+    List<Summary> summaries = c.tableOperations().summaries(table).flush(true).withConfiguration(sc).retrieve();
+    Assert.assertEquals(1, summaries.size());
+
+    Summary summary = summaries.get(0);
+
+    Assert.assertEquals(1000l, (long) summary.getStatistics().get(DeletesSummarizer.TOTAL_STAT));
+    Assert.assertEquals(0l, (long) summary.getStatistics().get(DeletesSummarizer.DELETES_STAT));
+
+    try (BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig())) {
+      for (int i = 0; i < 100; i++) {
+        Mutation m = new Mutation("row" + i);
+        m.putDelete("f", "q");
+        bw.addMutation(m);
+      }
+    }
+
+    summaries = c.tableOperations().summaries(table).flush(true).withConfiguration(sc).retrieve();
+    Assert.assertEquals(1, summaries.size());
+
+    summary = summaries.get(0);
+
+    Assert.assertEquals(1100l, (long) summary.getStatistics().get(DeletesSummarizer.TOTAL_STAT));
+    Assert.assertEquals(100l, (long) summary.getStatistics().get(DeletesSummarizer.DELETES_STAT));
+
+    try (BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig())) {
+      for (int i = 100; i < 300; i++) {
+        Mutation m = new Mutation("row" + i);
+        m.putDelete("f", "q");
+        bw.addMutation(m);
+      }
+    }
+
+    // after a flush occurs Accumulo will check if a major compaction is needed. This check should call the compaction strategy, which should decide to compact
+    // all files based on the number of deletes.
+    c.tableOperations().flush(table, null, null, true);
+
+    // wait for the compaction to happen
+    while (true) {
+      // the flush should cause
+      summaries = c.tableOperations().summaries(table).flush(false).withConfiguration(sc).retrieve();
+      Assert.assertEquals(1, summaries.size());
+
+      summary = summaries.get(0);
+      long total = summary.getStatistics().get(DeletesSummarizer.TOTAL_STAT);
+      long deletes = summary.getStatistics().get(DeletesSummarizer.DELETES_STAT);
+
+      if (total == 700 && deletes == 0) {
+        // a compaction was triggered based on the number of deletes
+        break;
+      }
+
+      UtilWaitThread.sleep(50);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
index f392b16..299b0b8 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.test.performance.thrift;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -30,6 +32,7 @@ import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Range;
@@ -48,6 +51,9 @@ import org.apache.accumulo.core.data.thrift.TConstraintViolationSummary;
 import org.apache.accumulo.core.data.thrift.TKeyExtent;
 import org.apache.accumulo.core.data.thrift.TMutation;
 import org.apache.accumulo.core.data.thrift.TRange;
+import org.apache.accumulo.core.data.thrift.TRowRange;
+import org.apache.accumulo.core.data.thrift.TSummaries;
+import org.apache.accumulo.core.data.thrift.TSummaryRequest;
 import org.apache.accumulo.core.data.thrift.UpdateErrors;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.security.thrift.TCredentials;
@@ -56,6 +62,7 @@ import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.TDurability;
 import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
+import org.apache.accumulo.core.tabletserver.thrift.TUnloadTabletGoal;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Processor;
@@ -78,9 +85,6 @@ import org.apache.thrift.TException;
 import com.beust.jcommander.Parameter;
 import com.google.common.net.HostAndPort;
 
-import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
-import org.apache.accumulo.core.tabletserver.thrift.TUnloadTabletGoal;
-
 /**
  * The purpose of this class is to server as fake tserver that is a data sink like /dev/null. NullTserver modifies the metadata location entries for a table to
  * point to it. This allows thrift performance to be measured by running any client code that writes to a table.
@@ -235,6 +239,29 @@ public class NullTserver {
 
     @Override
     public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException { }
+
+    @Override
+    public TSummaries startGetSummaries(TInfo tinfo, TCredentials credentials, TSummaryRequest request)
+        throws ThriftSecurityException, ThriftTableOperationException, NoSuchScanIDException, TException {
+      return null;
+    }
+
+    @Override
+    public TSummaries startGetSummariesForPartition(TInfo tinfo, TCredentials credentials, TSummaryRequest request, int modulus, int remainder)
+        throws ThriftSecurityException, NoSuchScanIDException, TException {
+      return null;
+    }
+
+    @Override
+    public TSummaries startGetSummariesFromFiles(TInfo tinfo, TCredentials credentials, TSummaryRequest request, Map<String,List<TRowRange>> files)
+        throws ThriftSecurityException, NoSuchScanIDException, TException {
+      return null;
+    }
+
+    @Override
+    public TSummaries contiuneGetSummaries(TInfo tinfo, long sessionId) throws NoSuchScanIDException, TException {
+      return null;
+    }
   }
 
   static class Opts extends Help {


[7/9] accumulo git commit: ACCUMULO-4501 ACCUMULO-96 Added Summarization

Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummaries.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummaries.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummaries.java
new file mode 100644
index 0000000..1c860ff
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummaries.java
@@ -0,0 +1,831 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.10.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.10.0)")
+public class TSummaries implements org.apache.thrift.TBase<TSummaries, TSummaries._Fields>, java.io.Serializable, Cloneable, Comparable<TSummaries> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TSummaries");
+
+  private static final org.apache.thrift.protocol.TField FINISHED_FIELD_DESC = new org.apache.thrift.protocol.TField("finished", org.apache.thrift.protocol.TType.BOOL, (short)1);
+  private static final org.apache.thrift.protocol.TField SESSION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionId", org.apache.thrift.protocol.TType.I64, (short)2);
+  private static final org.apache.thrift.protocol.TField TOTAL_FILES_FIELD_DESC = new org.apache.thrift.protocol.TField("totalFiles", org.apache.thrift.protocol.TType.I64, (short)3);
+  private static final org.apache.thrift.protocol.TField DELETED_FILES_FIELD_DESC = new org.apache.thrift.protocol.TField("deletedFiles", org.apache.thrift.protocol.TType.I64, (short)4);
+  private static final org.apache.thrift.protocol.TField SUMMARIES_FIELD_DESC = new org.apache.thrift.protocol.TField("summaries", org.apache.thrift.protocol.TType.LIST, (short)5);
+
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TSummariesStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TSummariesTupleSchemeFactory();
+
+  public boolean finished; // required
+  public long sessionId; // required
+  public long totalFiles; // required
+  public long deletedFiles; // required
+  public java.util.List<TSummary> summaries; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    FINISHED((short)1, "finished"),
+    SESSION_ID((short)2, "sessionId"),
+    TOTAL_FILES((short)3, "totalFiles"),
+    DELETED_FILES((short)4, "deletedFiles"),
+    SUMMARIES((short)5, "summaries");
+
+    private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+    static {
+      for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // FINISHED
+          return FINISHED;
+        case 2: // SESSION_ID
+          return SESSION_ID;
+        case 3: // TOTAL_FILES
+          return TOTAL_FILES;
+        case 4: // DELETED_FILES
+          return DELETED_FILES;
+        case 5: // SUMMARIES
+          return SUMMARIES;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(java.lang.String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final java.lang.String _fieldName;
+
+    _Fields(short thriftId, java.lang.String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public java.lang.String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __FINISHED_ISSET_ID = 0;
+  private static final int __SESSIONID_ISSET_ID = 1;
+  private static final int __TOTALFILES_ISSET_ID = 2;
+  private static final int __DELETEDFILES_ISSET_ID = 3;
+  private byte __isset_bitfield = 0;
+  public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.FINISHED, new org.apache.thrift.meta_data.FieldMetaData("finished", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.SESSION_ID, new org.apache.thrift.meta_data.FieldMetaData("sessionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.TOTAL_FILES, new org.apache.thrift.meta_data.FieldMetaData("totalFiles", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.DELETED_FILES, new org.apache.thrift.meta_data.FieldMetaData("deletedFiles", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.SUMMARIES, new org.apache.thrift.meta_data.FieldMetaData("summaries", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSummary.class))));
+    metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TSummaries.class, metaDataMap);
+  }
+
+  public TSummaries() {
+  }
+
+  public TSummaries(
+    boolean finished,
+    long sessionId,
+    long totalFiles,
+    long deletedFiles,
+    java.util.List<TSummary> summaries)
+  {
+    this();
+    this.finished = finished;
+    setFinishedIsSet(true);
+    this.sessionId = sessionId;
+    setSessionIdIsSet(true);
+    this.totalFiles = totalFiles;
+    setTotalFilesIsSet(true);
+    this.deletedFiles = deletedFiles;
+    setDeletedFilesIsSet(true);
+    this.summaries = summaries;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TSummaries(TSummaries other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.finished = other.finished;
+    this.sessionId = other.sessionId;
+    this.totalFiles = other.totalFiles;
+    this.deletedFiles = other.deletedFiles;
+    if (other.isSetSummaries()) {
+      java.util.List<TSummary> __this__summaries = new java.util.ArrayList<TSummary>(other.summaries.size());
+      for (TSummary other_element : other.summaries) {
+        __this__summaries.add(new TSummary(other_element));
+      }
+      this.summaries = __this__summaries;
+    }
+  }
+
+  public TSummaries deepCopy() {
+    return new TSummaries(this);
+  }
+
+  @Override
+  public void clear() {
+    setFinishedIsSet(false);
+    this.finished = false;
+    setSessionIdIsSet(false);
+    this.sessionId = 0;
+    setTotalFilesIsSet(false);
+    this.totalFiles = 0;
+    setDeletedFilesIsSet(false);
+    this.deletedFiles = 0;
+    this.summaries = null;
+  }
+
+  public boolean isFinished() {
+    return this.finished;
+  }
+
+  public TSummaries setFinished(boolean finished) {
+    this.finished = finished;
+    setFinishedIsSet(true);
+    return this;
+  }
+
+  public void unsetFinished() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __FINISHED_ISSET_ID);
+  }
+
+  /** Returns true if field finished is set (has been assigned a value) and false otherwise */
+  public boolean isSetFinished() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __FINISHED_ISSET_ID);
+  }
+
+  public void setFinishedIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __FINISHED_ISSET_ID, value);
+  }
+
+  public long getSessionId() {
+    return this.sessionId;
+  }
+
+  public TSummaries setSessionId(long sessionId) {
+    this.sessionId = sessionId;
+    setSessionIdIsSet(true);
+    return this;
+  }
+
+  public void unsetSessionId() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SESSIONID_ISSET_ID);
+  }
+
+  /** Returns true if field sessionId is set (has been assigned a value) and false otherwise */
+  public boolean isSetSessionId() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SESSIONID_ISSET_ID);
+  }
+
+  public void setSessionIdIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SESSIONID_ISSET_ID, value);
+  }
+
+  public long getTotalFiles() {
+    return this.totalFiles;
+  }
+
+  public TSummaries setTotalFiles(long totalFiles) {
+    this.totalFiles = totalFiles;
+    setTotalFilesIsSet(true);
+    return this;
+  }
+
+  public void unsetTotalFiles() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TOTALFILES_ISSET_ID);
+  }
+
+  /** Returns true if field totalFiles is set (has been assigned a value) and false otherwise */
+  public boolean isSetTotalFiles() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TOTALFILES_ISSET_ID);
+  }
+
+  public void setTotalFilesIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TOTALFILES_ISSET_ID, value);
+  }
+
+  public long getDeletedFiles() {
+    return this.deletedFiles;
+  }
+
+  public TSummaries setDeletedFiles(long deletedFiles) {
+    this.deletedFiles = deletedFiles;
+    setDeletedFilesIsSet(true);
+    return this;
+  }
+
+  public void unsetDeletedFiles() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __DELETEDFILES_ISSET_ID);
+  }
+
+  /** Returns true if field deletedFiles is set (has been assigned a value) and false otherwise */
+  public boolean isSetDeletedFiles() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __DELETEDFILES_ISSET_ID);
+  }
+
+  public void setDeletedFilesIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __DELETEDFILES_ISSET_ID, value);
+  }
+
+  public int getSummariesSize() {
+    return (this.summaries == null) ? 0 : this.summaries.size();
+  }
+
+  public java.util.Iterator<TSummary> getSummariesIterator() {
+    return (this.summaries == null) ? null : this.summaries.iterator();
+  }
+
+  public void addToSummaries(TSummary elem) {
+    if (this.summaries == null) {
+      this.summaries = new java.util.ArrayList<TSummary>();
+    }
+    this.summaries.add(elem);
+  }
+
+  public java.util.List<TSummary> getSummaries() {
+    return this.summaries;
+  }
+
+  public TSummaries setSummaries(java.util.List<TSummary> summaries) {
+    this.summaries = summaries;
+    return this;
+  }
+
+  public void unsetSummaries() {
+    this.summaries = null;
+  }
+
+  /** Returns true if field summaries is set (has been assigned a value) and false otherwise */
+  public boolean isSetSummaries() {
+    return this.summaries != null;
+  }
+
+  public void setSummariesIsSet(boolean value) {
+    if (!value) {
+      this.summaries = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, java.lang.Object value) {
+    switch (field) {
+    case FINISHED:
+      if (value == null) {
+        unsetFinished();
+      } else {
+        setFinished((java.lang.Boolean)value);
+      }
+      break;
+
+    case SESSION_ID:
+      if (value == null) {
+        unsetSessionId();
+      } else {
+        setSessionId((java.lang.Long)value);
+      }
+      break;
+
+    case TOTAL_FILES:
+      if (value == null) {
+        unsetTotalFiles();
+      } else {
+        setTotalFiles((java.lang.Long)value);
+      }
+      break;
+
+    case DELETED_FILES:
+      if (value == null) {
+        unsetDeletedFiles();
+      } else {
+        setDeletedFiles((java.lang.Long)value);
+      }
+      break;
+
+    case SUMMARIES:
+      if (value == null) {
+        unsetSummaries();
+      } else {
+        setSummaries((java.util.List<TSummary>)value);
+      }
+      break;
+
+    }
+  }
+
+  public java.lang.Object getFieldValue(_Fields field) {
+    switch (field) {
+    case FINISHED:
+      return isFinished();
+
+    case SESSION_ID:
+      return getSessionId();
+
+    case TOTAL_FILES:
+      return getTotalFiles();
+
+    case DELETED_FILES:
+      return getDeletedFiles();
+
+    case SUMMARIES:
+      return getSummaries();
+
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new java.lang.IllegalArgumentException();
+    }
+
+    switch (field) {
+    case FINISHED:
+      return isSetFinished();
+    case SESSION_ID:
+      return isSetSessionId();
+    case TOTAL_FILES:
+      return isSetTotalFiles();
+    case DELETED_FILES:
+      return isSetDeletedFiles();
+    case SUMMARIES:
+      return isSetSummaries();
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(java.lang.Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TSummaries)
+      return this.equals((TSummaries)that);
+    return false;
+  }
+
+  public boolean equals(TSummaries that) {
+    if (that == null)
+      return false;
+    if (this == that)
+      return true;
+
+    boolean this_present_finished = true;
+    boolean that_present_finished = true;
+    if (this_present_finished || that_present_finished) {
+      if (!(this_present_finished && that_present_finished))
+        return false;
+      if (this.finished != that.finished)
+        return false;
+    }
+
+    boolean this_present_sessionId = true;
+    boolean that_present_sessionId = true;
+    if (this_present_sessionId || that_present_sessionId) {
+      if (!(this_present_sessionId && that_present_sessionId))
+        return false;
+      if (this.sessionId != that.sessionId)
+        return false;
+    }
+
+    boolean this_present_totalFiles = true;
+    boolean that_present_totalFiles = true;
+    if (this_present_totalFiles || that_present_totalFiles) {
+      if (!(this_present_totalFiles && that_present_totalFiles))
+        return false;
+      if (this.totalFiles != that.totalFiles)
+        return false;
+    }
+
+    boolean this_present_deletedFiles = true;
+    boolean that_present_deletedFiles = true;
+    if (this_present_deletedFiles || that_present_deletedFiles) {
+      if (!(this_present_deletedFiles && that_present_deletedFiles))
+        return false;
+      if (this.deletedFiles != that.deletedFiles)
+        return false;
+    }
+
+    boolean this_present_summaries = true && this.isSetSummaries();
+    boolean that_present_summaries = true && that.isSetSummaries();
+    if (this_present_summaries || that_present_summaries) {
+      if (!(this_present_summaries && that_present_summaries))
+        return false;
+      if (!this.summaries.equals(that.summaries))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int hashCode = 1;
+
+    hashCode = hashCode * 8191 + ((finished) ? 131071 : 524287);
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(sessionId);
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(totalFiles);
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(deletedFiles);
+
+    hashCode = hashCode * 8191 + ((isSetSummaries()) ? 131071 : 524287);
+    if (isSetSummaries())
+      hashCode = hashCode * 8191 + summaries.hashCode();
+
+    return hashCode;
+  }
+
+  @Override
+  public int compareTo(TSummaries other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = java.lang.Boolean.valueOf(isSetFinished()).compareTo(other.isSetFinished());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFinished()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.finished, other.finished);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetSessionId()).compareTo(other.isSetSessionId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSessionId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionId, other.sessionId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetTotalFiles()).compareTo(other.isSetTotalFiles());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTotalFiles()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.totalFiles, other.totalFiles);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetDeletedFiles()).compareTo(other.isSetDeletedFiles());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDeletedFiles()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.deletedFiles, other.deletedFiles);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetSummaries()).compareTo(other.isSetSummaries());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSummaries()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.summaries, other.summaries);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    scheme(iprot).read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    scheme(oprot).write(oprot, this);
+  }
+
+  @Override
+  public java.lang.String toString() {
+    java.lang.StringBuilder sb = new java.lang.StringBuilder("TSummaries(");
+    boolean first = true;
+
+    sb.append("finished:");
+    sb.append(this.finished);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("sessionId:");
+    sb.append(this.sessionId);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("totalFiles:");
+    sb.append(this.totalFiles);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("deletedFiles:");
+    sb.append(this.deletedFiles);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("summaries:");
+    if (this.summaries == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.summaries);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TSummariesStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public TSummariesStandardScheme getScheme() {
+      return new TSummariesStandardScheme();
+    }
+  }
+
+  private static class TSummariesStandardScheme extends org.apache.thrift.scheme.StandardScheme<TSummaries> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TSummaries struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // FINISHED
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.finished = iprot.readBool();
+              struct.setFinishedIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // SESSION_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.sessionId = iprot.readI64();
+              struct.setSessionIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TOTAL_FILES
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.totalFiles = iprot.readI64();
+              struct.setTotalFilesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // DELETED_FILES
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.deletedFiles = iprot.readI64();
+              struct.setDeletedFilesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // SUMMARIES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list114 = iprot.readListBegin();
+                struct.summaries = new java.util.ArrayList<TSummary>(_list114.size);
+                TSummary _elem115;
+                for (int _i116 = 0; _i116 < _list114.size; ++_i116)
+                {
+                  _elem115 = new TSummary();
+                  _elem115.read(iprot);
+                  struct.summaries.add(_elem115);
+                }
+                iprot.readListEnd();
+              }
+              struct.setSummariesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TSummaries struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(FINISHED_FIELD_DESC);
+      oprot.writeBool(struct.finished);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(SESSION_ID_FIELD_DESC);
+      oprot.writeI64(struct.sessionId);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(TOTAL_FILES_FIELD_DESC);
+      oprot.writeI64(struct.totalFiles);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(DELETED_FILES_FIELD_DESC);
+      oprot.writeI64(struct.deletedFiles);
+      oprot.writeFieldEnd();
+      if (struct.summaries != null) {
+        oprot.writeFieldBegin(SUMMARIES_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.summaries.size()));
+          for (TSummary _iter117 : struct.summaries)
+          {
+            _iter117.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TSummariesTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public TSummariesTupleScheme getScheme() {
+      return new TSummariesTupleScheme();
+    }
+  }
+
+  private static class TSummariesTupleScheme extends org.apache.thrift.scheme.TupleScheme<TSummaries> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TSummaries struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet optionals = new java.util.BitSet();
+      if (struct.isSetFinished()) {
+        optionals.set(0);
+      }
+      if (struct.isSetSessionId()) {
+        optionals.set(1);
+      }
+      if (struct.isSetTotalFiles()) {
+        optionals.set(2);
+      }
+      if (struct.isSetDeletedFiles()) {
+        optionals.set(3);
+      }
+      if (struct.isSetSummaries()) {
+        optionals.set(4);
+      }
+      oprot.writeBitSet(optionals, 5);
+      if (struct.isSetFinished()) {
+        oprot.writeBool(struct.finished);
+      }
+      if (struct.isSetSessionId()) {
+        oprot.writeI64(struct.sessionId);
+      }
+      if (struct.isSetTotalFiles()) {
+        oprot.writeI64(struct.totalFiles);
+      }
+      if (struct.isSetDeletedFiles()) {
+        oprot.writeI64(struct.deletedFiles);
+      }
+      if (struct.isSetSummaries()) {
+        {
+          oprot.writeI32(struct.summaries.size());
+          for (TSummary _iter118 : struct.summaries)
+          {
+            _iter118.write(oprot);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TSummaries struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet incoming = iprot.readBitSet(5);
+      if (incoming.get(0)) {
+        struct.finished = iprot.readBool();
+        struct.setFinishedIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.sessionId = iprot.readI64();
+        struct.setSessionIdIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.totalFiles = iprot.readI64();
+        struct.setTotalFilesIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.deletedFiles = iprot.readI64();
+        struct.setDeletedFilesIsSet(true);
+      }
+      if (incoming.get(4)) {
+        {
+          org.apache.thrift.protocol.TList _list119 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.summaries = new java.util.ArrayList<TSummary>(_list119.size);
+          TSummary _elem120;
+          for (int _i121 = 0; _i121 < _list119.size; ++_i121)
+          {
+            _elem120 = new TSummary();
+            _elem120.read(iprot);
+            struct.summaries.add(_elem120);
+          }
+        }
+        struct.setSummariesIsSet(true);
+      }
+    }
+  }
+
+  private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+    return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+  }
+  private static void unusedMethod() {}
+}
+

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummarizerConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummarizerConfiguration.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummarizerConfiguration.java
new file mode 100644
index 0000000..c4a72f3
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummarizerConfiguration.java
@@ -0,0 +1,649 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.10.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.10.0)")
+public class TSummarizerConfiguration implements org.apache.thrift.TBase<TSummarizerConfiguration, TSummarizerConfiguration._Fields>, java.io.Serializable, Cloneable, Comparable<TSummarizerConfiguration> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TSummarizerConfiguration");
+
+  private static final org.apache.thrift.protocol.TField CLASSNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("classname", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("options", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField CONFIG_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("configId", org.apache.thrift.protocol.TType.STRING, (short)3);
+
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TSummarizerConfigurationStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TSummarizerConfigurationTupleSchemeFactory();
+
+  public java.lang.String classname; // required
+  public java.util.Map<java.lang.String,java.lang.String> options; // required
+  public java.lang.String configId; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    CLASSNAME((short)1, "classname"),
+    OPTIONS((short)2, "options"),
+    CONFIG_ID((short)3, "configId");
+
+    private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+    static {
+      for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // CLASSNAME
+          return CLASSNAME;
+        case 2: // OPTIONS
+          return OPTIONS;
+        case 3: // CONFIG_ID
+          return CONFIG_ID;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(java.lang.String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final java.lang.String _fieldName;
+
+    _Fields(short thriftId, java.lang.String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public java.lang.String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CLASSNAME, new org.apache.thrift.meta_data.FieldMetaData("classname", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("options", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.CONFIG_ID, new org.apache.thrift.meta_data.FieldMetaData("configId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TSummarizerConfiguration.class, metaDataMap);
+  }
+
+  public TSummarizerConfiguration() {
+  }
+
+  public TSummarizerConfiguration(
+    java.lang.String classname,
+    java.util.Map<java.lang.String,java.lang.String> options,
+    java.lang.String configId)
+  {
+    this();
+    this.classname = classname;
+    this.options = options;
+    this.configId = configId;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TSummarizerConfiguration(TSummarizerConfiguration other) {
+    if (other.isSetClassname()) {
+      this.classname = other.classname;
+    }
+    if (other.isSetOptions()) {
+      java.util.Map<java.lang.String,java.lang.String> __this__options = new java.util.HashMap<java.lang.String,java.lang.String>(other.options);
+      this.options = __this__options;
+    }
+    if (other.isSetConfigId()) {
+      this.configId = other.configId;
+    }
+  }
+
+  public TSummarizerConfiguration deepCopy() {
+    return new TSummarizerConfiguration(this);
+  }
+
+  @Override
+  public void clear() {
+    this.classname = null;
+    this.options = null;
+    this.configId = null;
+  }
+
+  public java.lang.String getClassname() {
+    return this.classname;
+  }
+
+  public TSummarizerConfiguration setClassname(java.lang.String classname) {
+    this.classname = classname;
+    return this;
+  }
+
+  public void unsetClassname() {
+    this.classname = null;
+  }
+
+  /** Returns true if field classname is set (has been assigned a value) and false otherwise */
+  public boolean isSetClassname() {
+    return this.classname != null;
+  }
+
+  public void setClassnameIsSet(boolean value) {
+    if (!value) {
+      this.classname = null;
+    }
+  }
+
+  public int getOptionsSize() {
+    return (this.options == null) ? 0 : this.options.size();
+  }
+
+  public void putToOptions(java.lang.String key, java.lang.String val) {
+    if (this.options == null) {
+      this.options = new java.util.HashMap<java.lang.String,java.lang.String>();
+    }
+    this.options.put(key, val);
+  }
+
+  public java.util.Map<java.lang.String,java.lang.String> getOptions() {
+    return this.options;
+  }
+
+  public TSummarizerConfiguration setOptions(java.util.Map<java.lang.String,java.lang.String> options) {
+    this.options = options;
+    return this;
+  }
+
+  public void unsetOptions() {
+    this.options = null;
+  }
+
+  /** Returns true if field options is set (has been assigned a value) and false otherwise */
+  public boolean isSetOptions() {
+    return this.options != null;
+  }
+
+  public void setOptionsIsSet(boolean value) {
+    if (!value) {
+      this.options = null;
+    }
+  }
+
+  public java.lang.String getConfigId() {
+    return this.configId;
+  }
+
+  public TSummarizerConfiguration setConfigId(java.lang.String configId) {
+    this.configId = configId;
+    return this;
+  }
+
+  public void unsetConfigId() {
+    this.configId = null;
+  }
+
+  /** Returns true if field configId is set (has been assigned a value) and false otherwise */
+  public boolean isSetConfigId() {
+    return this.configId != null;
+  }
+
+  public void setConfigIdIsSet(boolean value) {
+    if (!value) {
+      this.configId = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, java.lang.Object value) {
+    switch (field) {
+    case CLASSNAME:
+      if (value == null) {
+        unsetClassname();
+      } else {
+        setClassname((java.lang.String)value);
+      }
+      break;
+
+    case OPTIONS:
+      if (value == null) {
+        unsetOptions();
+      } else {
+        setOptions((java.util.Map<java.lang.String,java.lang.String>)value);
+      }
+      break;
+
+    case CONFIG_ID:
+      if (value == null) {
+        unsetConfigId();
+      } else {
+        setConfigId((java.lang.String)value);
+      }
+      break;
+
+    }
+  }
+
+  public java.lang.Object getFieldValue(_Fields field) {
+    switch (field) {
+    case CLASSNAME:
+      return getClassname();
+
+    case OPTIONS:
+      return getOptions();
+
+    case CONFIG_ID:
+      return getConfigId();
+
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new java.lang.IllegalArgumentException();
+    }
+
+    switch (field) {
+    case CLASSNAME:
+      return isSetClassname();
+    case OPTIONS:
+      return isSetOptions();
+    case CONFIG_ID:
+      return isSetConfigId();
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(java.lang.Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TSummarizerConfiguration)
+      return this.equals((TSummarizerConfiguration)that);
+    return false;
+  }
+
+  public boolean equals(TSummarizerConfiguration that) {
+    if (that == null)
+      return false;
+    if (this == that)
+      return true;
+
+    boolean this_present_classname = true && this.isSetClassname();
+    boolean that_present_classname = true && that.isSetClassname();
+    if (this_present_classname || that_present_classname) {
+      if (!(this_present_classname && that_present_classname))
+        return false;
+      if (!this.classname.equals(that.classname))
+        return false;
+    }
+
+    boolean this_present_options = true && this.isSetOptions();
+    boolean that_present_options = true && that.isSetOptions();
+    if (this_present_options || that_present_options) {
+      if (!(this_present_options && that_present_options))
+        return false;
+      if (!this.options.equals(that.options))
+        return false;
+    }
+
+    boolean this_present_configId = true && this.isSetConfigId();
+    boolean that_present_configId = true && that.isSetConfigId();
+    if (this_present_configId || that_present_configId) {
+      if (!(this_present_configId && that_present_configId))
+        return false;
+      if (!this.configId.equals(that.configId))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int hashCode = 1;
+
+    hashCode = hashCode * 8191 + ((isSetClassname()) ? 131071 : 524287);
+    if (isSetClassname())
+      hashCode = hashCode * 8191 + classname.hashCode();
+
+    hashCode = hashCode * 8191 + ((isSetOptions()) ? 131071 : 524287);
+    if (isSetOptions())
+      hashCode = hashCode * 8191 + options.hashCode();
+
+    hashCode = hashCode * 8191 + ((isSetConfigId()) ? 131071 : 524287);
+    if (isSetConfigId())
+      hashCode = hashCode * 8191 + configId.hashCode();
+
+    return hashCode;
+  }
+
+  @Override
+  public int compareTo(TSummarizerConfiguration other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = java.lang.Boolean.valueOf(isSetClassname()).compareTo(other.isSetClassname());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetClassname()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.classname, other.classname);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetOptions()).compareTo(other.isSetOptions());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetOptions()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.options, other.options);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetConfigId()).compareTo(other.isSetConfigId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetConfigId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.configId, other.configId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    scheme(iprot).read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    scheme(oprot).write(oprot, this);
+  }
+
+  @Override
+  public java.lang.String toString() {
+    java.lang.StringBuilder sb = new java.lang.StringBuilder("TSummarizerConfiguration(");
+    boolean first = true;
+
+    sb.append("classname:");
+    if (this.classname == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.classname);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("options:");
+    if (this.options == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.options);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("configId:");
+    if (this.configId == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.configId);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TSummarizerConfigurationStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public TSummarizerConfigurationStandardScheme getScheme() {
+      return new TSummarizerConfigurationStandardScheme();
+    }
+  }
+
+  private static class TSummarizerConfigurationStandardScheme extends org.apache.thrift.scheme.StandardScheme<TSummarizerConfiguration> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TSummarizerConfiguration struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // CLASSNAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.classname = iprot.readString();
+              struct.setClassnameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // OPTIONS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map94 = iprot.readMapBegin();
+                struct.options = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map94.size);
+                java.lang.String _key95;
+                java.lang.String _val96;
+                for (int _i97 = 0; _i97 < _map94.size; ++_i97)
+                {
+                  _key95 = iprot.readString();
+                  _val96 = iprot.readString();
+                  struct.options.put(_key95, _val96);
+                }
+                iprot.readMapEnd();
+              }
+              struct.setOptionsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // CONFIG_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.configId = iprot.readString();
+              struct.setConfigIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TSummarizerConfiguration struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.classname != null) {
+        oprot.writeFieldBegin(CLASSNAME_FIELD_DESC);
+        oprot.writeString(struct.classname);
+        oprot.writeFieldEnd();
+      }
+      if (struct.options != null) {
+        oprot.writeFieldBegin(OPTIONS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.options.size()));
+          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter98 : struct.options.entrySet())
+          {
+            oprot.writeString(_iter98.getKey());
+            oprot.writeString(_iter98.getValue());
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.configId != null) {
+        oprot.writeFieldBegin(CONFIG_ID_FIELD_DESC);
+        oprot.writeString(struct.configId);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TSummarizerConfigurationTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public TSummarizerConfigurationTupleScheme getScheme() {
+      return new TSummarizerConfigurationTupleScheme();
+    }
+  }
+
+  private static class TSummarizerConfigurationTupleScheme extends org.apache.thrift.scheme.TupleScheme<TSummarizerConfiguration> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TSummarizerConfiguration struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet optionals = new java.util.BitSet();
+      if (struct.isSetClassname()) {
+        optionals.set(0);
+      }
+      if (struct.isSetOptions()) {
+        optionals.set(1);
+      }
+      if (struct.isSetConfigId()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetClassname()) {
+        oprot.writeString(struct.classname);
+      }
+      if (struct.isSetOptions()) {
+        {
+          oprot.writeI32(struct.options.size());
+          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter99 : struct.options.entrySet())
+          {
+            oprot.writeString(_iter99.getKey());
+            oprot.writeString(_iter99.getValue());
+          }
+        }
+      }
+      if (struct.isSetConfigId()) {
+        oprot.writeString(struct.configId);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TSummarizerConfiguration struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.classname = iprot.readString();
+        struct.setClassnameIsSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TMap _map100 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.options = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map100.size);
+          java.lang.String _key101;
+          java.lang.String _val102;
+          for (int _i103 = 0; _i103 < _map100.size; ++_i103)
+          {
+            _key101 = iprot.readString();
+            _val102 = iprot.readString();
+            struct.options.put(_key101, _val102);
+          }
+        }
+        struct.setOptionsIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.configId = iprot.readString();
+        struct.setConfigIdIsSet(true);
+      }
+    }
+  }
+
+  private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+    return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+  }
+  private static void unusedMethod() {}
+}
+

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummary.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummary.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummary.java
new file mode 100644
index 0000000..64b49c2
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummary.java
@@ -0,0 +1,842 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.10.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.10.0)")
+public class TSummary implements org.apache.thrift.TBase<TSummary, TSummary._Fields>, java.io.Serializable, Cloneable, Comparable<TSummary> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TSummary");
+
+  private static final org.apache.thrift.protocol.TField SUMMARY_FIELD_DESC = new org.apache.thrift.protocol.TField("summary", org.apache.thrift.protocol.TType.MAP, (short)1);
+  private static final org.apache.thrift.protocol.TField CONFIG_FIELD_DESC = new org.apache.thrift.protocol.TField("config", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+  private static final org.apache.thrift.protocol.TField FILES_CONTAINING_FIELD_DESC = new org.apache.thrift.protocol.TField("filesContaining", org.apache.thrift.protocol.TType.I64, (short)3);
+  private static final org.apache.thrift.protocol.TField FILES_EXCEEDING_FIELD_DESC = new org.apache.thrift.protocol.TField("filesExceeding", org.apache.thrift.protocol.TType.I64, (short)4);
+  private static final org.apache.thrift.protocol.TField FILES_LARGE_FIELD_DESC = new org.apache.thrift.protocol.TField("filesLarge", org.apache.thrift.protocol.TType.I64, (short)5);
+
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TSummaryStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TSummaryTupleSchemeFactory();
+
+  public java.util.Map<java.lang.String,java.lang.Long> summary; // required
+  public TSummarizerConfiguration config; // required
+  public long filesContaining; // required
+  public long filesExceeding; // required
+  public long filesLarge; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SUMMARY((short)1, "summary"),
+    CONFIG((short)2, "config"),
+    FILES_CONTAINING((short)3, "filesContaining"),
+    FILES_EXCEEDING((short)4, "filesExceeding"),
+    FILES_LARGE((short)5, "filesLarge");
+
+    private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+    static {
+      for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SUMMARY
+          return SUMMARY;
+        case 2: // CONFIG
+          return CONFIG;
+        case 3: // FILES_CONTAINING
+          return FILES_CONTAINING;
+        case 4: // FILES_EXCEEDING
+          return FILES_EXCEEDING;
+        case 5: // FILES_LARGE
+          return FILES_LARGE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(java.lang.String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final java.lang.String _fieldName;
+
+    _Fields(short thriftId, java.lang.String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public java.lang.String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __FILESCONTAINING_ISSET_ID = 0;
+  private static final int __FILESEXCEEDING_ISSET_ID = 1;
+  private static final int __FILESLARGE_ISSET_ID = 2;
+  private byte __isset_bitfield = 0;
+  public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SUMMARY, new org.apache.thrift.meta_data.FieldMetaData("summary", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.CONFIG, new org.apache.thrift.meta_data.FieldMetaData("config", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSummarizerConfiguration.class)));
+    tmpMap.put(_Fields.FILES_CONTAINING, new org.apache.thrift.meta_data.FieldMetaData("filesContaining", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.FILES_EXCEEDING, new org.apache.thrift.meta_data.FieldMetaData("filesExceeding", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.FILES_LARGE, new org.apache.thrift.meta_data.FieldMetaData("filesLarge", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TSummary.class, metaDataMap);
+  }
+
+  public TSummary() {
+  }
+
+  public TSummary(
+    java.util.Map<java.lang.String,java.lang.Long> summary,
+    TSummarizerConfiguration config,
+    long filesContaining,
+    long filesExceeding,
+    long filesLarge)
+  {
+    this();
+    this.summary = summary;
+    this.config = config;
+    this.filesContaining = filesContaining;
+    setFilesContainingIsSet(true);
+    this.filesExceeding = filesExceeding;
+    setFilesExceedingIsSet(true);
+    this.filesLarge = filesLarge;
+    setFilesLargeIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TSummary(TSummary other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetSummary()) {
+      java.util.Map<java.lang.String,java.lang.Long> __this__summary = new java.util.HashMap<java.lang.String,java.lang.Long>(other.summary);
+      this.summary = __this__summary;
+    }
+    if (other.isSetConfig()) {
+      this.config = new TSummarizerConfiguration(other.config);
+    }
+    this.filesContaining = other.filesContaining;
+    this.filesExceeding = other.filesExceeding;
+    this.filesLarge = other.filesLarge;
+  }
+
+  public TSummary deepCopy() {
+    return new TSummary(this);
+  }
+
+  @Override
+  public void clear() {
+    this.summary = null;
+    this.config = null;
+    setFilesContainingIsSet(false);
+    this.filesContaining = 0;
+    setFilesExceedingIsSet(false);
+    this.filesExceeding = 0;
+    setFilesLargeIsSet(false);
+    this.filesLarge = 0;
+  }
+
+  public int getSummarySize() {
+    return (this.summary == null) ? 0 : this.summary.size();
+  }
+
+  public void putToSummary(java.lang.String key, long val) {
+    if (this.summary == null) {
+      this.summary = new java.util.HashMap<java.lang.String,java.lang.Long>();
+    }
+    this.summary.put(key, val);
+  }
+
+  public java.util.Map<java.lang.String,java.lang.Long> getSummary() {
+    return this.summary;
+  }
+
+  public TSummary setSummary(java.util.Map<java.lang.String,java.lang.Long> summary) {
+    this.summary = summary;
+    return this;
+  }
+
+  public void unsetSummary() {
+    this.summary = null;
+  }
+
+  /** Returns true if field summary is set (has been assigned a value) and false otherwise */
+  public boolean isSetSummary() {
+    return this.summary != null;
+  }
+
+  public void setSummaryIsSet(boolean value) {
+    if (!value) {
+      this.summary = null;
+    }
+  }
+
+  public TSummarizerConfiguration getConfig() {
+    return this.config;
+  }
+
+  public TSummary setConfig(TSummarizerConfiguration config) {
+    this.config = config;
+    return this;
+  }
+
+  public void unsetConfig() {
+    this.config = null;
+  }
+
+  /** Returns true if field config is set (has been assigned a value) and false otherwise */
+  public boolean isSetConfig() {
+    return this.config != null;
+  }
+
+  public void setConfigIsSet(boolean value) {
+    if (!value) {
+      this.config = null;
+    }
+  }
+
+  public long getFilesContaining() {
+    return this.filesContaining;
+  }
+
+  public TSummary setFilesContaining(long filesContaining) {
+    this.filesContaining = filesContaining;
+    setFilesContainingIsSet(true);
+    return this;
+  }
+
+  public void unsetFilesContaining() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __FILESCONTAINING_ISSET_ID);
+  }
+
+  /** Returns true if field filesContaining is set (has been assigned a value) and false otherwise */
+  public boolean isSetFilesContaining() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __FILESCONTAINING_ISSET_ID);
+  }
+
+  public void setFilesContainingIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __FILESCONTAINING_ISSET_ID, value);
+  }
+
+  public long getFilesExceeding() {
+    return this.filesExceeding;
+  }
+
+  public TSummary setFilesExceeding(long filesExceeding) {
+    this.filesExceeding = filesExceeding;
+    setFilesExceedingIsSet(true);
+    return this;
+  }
+
+  public void unsetFilesExceeding() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __FILESEXCEEDING_ISSET_ID);
+  }
+
+  /** Returns true if field filesExceeding is set (has been assigned a value) and false otherwise */
+  public boolean isSetFilesExceeding() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __FILESEXCEEDING_ISSET_ID);
+  }
+
+  public void setFilesExceedingIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __FILESEXCEEDING_ISSET_ID, value);
+  }
+
+  public long getFilesLarge() {
+    return this.filesLarge;
+  }
+
+  public TSummary setFilesLarge(long filesLarge) {
+    this.filesLarge = filesLarge;
+    setFilesLargeIsSet(true);
+    return this;
+  }
+
+  public void unsetFilesLarge() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __FILESLARGE_ISSET_ID);
+  }
+
+  /** Returns true if field filesLarge is set (has been assigned a value) and false otherwise */
+  public boolean isSetFilesLarge() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __FILESLARGE_ISSET_ID);
+  }
+
+  public void setFilesLargeIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __FILESLARGE_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, java.lang.Object value) {
+    switch (field) {
+    case SUMMARY:
+      if (value == null) {
+        unsetSummary();
+      } else {
+        setSummary((java.util.Map<java.lang.String,java.lang.Long>)value);
+      }
+      break;
+
+    case CONFIG:
+      if (value == null) {
+        unsetConfig();
+      } else {
+        setConfig((TSummarizerConfiguration)value);
+      }
+      break;
+
+    case FILES_CONTAINING:
+      if (value == null) {
+        unsetFilesContaining();
+      } else {
+        setFilesContaining((java.lang.Long)value);
+      }
+      break;
+
+    case FILES_EXCEEDING:
+      if (value == null) {
+        unsetFilesExceeding();
+      } else {
+        setFilesExceeding((java.lang.Long)value);
+      }
+      break;
+
+    case FILES_LARGE:
+      if (value == null) {
+        unsetFilesLarge();
+      } else {
+        setFilesLarge((java.lang.Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public java.lang.Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SUMMARY:
+      return getSummary();
+
+    case CONFIG:
+      return getConfig();
+
+    case FILES_CONTAINING:
+      return getFilesContaining();
+
+    case FILES_EXCEEDING:
+      return getFilesExceeding();
+
+    case FILES_LARGE:
+      return getFilesLarge();
+
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new java.lang.IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SUMMARY:
+      return isSetSummary();
+    case CONFIG:
+      return isSetConfig();
+    case FILES_CONTAINING:
+      return isSetFilesContaining();
+    case FILES_EXCEEDING:
+      return isSetFilesExceeding();
+    case FILES_LARGE:
+      return isSetFilesLarge();
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(java.lang.Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TSummary)
+      return this.equals((TSummary)that);
+    return false;
+  }
+
+  public boolean equals(TSummary that) {
+    if (that == null)
+      return false;
+    if (this == that)
+      return true;
+
+    boolean this_present_summary = true && this.isSetSummary();
+    boolean that_present_summary = true && that.isSetSummary();
+    if (this_present_summary || that_present_summary) {
+      if (!(this_present_summary && that_present_summary))
+        return false;
+      if (!this.summary.equals(that.summary))
+        return false;
+    }
+
+    boolean this_present_config = true && this.isSetConfig();
+    boolean that_present_config = true && that.isSetConfig();
+    if (this_present_config || that_present_config) {
+      if (!(this_present_config && that_present_config))
+        return false;
+      if (!this.config.equals(that.config))
+        return false;
+    }
+
+    boolean this_present_filesContaining = true;
+    boolean that_present_filesContaining = true;
+    if (this_present_filesContaining || that_present_filesContaining) {
+      if (!(this_present_filesContaining && that_present_filesContaining))
+        return false;
+      if (this.filesContaining != that.filesContaining)
+        return false;
+    }
+
+    boolean this_present_filesExceeding = true;
+    boolean that_present_filesExceeding = true;
+    if (this_present_filesExceeding || that_present_filesExceeding) {
+      if (!(this_present_filesExceeding && that_present_filesExceeding))
+        return false;
+      if (this.filesExceeding != that.filesExceeding)
+        return false;
+    }
+
+    boolean this_present_filesLarge = true;
+    boolean that_present_filesLarge = true;
+    if (this_present_filesLarge || that_present_filesLarge) {
+      if (!(this_present_filesLarge && that_present_filesLarge))
+        return false;
+      if (this.filesLarge != that.filesLarge)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int hashCode = 1;
+
+    hashCode = hashCode * 8191 + ((isSetSummary()) ? 131071 : 524287);
+    if (isSetSummary())
+      hashCode = hashCode * 8191 + summary.hashCode();
+
+    hashCode = hashCode * 8191 + ((isSetConfig()) ? 131071 : 524287);
+    if (isSetConfig())
+      hashCode = hashCode * 8191 + config.hashCode();
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(filesContaining);
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(filesExceeding);
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(filesLarge);
+
+    return hashCode;
+  }
+
+  @Override
+  public int compareTo(TSummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = java.lang.Boolean.valueOf(isSetSummary()).compareTo(other.isSetSummary());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSummary()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.summary, other.summary);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetConfig()).compareTo(other.isSetConfig());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetConfig()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.config, other.config);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetFilesContaining()).compareTo(other.isSetFilesContaining());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFilesContaining()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filesContaining, other.filesContaining);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetFilesExceeding()).compareTo(other.isSetFilesExceeding());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFilesExceeding()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filesExceeding, other.filesExceeding);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetFilesLarge()).compareTo(other.isSetFilesLarge());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFilesLarge()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filesLarge, other.filesLarge);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    scheme(iprot).read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    scheme(oprot).write(oprot, this);
+  }
+
+  @Override
+  public java.lang.String toString() {
+    java.lang.StringBuilder sb = new java.lang.StringBuilder("TSummary(");
+    boolean first = true;
+
+    sb.append("summary:");
+    if (this.summary == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.summary);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("config:");
+    if (this.config == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.config);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("filesContaining:");
+    sb.append(this.filesContaining);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("filesExceeding:");
+    sb.append(this.filesExceeding);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("filesLarge:");
+    sb.append(this.filesLarge);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (config != null) {
+      config.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TSummaryStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public TSummaryStandardScheme getScheme() {
+      return new TSummaryStandardScheme();
+    }
+  }
+
+  private static class TSummaryStandardScheme extends org.apache.thrift.scheme.StandardScheme<TSummary> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TSummary struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SUMMARY
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map104 = iprot.readMapBegin();
+                struct.summary = new java.util.HashMap<java.lang.String,java.lang.Long>(2*_map104.size);
+                java.lang.String _key105;
+                long _val106;
+                for (int _i107 = 0; _i107 < _map104.size; ++_i107)
+                {
+                  _key105 = iprot.readString();
+                  _val106 = iprot.readI64();
+                  struct.summary.put(_key105, _val106);
+                }
+                iprot.readMapEnd();
+              }
+              struct.setSummaryIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // CONFIG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.config = new TSummarizerConfiguration();
+              struct.config.read(iprot);
+              struct.setConfigIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // FILES_CONTAINING
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.filesContaining = iprot.readI64();
+              struct.setFilesContainingIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // FILES_EXCEEDING
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.filesExceeding = iprot.readI64();
+              struct.setFilesExceedingIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // FILES_LARGE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.filesLarge = iprot.readI64();
+              struct.setFilesLargeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TSummary struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.summary != null) {
+        oprot.writeFieldBegin(SUMMARY_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.summary.size()));
+          for (java.util.Map.Entry<java.lang.String, java.lang.Long> _iter108 : struct.summary.entrySet())
+          {
+            oprot.writeString(_iter108.getKey());
+            oprot.writeI64(_iter108.getValue());
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.config != null) {
+        oprot.writeFieldBegin(CONFIG_FIELD_DESC);
+        struct.config.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(FILES_CONTAINING_FIELD_DESC);
+      oprot.writeI64(struct.filesContaining);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(FILES_EXCEEDING_FIELD_DESC);
+      oprot.writeI64(struct.filesExceeding);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(FILES_LARGE_FIELD_DESC);
+      oprot.writeI64(struct.filesLarge);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TSummaryTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public TSummaryTupleScheme getScheme() {
+      return new TSummaryTupleScheme();
+    }
+  }
+
+  private static class TSummaryTupleScheme extends org.apache.thrift.scheme.TupleScheme<TSummary> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TSummary struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet optionals = new java.util.BitSet();
+      if (struct.isSetSummary()) {
+        optionals.set(0);
+      }
+      if (struct.isSetConfig()) {
+        optionals.set(1);
+      }
+      if (struct.isSetFilesContaining()) {
+        optionals.set(2);
+      }
+      if (struct.isSetFilesExceeding()) {
+        optionals.set(3);
+      }
+      if (struct.isSetFilesLarge()) {
+        optionals.set(4);
+      }
+      oprot.writeBitSet(optionals, 5);
+      if (struct.isSetSummary()) {
+        {
+          oprot.writeI32(struct.summary.size());
+          for (java.util.Map.Entry<java.lang.String, java.lang.Long> _iter109 : struct.summary.entrySet())
+          {
+            oprot.writeString(_iter109.getKey());
+            oprot.writeI64(_iter109.getValue());
+          }
+        }
+      }
+      if (struct.isSetConfig()) {
+        struct.config.write(oprot);
+      }
+      if (struct.isSetFilesContaining()) {
+        oprot.writeI64(struct.filesContaining);
+      }
+      if (struct.isSetFilesExceeding()) {
+        oprot.writeI64(struct.filesExceeding);
+      }
+      if (struct.isSetFilesLarge()) {
+        oprot.writeI64(struct.filesLarge);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TSummary struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet incoming = iprot.readBitSet(5);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TMap _map110 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.summary = new java.util.HashMap<java.lang.String,java.lang.Long>(2*_map110.size);
+          java.lang.String _key111;
+          long _val112;
+          for (int _i113 = 0; _i113 < _map110.size; ++_i113)
+          {
+            _key111 = iprot.readString();
+            _val112 = iprot.readI64();
+            struct.summary.put(_key111, _val112);
+          }
+        }
+        struct.setSummaryIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.config = new TSummarizerConfiguration();
+        struct.config.read(iprot);
+        struct.setConfigIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.filesContaining = iprot.readI64();
+        struct.setFilesContainingIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.filesExceeding = iprot.readI64();
+        struct.setFilesExceedingIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.filesLarge = iprot.readI64();
+        struct.setFilesLargeIsSet(true);
+      }
+    }
+  }
+
+  private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+    return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+  }
+  private static void unusedMethod() {}
+}
+


[9/9] accumulo git commit: ACCUMULO-4501 ACCUMULO-96 Added Summarization

Posted by kt...@apache.org.
ACCUMULO-4501 ACCUMULO-96 Added Summarization

closes apache/accumulo#224
closes apache/accumulo#168


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/94cdcc4d
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/94cdcc4d
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/94cdcc4d

Branch: refs/heads/master
Commit: 94cdcc4d3f0a8ccf95894f206cb71e6117f4e51d
Parents: 68ba2ef
Author: Keith Turner <kt...@apache.org>
Authored: Mon Mar 20 10:47:00 2017 -0400
Committer: Keith Turner <kt...@apache.org>
Committed: Mon Mar 20 10:47:00 2017 -0400

----------------------------------------------------------------------
 .../client/admin/NewTableConfiguration.java     |   49 +-
 .../core/client/admin/SummaryRetriever.java     |  112 +
 .../core/client/admin/TableOperations.java      |   65 +
 .../accumulo/core/client/impl/ServerClient.java |   36 +-
 .../core/client/impl/TableOperationsImpl.java   |  151 +-
 .../client/mapred/AccumuloFileOutputFormat.java |   16 +
 .../mapreduce/AccumuloFileOutputFormat.java     |   16 +
 .../lib/impl/FileOutputConfigurator.java        |    9 +
 .../core/client/mock/MockTableOperations.java   |   27 +
 .../accumulo/core/client/rfile/RFile.java       |  129 +
 .../core/client/rfile/RFileScannerBuilder.java  |    2 +
 .../client/rfile/RFileSummariesRetriever.java   |  122 +
 .../accumulo/core/client/rfile/RFileWriter.java |   26 +
 .../core/client/rfile/RFileWriterBuilder.java   |   38 +-
 .../core/client/summary/CounterSummary.java     |  123 +
 .../core/client/summary/CountingSummarizer.java |  302 +
 .../core/client/summary/Summarizer.java         |  227 +
 .../client/summary/SummarizerConfiguration.java |  285 +
 .../accumulo/core/client/summary/Summary.java   |  145 +
 .../summary/summarizers/DeletesSummarizer.java  |   75 +
 .../summary/summarizers/FamilySummarizer.java   |   46 +
 .../summarizers/VisibilitySummarizer.java       |   47 +
 .../core/compaction/CompactionSettings.java     |    2 +
 .../org/apache/accumulo/core/conf/Property.java |   22 +-
 .../apache/accumulo/core/conf/PropertyType.java |    9 +-
 .../accumulo/core/data/ArrayByteSequence.java   |   18 +
 .../accumulo/core/data/thrift/TRowRange.java    |  521 ++
 .../accumulo/core/data/thrift/TSummaries.java   |  831 +++
 .../data/thrift/TSummarizerConfiguration.java   |  649 ++
 .../accumulo/core/data/thrift/TSummary.java     |  842 +++
 .../core/data/thrift/TSummaryRequest.java       |  760 +++
 .../accumulo/core/file/BloomFilterLayer.java    |   10 +-
 .../core/file/DispatchingFileFactory.java       |    7 +-
 .../accumulo/core/file/FileOperations.java      |   18 +
 .../accumulo/core/file/rfile/PrintInfo.java     |    7 +
 .../core/file/rfile/RFileOperations.java        |    2 +-
 .../core/metadata/schema/MetadataScanner.java   |  236 +
 .../core/metadata/schema/TabletMetadata.java    |  182 +
 .../sample/impl/SamplerConfigurationImpl.java   |   12 -
 .../core/sample/impl/SamplerFactory.java        |    8 +-
 .../accumulo/core/security/TablePermission.java |    5 +-
 .../apache/accumulo/core/summary/Gatherer.java  |  631 ++
 .../summary/SummarizerConfigurationUtil.java    |  128 +
 .../core/summary/SummarizerFactory.java         |   63 +
 .../core/summary/SummaryCollection.java         |  188 +
 .../accumulo/core/summary/SummaryInfo.java      |   53 +
 .../accumulo/core/summary/SummaryReader.java    |  257 +
 .../core/summary/SummarySerializer.java         |  542 ++
 .../accumulo/core/summary/SummaryWriter.java    |  157 +
 .../thrift/TabletClientService.java             | 5642 +++++++++++++++++-
 .../accumulo/core/util/CancelFlagFuture.java    |   67 +
 .../core/util/CompletableFutureUtil.java        |   49 +
 core/src/main/thrift/data.thrift                |   34 +
 core/src/main/thrift/tabletserver.thrift        |    5 +
 .../client/impl/TableOperationsHelperTest.java  |   26 +
 .../mapred/AccumuloFileOutputFormatTest.java    |   18 +
 .../mapreduce/AccumuloFileOutputFormatTest.java |   18 +
 .../accumulo/core/client/rfile/RFileTest.java   |  158 +-
 .../client/summary/CountingSummarizerTest.java  |  259 +
 .../core/summary/SummaryCollectionTest.java     |   72 +
 .../core/util/CompletableFutureUtilTest.java    |   53 +
 .../main/asciidoc/accumulo_user_manual.asciidoc |    2 +
 docs/src/main/asciidoc/chapters/summaries.txt   |  232 +
 .../standalone/StandaloneAccumuloCluster.java   |    3 +-
 .../standalone/StandaloneClusterControl.java    |    3 +-
 .../impl/MiniAccumuloConfigImpl.java            |    3 +-
 .../server/security/SecurityOperation.java      |    5 +
 .../apache/accumulo/tserver/TabletServer.java   |  115 +
 .../tserver/TabletServerResourceManager.java    |   49 +-
 .../tserver/compaction/CompactionStrategy.java  |    1 -
 .../compaction/MajorCompactionRequest.java      |   84 +-
 .../ConfigurableCompactionStrategy.java         |   99 +-
 .../TooManyDeletesCompactionStrategy.java       |  173 +
 .../tserver/session/SummarySession.java         |   42 +
 .../apache/accumulo/tserver/tablet/Tablet.java  |    9 +-
 .../DefaultCompactionStrategyTest.java          |    2 +-
 .../SizeLimitCompactionStrategyTest.java        |    2 +-
 .../TwoTierCompactionStrategyTest.java          |    6 +-
 .../ConfigurableCompactionStrategyTest.java     |    2 +-
 .../java/org/apache/accumulo/shell/Shell.java   |    4 +-
 .../accumulo/shell/commands/CompactCommand.java |    8 +-
 .../shell/commands/SummariesCommand.java        |  115 +
 .../org/apache/accumulo/test/ShellServerIT.java |  129 +-
 .../test/functional/BasicSummarizer.java        |   80 +
 .../accumulo/test/functional/SummaryIT.java     |  820 +++
 .../test/functional/TooManyDeletesIT.java       |  121 +
 .../test/performance/thrift/NullTserver.java    |   33 +-
 87 files changed, 16482 insertions(+), 269 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java
index 4694e1e..9d5d31a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java
@@ -19,14 +19,18 @@ package org.apache.accumulo.core.client.admin;
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.util.Objects.requireNonNull;
 
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
+import org.apache.accumulo.core.summary.SummarizerConfigurationUtil;
 
 /**
  * This object stores table creation parameters. Currently includes: {@link TimeType}, whether to include default iterators, and user-specified initial
@@ -41,8 +45,13 @@ public class NewTableConfiguration {
 
   private boolean limitVersion = true;
 
-  private Map<String,String> properties = new HashMap<>();
-  private SamplerConfiguration samplerConfiguration;
+  private Map<String,String> properties = Collections.emptyMap();
+  private Map<String,String> samplerProps = Collections.emptyMap();
+  private Map<String,String> summarizerProps = Collections.emptyMap();
+
+  private void checkDisjoint(Map<String,String> props, Map<String,String> derivedProps, String kind) {
+    checkArgument(Collections.disjoint(props.keySet(), derivedProps.keySet()), "Properties and derived %s properties are not disjoint", kind);
+  }
 
   /**
    * Configure logical or millisecond time for tables created with this configuration.
@@ -82,15 +91,15 @@ public class NewTableConfiguration {
    * Sets additional properties to be applied to tables created with this configuration. Additional calls to this method replaces properties set by previous
    * calls.
    *
-   * @param prop
+   * @param props
    *          additional properties to add to the table when it is created
    * @return this
    */
-  public NewTableConfiguration setProperties(Map<String,String> prop) {
-    checkArgument(prop != null, "properties is null");
-    SamplerConfigurationImpl.checkDisjoint(prop, samplerConfiguration);
-
-    this.properties = new HashMap<>(prop);
+  public NewTableConfiguration setProperties(Map<String,String> props) {
+    checkArgument(props != null, "properties is null");
+    checkDisjoint(props, samplerProps, "sampler");
+    checkDisjoint(props, summarizerProps, "summarizer");
+    this.properties = new HashMap<>(props);
     return this;
   }
 
@@ -106,10 +115,8 @@ public class NewTableConfiguration {
       propertyMap.putAll(IteratorUtil.generateInitialTableProperties(limitVersion));
     }
 
-    if (samplerConfiguration != null) {
-      propertyMap.putAll(new SamplerConfigurationImpl(samplerConfiguration).toTablePropertiesMap());
-    }
-
+    propertyMap.putAll(summarizerProps);
+    propertyMap.putAll(samplerProps);
     propertyMap.putAll(properties);
     return Collections.unmodifiableMap(propertyMap);
   }
@@ -121,8 +128,22 @@ public class NewTableConfiguration {
    */
   public NewTableConfiguration enableSampling(SamplerConfiguration samplerConfiguration) {
     requireNonNull(samplerConfiguration);
-    SamplerConfigurationImpl.checkDisjoint(properties, samplerConfiguration);
-    this.samplerConfiguration = samplerConfiguration;
+    Map<String,String> tmp = new SamplerConfigurationImpl(samplerConfiguration).toTablePropertiesMap();
+    checkDisjoint(properties, tmp, "sampler");
+    this.samplerProps = tmp;
+    return this;
+  }
+
+  /**
+   * Enables creating summary statistics using {@link Summarizer}'s for the new table.
+   *
+   * @since 2.0.0
+   */
+  public NewTableConfiguration enableSummarization(SummarizerConfiguration... configs) {
+    requireNonNull(configs);
+    Map<String,String> tmp = SummarizerConfigurationUtil.toTablePropertiesMap(Arrays.asList(configs));
+    checkDisjoint(properties, tmp, "summarizer");
+    summarizerProps = tmp;
     return this;
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/admin/SummaryRetriever.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/SummaryRetriever.java b/core/src/main/java/org/apache/accumulo/core/client/admin/SummaryRetriever.java
new file mode 100644
index 0000000..8dcf048
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/SummaryRetriever.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.client.admin;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.Summary;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This interface allows configuring where and which summary data to retrieve before retrieving it.
+ *
+ * @since 2.0.0
+ */
+public interface SummaryRetriever {
+
+  /**
+   * Forces a flush of data in memory to files before summary data is retrieved. Data recently written to Accumulo may be in memory. Summary data is only
+   * retrieved from files. Therefore recently written data may not be represented in summaries, unless this options is set to true. This is optional and
+   * defaults to false.
+   *
+   * @return this
+   */
+  SummaryRetriever flush(boolean shouldFlush);
+
+  /**
+   * The start row is not inclusive. Calling this method is optional.
+   */
+  SummaryRetriever startRow(Text startRow);
+
+  /**
+   * The start row is not inclusive. Calling this method is optional.
+   */
+  SummaryRetriever startRow(CharSequence startRow);
+
+  /**
+   * The end row is inclusive. Calling this method is optional.
+   */
+  SummaryRetriever endRow(Text endRow);
+
+  /**
+   * The end row is inclusive. Calling this method is optional.
+   */
+  SummaryRetriever endRow(CharSequence endRow);
+
+  /**
+   * Filters which summary data is retrieved. By default all summary data present is retrieved. If only a subset of summary data is needed, then its best to be
+   * selective in order to avoid polluting summary data cache.
+   *
+   * <p>
+   * Each set of summary data is generated using a specific {@link SummarizerConfiguration}. The methods {@link #withConfiguration(Collection)} and
+   * {@link #withConfiguration(SummarizerConfiguration...)} allow selecting sets of summary data based on exact {@link SummarizerConfiguration} matches. This
+   * method enables less exact matching using regular expressions.
+   *
+   * <p>
+   * The regular expression passed to this method is used in the following way on the server side to match {@link SummarizerConfiguration} object. When a
+   * {@link SummarizerConfiguration} matches, the summary data generated using that configuration is returned.
+   *
+   * <pre>
+   * <code>
+   *    boolean doesConfigurationMatch(SummarizerConfiguration conf, String regex) {
+   *      // This is how conf is converted to a String in tablet servers for matching.
+   *      // The options are sorted to make writing regular expressions easier.
+   *      String confString = conf.getClassName()+" "+new TreeMap&lt;&gt;(conf.getOptions());
+   *      return Pattern.compile(regex).matcher(confString).matches();
+   *    }
+   * </code>
+   * </pre>
+   */
+  SummaryRetriever withMatchingConfiguration(String regex);
+
+  /**
+   * Allows specifying a set of summaries, generated using the specified configs, to retrieve. By default will retrieve all present.
+   *
+   * <p>
+   * Using this method to be more selective may pull less data in to the tablet servers summary cache.
+   */
+  SummaryRetriever withConfiguration(SummarizerConfiguration... config);
+
+  /**
+   * Allows specifying a set of summaries, generated using the specified configs, to retrieve. By default will retrieve all present.
+   *
+   * <p>
+   * Using this method to be more selective may pull less data in to the tablet servers summary cache.
+   */
+  SummaryRetriever withConfiguration(Collection<SummarizerConfiguration> configs);
+
+  /**
+   * @return a map of counter groups to counts
+   */
+  List<Summary> retrieve() throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
index cabcfa3..f88d28e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
+import java.util.function.Predicate;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -34,9 +35,12 @@ import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.mapreduce.AccumuloFileOutputFormat;
 import org.apache.accumulo.core.client.rfile.RFile;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TablePermission;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -808,4 +812,65 @@ public interface TableOperations {
    * @since 1.8.0
    */
   SamplerConfiguration getSamplerConfiguration(String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
+
+  /**
+   * Entry point for retrieving summaries with optional restrictions.
+   *
+   * <p>
+   * In order to retrieve Summaries, the Accumulo user making the request will need the {@link TablePermission#GET_SUMMARIES} table permission.
+   *
+   * <p>
+   * Accumulo stores summary data with each file in each tablet. In order to make retrieving it faster there is a per tablet server cache of summary data. When
+   * summary data for a file is not present, it will be retrieved using threads on the tserver. The tablet server properties
+   * {@code tserver.summary.partition.threads}, {@code tserver.summary.remote.threads}, {@code tserver.summary.retrieval.threads}, and
+   * {@code tserver.cache.summary.size} impact the performance of retrieving summaries.
+   *
+   * <p>
+   * Since summary data is cached, its important to use the summary selection options to only read the needed data into the cache.
+   *
+   * <p>
+   * Summary data will be merged on the tablet servers and then in this client process. Therefore it's important that the required summarizers are on the
+   * clients classpath.
+   *
+   * @since 2.0.0
+   * @see Summarizer
+   */
+  SummaryRetriever summaries(String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
+
+  /**
+   * Enables summary generation for this table for future compactions.
+   *
+   * @param tableName
+   *          add summarizers to this table
+   * @param summarizers
+   *          summarizers to add
+   * @throws IllegalArgumentException
+   *           When new summarizers have the same property id as each other, or when the same summarizers previously added.
+   * @since 2.0.0
+   * @see SummarizerConfiguration#toTableProperties()
+   * @see SummarizerConfiguration#toTableProperties(SummarizerConfiguration...)
+   * @see SummarizerConfiguration#toTableProperties(Collection)
+   */
+  void addSummarizers(String tableName, SummarizerConfiguration... summarizers) throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
+
+  /**
+   * Removes summary generation for this table for the matching summarizers.
+   *
+   * @param tableName
+   *          remove summarizers from this table
+   * @param predicate
+   *          removes all summarizers whose configuration that matches this predicate
+   * @since 2.0.0
+   */
+  void removeSummarizers(String tableName, Predicate<SummarizerConfiguration> predicate) throws AccumuloException, TableNotFoundException,
+      AccumuloSecurityException;
+
+  /**
+   * @param tableName
+   *          list summarizers for this table
+   * @return the summarizers currently configured for the table
+   * @since 2.0.0
+   * @see SummarizerConfiguration#fromTableProperties(Map)
+   */
+  List<SummarizerConfiguration> listSummarizers(String tableName) throws AccumuloException, TableNotFoundException, AccumuloSecurityException;
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
index a4853f0..9d18f99 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
@@ -36,6 +36,9 @@ import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TServiceClient;
+import org.apache.thrift.TServiceClientFactory;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
@@ -47,8 +50,13 @@ public class ServerClient {
   private static final Logger log = LoggerFactory.getLogger(ServerClient.class);
 
   public static <T> T execute(ClientContext context, ClientExecReturn<T,ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
+    return execute(context, new ClientService.Client.Factory(), exec);
+  }
+
+  public static <CT extends TServiceClient,RT> RT execute(ClientContext context, TServiceClientFactory<CT> factory, ClientExecReturn<RT,CT> exec)
+      throws AccumuloException, AccumuloSecurityException {
     try {
-      return executeRaw(context, exec);
+      return executeRaw(context, factory, exec);
     } catch (ThriftSecurityException e) {
       throw new AccumuloSecurityException(e.user, e.code, e);
     } catch (AccumuloException e) {
@@ -71,14 +79,21 @@ public class ServerClient {
   }
 
   public static <T> T executeRaw(ClientContext context, ClientExecReturn<T,ClientService.Client> exec) throws Exception {
+    return executeRaw(context, new ClientService.Client.Factory(), exec);
+  }
+
+  public static <CT extends TServiceClient,RT> RT executeRaw(ClientContext context, TServiceClientFactory<CT> factory, ClientExecReturn<RT,CT> exec)
+      throws Exception {
     while (true) {
-      ClientService.Client client = null;
+      CT client = null;
       String server = null;
       try {
-        Pair<String,Client> pair = ServerClient.getConnection(context);
+        Pair<String,CT> pair = ServerClient.getConnection(context, factory);
         server = pair.getFirst();
         client = pair.getSecond();
         return exec.execute(client);
+      } catch (TApplicationException tae) {
+        throw new AccumuloServerException(server, tae);
       } catch (TTransportException tte) {
         log.debug("ClientService request failed " + server + ", retrying ... ", tte);
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
@@ -99,6 +114,8 @@ public class ServerClient {
         client = pair.getSecond();
         exec.execute(client);
         break;
+      } catch (TApplicationException tae) {
+        throw new AccumuloServerException(server, tae);
       } catch (TTransportException tte) {
         log.debug("ClientService request failed " + server + ", retrying ... ", tte);
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
@@ -115,12 +132,21 @@ public class ServerClient {
     return getConnection(context, true);
   }
 
+  public static <CT extends TServiceClient> Pair<String,CT> getConnection(ClientContext context, TServiceClientFactory<CT> factory) throws TTransportException {
+    return getConnection(context, factory, true, context.getClientTimeoutInMillis());
+  }
+
   public static Pair<String,ClientService.Client> getConnection(ClientContext context, boolean preferCachedConnections) throws TTransportException {
     return getConnection(context, preferCachedConnections, context.getClientTimeoutInMillis());
   }
 
   public static Pair<String,ClientService.Client> getConnection(ClientContext context, boolean preferCachedConnections, long rpcTimeout)
       throws TTransportException {
+    return getConnection(context, new ClientService.Client.Factory(), preferCachedConnections, rpcTimeout);
+  }
+
+  public static <CT extends TServiceClient> Pair<String,CT> getConnection(ClientContext context, TServiceClientFactory<CT> factory,
+      boolean preferCachedConnections, long rpcTimeout) throws TTransportException {
     checkArgument(context != null, "context is null");
     // create list of servers
     ArrayList<ThriftTransportKey> servers = new ArrayList<>();
@@ -141,7 +167,7 @@ public class ServerClient {
     boolean opened = false;
     try {
       Pair<String,TTransport> pair = ThriftTransportPool.getInstance().getAnyTransport(servers, preferCachedConnections);
-      ClientService.Client client = ThriftUtil.createClient(new ClientService.Client.Factory(), pair.getSecond());
+      CT client = ThriftUtil.createClient(factory, pair.getSecond());
       opened = true;
       warnedAboutTServersBeingDown = false;
       return new Pair<>(pair.getFirst(), client);
@@ -159,7 +185,7 @@ public class ServerClient {
     }
   }
 
-  public static void close(ClientService.Client client) {
+  public static void close(TServiceClient client) {
     if (client != null && client.getInputProtocol() != null && client.getInputProtocol().getTransport() != null) {
       ThriftTransportPool.getInstance().returnTransport(client.getInputProtocol().getTransport());
     } else {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
index 1c04a43..34b76fc 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
@@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Objects.requireNonNull;
+import static java.util.stream.Collectors.toSet;
 
 import java.io.BufferedReader;
 import java.io.FileNotFoundException;
@@ -38,6 +39,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Objects;
 import java.util.Random;
 import java.util.Set;
 import java.util.SortedSet;
@@ -48,6 +50,9 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipInputStream;
 
@@ -69,16 +74,19 @@ import org.apache.accumulo.core.client.admin.DiskUsage;
 import org.apache.accumulo.core.client.admin.FindMax;
 import org.apache.accumulo.core.client.admin.Locations;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.client.admin.SummaryRetriever;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.ClientService.Client;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.client.impl.thrift.TDiskUsage;
 import org.apache.accumulo.core.client.impl.thrift.ThriftNotActiveServiceException;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.Summary;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
@@ -90,6 +98,10 @@ import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.data.impl.TabletIdImpl;
+import org.apache.accumulo.core.data.thrift.TRowRange;
+import org.apache.accumulo.core.data.thrift.TSummaries;
+import org.apache.accumulo.core.data.thrift.TSummarizerConfiguration;
+import org.apache.accumulo.core.data.thrift.TSummaryRequest;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -103,6 +115,8 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.summary.SummarizerConfigurationUtil;
+import org.apache.accumulo.core.summary.SummaryCollection;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.Tracer;
@@ -126,6 +140,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
 import com.google.common.net.HostAndPort;
 
 public class TableOperationsImpl extends TableOperationsHelper {
@@ -1661,4 +1676,138 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     return new LoctionsImpl(binnedRanges);
   }
+
+  @Override
+  public SummaryRetriever summaries(String tableName) {
+
+    return new SummaryRetriever() {
+
+      private Text startRow = null;
+      private Text endRow = null;
+      private List<TSummarizerConfiguration> summariesToFetch = Collections.emptyList();
+      private String summarizerClassRegex;
+      private boolean flush = false;
+
+      @Override
+      public SummaryRetriever startRow(Text startRow) {
+        Objects.requireNonNull(startRow);
+        if (endRow != null) {
+          Preconditions.checkArgument(startRow.compareTo(endRow) < 0, "Start row must be less than end row : %s >= %s", startRow, endRow);
+        }
+        this.startRow = startRow;
+        return this;
+      }
+
+      @Override
+      public SummaryRetriever startRow(CharSequence startRow) {
+        return startRow(new Text(startRow.toString()));
+      }
+
+      @Override
+      public List<Summary> retrieve() throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+        String tableId = Tables.getTableId(context.getInstance(), tableName);
+        if (Tables.getTableState(context.getInstance(), tableId) == TableState.OFFLINE)
+          throw new TableOfflineException(context.getInstance(), tableId);
+
+        TRowRange range = new TRowRange(TextUtil.getByteBuffer(startRow), TextUtil.getByteBuffer(endRow));
+        TSummaryRequest request = new TSummaryRequest(tableId, range, summariesToFetch, summarizerClassRegex);
+        if (flush) {
+          _flush(tableId, startRow, endRow, true);
+        }
+
+        TSummaries ret = ServerClient.execute(context, new TabletClientService.Client.Factory(), client -> {
+          TSummaries tsr = client.startGetSummaries(Tracer.traceInfo(), context.rpcCreds(), request);
+          while (!tsr.finished) {
+            tsr = client.contiuneGetSummaries(Tracer.traceInfo(), tsr.sessionId);
+          }
+          return tsr;
+        });
+        return new SummaryCollection(ret).getSummaries();
+      }
+
+      @Override
+      public SummaryRetriever endRow(Text endRow) {
+        Objects.requireNonNull(endRow);
+        if (startRow != null) {
+          Preconditions.checkArgument(startRow.compareTo(endRow) < 0, "Start row must be less than end row : %s >= %s", startRow, endRow);
+        }
+        this.endRow = endRow;
+        return this;
+      }
+
+      @Override
+      public SummaryRetriever endRow(CharSequence endRow) {
+        return endRow(new Text(endRow.toString()));
+      }
+
+      @Override
+      public SummaryRetriever withConfiguration(Collection<SummarizerConfiguration> configs) {
+        Objects.requireNonNull(configs);
+        summariesToFetch = configs.stream().map(SummarizerConfigurationUtil::toThrift).collect(Collectors.toList());
+        return this;
+      }
+
+      @Override
+      public SummaryRetriever withConfiguration(SummarizerConfiguration... config) {
+        Objects.requireNonNull(config);
+        return withConfiguration(Arrays.asList(config));
+      }
+
+      @Override
+      public SummaryRetriever withMatchingConfiguration(String regex) {
+        Objects.requireNonNull(regex);
+        // Do a sanity check here to make sure that regex compiles, instead of having it fail on a tserver.
+        Pattern.compile(regex);
+        this.summarizerClassRegex = regex;
+        return this;
+      }
+
+      @Override
+      public SummaryRetriever flush(boolean b) {
+        this.flush = b;
+        return this;
+      }
+    };
+  }
+
+  @Override
+  public void addSummarizers(String tableName, SummarizerConfiguration... newConfigs) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
+    HashSet<SummarizerConfiguration> currentConfigs = new HashSet<>(SummarizerConfiguration.fromTableProperties(getProperties(tableName)));
+    HashSet<SummarizerConfiguration> newConfigSet = new HashSet<>(Arrays.asList(newConfigs));
+
+    newConfigSet.removeIf(sc -> currentConfigs.contains(sc));
+
+    Set<String> newIds = newConfigSet.stream().map(sc -> sc.getPropertyId()).collect(toSet());
+
+    for (SummarizerConfiguration csc : currentConfigs) {
+      if (newIds.contains(csc.getPropertyId())) {
+        throw new IllegalArgumentException("Summarizer property id is in use by " + csc);
+      }
+    }
+
+    Set<Entry<String,String>> es = SummarizerConfiguration.toTableProperties(newConfigSet).entrySet();
+    for (Entry<String,String> entry : es) {
+      setProperty(tableName, entry.getKey(), entry.getValue());
+    }
+  }
+
+  @Override
+  public void removeSummarizers(String tableName, Predicate<SummarizerConfiguration> predicate) throws AccumuloException, TableNotFoundException,
+      AccumuloSecurityException {
+    Collection<SummarizerConfiguration> summarizerConfigs = SummarizerConfiguration.fromTableProperties(getProperties(tableName));
+    for (SummarizerConfiguration sc : summarizerConfigs) {
+      if (predicate.test(sc)) {
+        Set<String> ks = sc.toTableProperties().keySet();
+        for (String key : ks) {
+          removeProperty(tableName, key);
+        }
+      }
+    }
+  }
+
+  @Override
+  public List<SummarizerConfiguration> listSummarizers(String tableName) throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
+    return new ArrayList<>(SummarizerConfiguration.fromTableProperties(getProperties(tableName)));
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
index 640a85d..d7d2b2d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
@@ -23,6 +23,8 @@ import org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator
 import org.apache.accumulo.core.client.rfile.RFile;
 import org.apache.accumulo.core.client.rfile.RFileWriter;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -136,6 +138,20 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
     FileOutputConfigurator.setSampler(CLASS, job, samplerConfig);
   }
 
+  /**
+   * Specifies a list of summarizer configurations to create summary data in the output file. Each Key Value written will be passed to the configured
+   * {@link Summarizer}'s.
+   *
+   * @param job
+   *          The Hadoop job instance to be configured
+   * @param sumarizerConfigs
+   *          summarizer configurations
+   * @since 2.0.0
+   */
+  public static void setSummarizers(JobConf job, SummarizerConfiguration... sumarizerConfigs) {
+    FileOutputConfigurator.setSummarizers(CLASS, job, sumarizerConfigs);
+  }
+
   @Override
   public RecordWriter<Key,Value> getRecordWriter(FileSystem ignored, JobConf job, String name, Progressable progress) throws IOException {
     // get the path of the temporary output file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
index 656dba7..dcdd42b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
@@ -23,6 +23,8 @@ import org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator
 import org.apache.accumulo.core.client.rfile.RFile;
 import org.apache.accumulo.core.client.rfile.RFileWriter;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -134,6 +136,20 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
     FileOutputConfigurator.setSampler(CLASS, job.getConfiguration(), samplerConfig);
   }
 
+  /**
+   * Specifies a list of summarizer configurations to create summary data in the output file. Each Key Value written will be passed to the configured
+   * {@link Summarizer}'s.
+   *
+   * @param job
+   *          The Hadoop job instance to be configured
+   * @param sumarizerConfigs
+   *          summarizer configurations
+   * @since 2.0.0
+   */
+  public static void setSummarizers(Job job, SummarizerConfiguration... sumarizerConfigs) {
+    FileOutputConfigurator.setSummarizers(CLASS, job.getConfiguration(), sumarizerConfigs);
+  }
+
   @Override
   public RecordWriter<Key,Value> getRecordWriter(TaskAttemptContext context) throws IOException {
     // get the path of the temporary output file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java
index 049395f..5f73e90 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java
@@ -22,6 +22,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
@@ -209,4 +210,12 @@ public class FileOutputConfigurator extends ConfiguratorBase {
     }
   }
 
+  public static void setSummarizers(Class<?> implementingClass, Configuration conf, SummarizerConfiguration[] sumarizerConfigs) {
+    Map<String,String> props = SummarizerConfiguration.toTableProperties(sumarizerConfigs);
+
+    for (Entry<String,String> entry : props.entrySet()) {
+      conf.set(enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + "." + entry.getKey(), entry.getValue());
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
index de89137..de486d7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
@@ -29,6 +29,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.function.Predicate;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -41,10 +42,12 @@ import org.apache.accumulo.core.client.admin.DiskUsage;
 import org.apache.accumulo.core.client.admin.FindMax;
 import org.apache.accumulo.core.client.admin.Locations;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.client.admin.SummaryRetriever;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.client.impl.TableOperationsHelper;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -502,4 +505,28 @@ class MockTableOperations extends TableOperationsHelper {
   public Locations locate(String tableName, Collection<Range> ranges) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public SummaryRetriever summaries(String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void addSummarizers(String tableName, SummarizerConfiguration... summarizerConf) throws TableNotFoundException, AccumuloException,
+      AccumuloSecurityException {
+    throw new UnsupportedOperationException();
+
+  }
+
+  @Override
+  public void removeSummarizers(String tableName, Predicate<SummarizerConfiguration> predicate) throws AccumuloException, TableNotFoundException,
+      AccumuloSecurityException {
+    throw new UnsupportedOperationException();
+
+  }
+
+  @Override
+  public List<SummarizerConfiguration> listSummarizers(String tableName) throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
index bc5995e..7c3f70e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
@@ -19,12 +19,18 @@ package org.apache.accumulo.core.client.rfile;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.util.Collection;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.function.Predicate;
 
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.Summary;
+import org.apache.accumulo.core.client.summary.Summary.FileStatistics;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Authorizations;
@@ -181,6 +187,119 @@ public class RFile {
   }
 
   /**
+   * This is an intermediate interface in a larger builder pattern. Supports setting the required input sources for reading summary data from an RFile.
+   *
+   * @since 2.0.0
+   */
+  public static interface SummaryInputArguments {
+    /**
+     * Specify RFiles to read from. When multiple inputs are specified the summary data will be merged.
+     *
+     * @param inputs
+     *          one or more RFiles to read.
+     * @return this
+     */
+    SummaryOptions from(RFileSource... inputs);
+
+    /**
+     * Specify RFiles to read from. When multiple are specified the summary data will be merged.
+     *
+     * @param files
+     *          one or more RFiles to read.
+     * @return this
+     */
+    SummaryFSOptions from(String... files);
+  }
+
+  /**
+   * This is an intermediate interface in a larger builder pattern. Enables optionally setting a FileSystem to read RFile summary data from.
+   *
+   * @since 2.0.0
+   */
+  public static interface SummaryFSOptions extends SummaryOptions {
+    /**
+     * Optionally provide a FileSystem to open RFiles. If not specified, the FileSystem will be constructed using configuration on the classpath.
+     *
+     * @param fs
+     *          use this FileSystem to open files.
+     * @return this
+     */
+    SummaryOptions withFileSystem(FileSystem fs);
+  }
+
+  /**
+   * This is an intermediate interface in a large builder pattern. Allows setting options for retrieving summary data.
+   *
+   * @since 2.0.0
+   */
+  public static interface SummaryOptions {
+    /**
+     * This method allows retrieving a subset of summary data from a file. If a file has lots of separate summaries, reading a subset may be faster.
+     *
+     * @param summarySelector
+     *          Only read summary data that was generated with configuration that this predicate matches.
+     * @return this
+     */
+    SummaryOptions selectSummaries(Predicate<SummarizerConfiguration> summarySelector);
+
+    /**
+     * Summary data may possibly be stored at a more granular level than the entire file. However there is no guarantee of this. If the data was stored at a
+     * more granular level, then this will get a subset of the summary data. The subset will very likely be an inaccurate approximation.
+     *
+     * @param startRow
+     *          A non-null start row. The startRow is used exclusively.
+     * @return this
+     *
+     * @see FileStatistics#getExtra()
+     */
+    SummaryOptions startRow(Text startRow);
+
+    /**
+     * @param startRow
+     *          UTF-8 encodes startRow. The startRow is used exclusively.
+     * @return this
+     * @see #startRow(Text)
+     */
+    SummaryOptions startRow(CharSequence startRow);
+
+    /**
+     * Summary data may possibly be stored at a more granular level than the entire file. However there is no guarantee of this. If the data was stored at a
+     * more granular level, then this will get a subset of the summary data. The subset will very likely be an inaccurate approximation.
+     *
+     * @param endRow
+     *          A non-null end row. The end row is used inclusively.
+     * @return this
+     *
+     * @see FileStatistics#getExtra()
+     */
+    SummaryOptions endRow(Text endRow);
+
+    /**
+     * @param endRow
+     *          UTF-8 encodes endRow. The end row is used inclusively.
+     * @return this
+     * @see #endRow(Text)
+     */
+    SummaryOptions endRow(CharSequence endRow);
+
+    /**
+     * Reads summary data from file.
+     *
+     * @return The summary data in the file that satisfied the selection criteria.
+     */
+    Collection<Summary> read() throws IOException;
+  }
+
+  /**
+   * Entry point for reading summary data from RFiles.
+   *
+   * @since 2.0.0
+   */
+  public static SummaryInputArguments summaries() {
+    return new RFileSummariesRetriever();
+  }
+
+  /**
    * This is an intermediate interface in a larger builder pattern. Supports setting the required output sink to write a RFile to.
    *
    * @since 1.8.0
@@ -224,6 +343,16 @@ public class RFile {
    * @since 1.8.0
    */
   public static interface WriterOptions {
+
+    /**
+     * Enable generating summary data in the created RFile by running {@link Summarizer}'s based on the specified configuration.
+     *
+     * @param summarizerConf
+     *          Configuration for summarizer to run.
+     * @since 2.0.0
+     */
+    public WriterOptions withSummarizers(SummarizerConfiguration... summarizerConf);
+
     /**
      * An option to store sample data in the generated RFile.
      *

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
index 3a55172..cfd331a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
@@ -112,12 +112,14 @@ class RFileScannerBuilder implements RFile.InputArguments, RFile.ScannerFSOption
 
   @Override
   public ScannerOptions from(RFileSource... inputs) {
+    Objects.requireNonNull(inputs);
     opts.in = new InputArgs(inputs);
     return this;
   }
 
   @Override
   public ScannerFSOptions from(String... files) {
+    Objects.requireNonNull(files);
     opts.in = new InputArgs(files);
     return this;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSummariesRetriever.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSummariesRetriever.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSummariesRetriever.java
new file mode 100644
index 0000000..367172a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSummariesRetriever.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.client.rfile;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.function.Predicate;
+
+import org.apache.accumulo.core.client.rfile.RFile.SummaryFSOptions;
+import org.apache.accumulo.core.client.rfile.RFile.SummaryInputArguments;
+import org.apache.accumulo.core.client.rfile.RFile.SummaryOptions;
+import org.apache.accumulo.core.client.rfile.RFileScannerBuilder.InputArgs;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.Summary;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.summary.SummaryReader;
+import org.apache.accumulo.core.summary.Gatherer;
+import org.apache.accumulo.core.summary.SummarizerFactory;
+import org.apache.accumulo.core.summary.SummaryCollection;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.Text;
+
+class RFileSummariesRetriever implements SummaryInputArguments, SummaryFSOptions, SummaryOptions {
+
+  private Predicate<SummarizerConfiguration> summarySelector = sc -> true;
+  private Text startRow;
+  private InputArgs in;
+  private Text endRow;
+
+  @Override
+  public SummaryOptions selectSummaries(Predicate<SummarizerConfiguration> summarySelector) {
+    Objects.requireNonNull(summarySelector);
+    this.summarySelector = summarySelector;
+    return this;
+  }
+
+  @Override
+  public SummaryOptions startRow(CharSequence startRow) {
+    return startRow(new Text(startRow.toString()));
+  }
+
+  @Override
+  public SummaryOptions startRow(Text startRow) {
+    Objects.requireNonNull(startRow);
+    this.startRow = startRow;
+    return this;
+  }
+
+  @Override
+  public SummaryOptions endRow(CharSequence endRow) {
+    return endRow(new Text(endRow.toString()));
+  }
+
+  @Override
+  public SummaryOptions endRow(Text endRow) {
+    Objects.requireNonNull(endRow);
+    this.endRow = endRow;
+    return this;
+  }
+
+  @Override
+  public Collection<Summary> read() throws IOException {
+    SummarizerFactory factory = new SummarizerFactory();
+    AccumuloConfiguration acuconf = AccumuloConfiguration.getDefaultConfiguration();
+    Configuration conf = in.getFileSystem().getConf();
+
+    RFileSource[] sources = in.getSources();
+    try {
+      SummaryCollection all = new SummaryCollection();
+      for (RFileSource source : in.getSources()) {
+        SummaryReader fileSummary = SummaryReader.load(conf, acuconf, source.getInputStream(), source.getLength(), summarySelector, factory);
+        SummaryCollection sc = fileSummary.getSummaries(Collections.singletonList(new Gatherer.RowRange(startRow, endRow)));
+        all.merge(sc, factory);
+      }
+
+      return all.getSummaries();
+    } finally {
+      for (RFileSource source : sources) {
+        source.getInputStream().close();
+      }
+    }
+  }
+
+  @Override
+  public SummaryOptions withFileSystem(FileSystem fs) {
+    Objects.requireNonNull(fs);
+    this.in.fs = fs;
+    return this;
+  }
+
+  @Override
+  public SummaryOptions from(RFileSource... inputs) {
+    Objects.requireNonNull(inputs);
+    in = new InputArgs(inputs);
+    return this;
+  }
+
+  @Override
+  public SummaryFSOptions from(String... files) {
+    Objects.requireNonNull(files);
+    in = new InputArgs(files);
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriter.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriter.java
index 9995888..9ae7fb0 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriter.java
@@ -210,6 +210,32 @@ public class RFileWriter implements AutoCloseable {
   }
 
   /**
+   * This method has the same behavior as {@link #append(Key, Value)}.
+   *
+   * @param key
+   *          Same restrictions on key as {@link #append(Key, Value)}.
+   * @param value
+   *          this parameter will be UTF-8 encoded. Must be non-null.
+   * @since 2.0.0
+   */
+  public void append(Key key, CharSequence value) throws IOException {
+    append(key, new Value(value));
+  }
+
+  /**
+   * This method has the same behavior as {@link #append(Key, Value)}.
+   *
+   * @param key
+   *          Same restrictions on key as {@link #append(Key, Value)}.
+   * @param value
+   *          Must be non-null.
+   * @since 2.0.0
+   */
+  public void append(Key key, byte[] value) throws IOException {
+    append(key, new Value(value));
+  }
+
+  /**
    * Append the keys and values to the last locality group that was started.
    *
    * @param keyValues

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
index 667cbef..a7decb1 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
@@ -17,6 +17,8 @@
 
 package org.apache.accumulo.core.client.rfile;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Collections;
@@ -28,6 +30,7 @@ import java.util.Objects;
 import org.apache.accumulo.core.client.rfile.RFile.WriterFSOptions;
 import org.apache.accumulo.core.client.rfile.RFile.WriterOptions;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.file.FileOperations;
@@ -59,15 +62,21 @@ class RFileWriterBuilder implements RFile.OutputArguments, RFile.WriterFSOptions
   }
 
   private OutputArgs out;
-  private SamplerConfiguration sampler = null;
   private Map<String,String> tableConfig = Collections.emptyMap();
   private int visCacheSize = 1000;
+  private Map<String,String> samplerProps = Collections.emptyMap();
+  private Map<String,String> summarizerProps = Collections.emptyMap();
+
+  private void checkDisjoint(Map<String,String> props, Map<String,String> derivedProps, String kind) {
+    checkArgument(Collections.disjoint(props.keySet(), derivedProps.keySet()), "Properties and derived %s properties are not disjoint", kind);
+  }
 
   @Override
   public WriterOptions withSampler(SamplerConfiguration samplerConf) {
     Objects.requireNonNull(samplerConf);
-    SamplerConfigurationImpl.checkDisjoint(tableConfig, samplerConf);
-    this.sampler = samplerConf;
+    Map<String,String> tmp = new SamplerConfigurationImpl(samplerConf).toTablePropertiesMap();
+    checkDisjoint(tableConfig, tmp, "sampler");
+    this.samplerProps = tmp;
     return this;
   }
 
@@ -76,10 +85,10 @@ class RFileWriterBuilder implements RFile.OutputArguments, RFile.WriterFSOptions
     FileOperations fileops = FileOperations.getInstance();
     AccumuloConfiguration acuconf = AccumuloConfiguration.getDefaultConfiguration();
     HashMap<String,String> userProps = new HashMap<>();
-    if (sampler != null) {
-      userProps.putAll(new SamplerConfigurationImpl(sampler).toTablePropertiesMap());
-    }
+
     userProps.putAll(tableConfig);
+    userProps.putAll(summarizerProps);
+    userProps.putAll(samplerProps);
 
     if (userProps.size() > 0) {
       acuconf = new ConfigurationCopy(Iterables.concat(acuconf, userProps.entrySet()));
@@ -92,10 +101,11 @@ class RFileWriterBuilder implements RFile.OutputArguments, RFile.WriterFSOptions
       } else {
         fsdo = new FSDataOutputStream(out.getOutputStream(), new FileSystem.Statistics("foo"));
       }
-      return new RFileWriter(fileops.newWriterBuilder().forOutputStream(".rf", fsdo, out.getConf()).withTableConfiguration(acuconf).build(), visCacheSize);
+      return new RFileWriter(fileops.newWriterBuilder().forOutputStream(".rf", fsdo, out.getConf()).withTableConfiguration(acuconf)
+          .setAccumuloStartEnabled(false).build(), visCacheSize);
     } else {
       return new RFileWriter(fileops.newWriterBuilder().forFile(out.path.toString(), out.getFileSystem(), out.getConf()).withTableConfiguration(acuconf)
-          .build(), visCacheSize);
+          .setAccumuloStartEnabled(false).build(), visCacheSize);
     }
   }
 
@@ -128,7 +138,8 @@ class RFileWriterBuilder implements RFile.OutputArguments, RFile.WriterFSOptions
       cfg.put(entry.getKey(), entry.getValue());
     }
 
-    SamplerConfigurationImpl.checkDisjoint(cfg, sampler);
+    checkDisjoint(cfg, samplerProps, "sampler");
+    checkDisjoint(cfg, summarizerProps, "summarizer");
     this.tableConfig = cfg;
     return this;
   }
@@ -145,4 +156,13 @@ class RFileWriterBuilder implements RFile.OutputArguments, RFile.WriterFSOptions
     this.visCacheSize = maxSize;
     return this;
   }
+
+  @Override
+  public WriterOptions withSummarizers(SummarizerConfiguration... summarizerConf) {
+    Objects.requireNonNull(summarizerConf);
+    Map<String,String> tmp = SummarizerConfiguration.toTableProperties(summarizerConf);
+    checkDisjoint(tableConfig, tmp, "summarizer");
+    this.summarizerProps = tmp;
+    return this;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/summary/CounterSummary.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/summary/CounterSummary.java b/core/src/main/java/org/apache/accumulo/core/client/summary/CounterSummary.java
new file mode 100644
index 0000000..a0f9bc5
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/summary/CounterSummary.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.client.summary;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This a convenience class for interpreting summary data generated by implementations of {@link CountingSummarizer}
+ *
+ * @since 2.0.0
+ */
+
+public class CounterSummary {
+  private Map<String,Long> stats;
+
+  /**
+   * This method will call {@link #CounterSummary(Summary, boolean)} with true.
+   */
+  public CounterSummary(Summary summary) {
+    this(summary, true);
+  }
+
+  /**
+   * @param summary
+   *          a summary
+   * @param checkType
+   *          If true will try to ensure the classname from {@link Summary#getSummarizerConfiguration()} is an instance of {@link CountingSummarizer}. However
+   *          this check can only succeed if the class is on the classpath. For cases where the summary data needs to be used and the class is not on the
+   *          classpath, set this to false.
+   */
+  public CounterSummary(Summary summary, boolean checkType) {
+    if (checkType) {
+      String className = summary.getSummarizerConfiguration().getClassName();
+      try {
+        getClass().getClassLoader().loadClass(className).asSubclass(CountingSummarizer.class);
+      } catch (ClassCastException e) {
+        throw new IllegalArgumentException(className + " is not an instance of " + CountingSummarizer.class.getSimpleName(), e);
+      } catch (ClassNotFoundException e) {
+        throw new IllegalArgumentException("Unable to check summary was produced by a " + CountingSummarizer.class.getSimpleName(), e);
+      }
+    }
+    this.stats = summary.getStatistics();
+  }
+
+  @VisibleForTesting
+  CounterSummary(Map<String,Long> stats) {
+    this.stats = stats;
+  }
+
+  /**
+   * @return statistic for {@link CountingSummarizer#SEEN_STAT}
+   */
+  public long getSeen() {
+    return stats.getOrDefault(CountingSummarizer.SEEN_STAT, 0l);
+  }
+
+  /**
+   * @return statistic for {@link CountingSummarizer#EMITTED_STAT}
+   */
+  public long getEmitted() {
+    return stats.getOrDefault(CountingSummarizer.EMITTED_STAT, 0l);
+  }
+
+  /**
+   * @return the sum of {@link #getTooLong()} and {@link #getTooLong()}
+   */
+  public long getIgnored() {
+    return getTooLong() + getTooMany();
+  }
+
+  /**
+   * @return statistic for {@link CountingSummarizer#TOO_LONG_STAT}
+   */
+  public long getTooLong() {
+    return stats.getOrDefault(CountingSummarizer.TOO_LONG_STAT, 0l);
+  }
+
+  /**
+   * @return statistic for {@link CountingSummarizer#TOO_MANY_STAT}
+   */
+  public long getTooMany() {
+    return stats.getOrDefault(CountingSummarizer.TOO_MANY_STAT, 0l);
+  }
+
+  /**
+   * @return statistic for {@link CountingSummarizer#DELETES_IGNORED_STAT}
+   */
+  public long getDeletesIgnored() {
+    return stats.getOrDefault(CountingSummarizer.DELETES_IGNORED_STAT, 0l);
+  }
+
+  /**
+   * @return All statistics with a prefix of {@link CountingSummarizer#COUNTER_STAT_PREFIX} with the prefix stripped off.
+   */
+  public Map<String,Long> getCounters() {
+    HashMap<String,Long> ret = new HashMap<>();
+    for (Entry<String,Long> entry : stats.entrySet()) {
+      if (entry.getKey().startsWith(CountingSummarizer.COUNTER_STAT_PREFIX)) {
+        ret.put(entry.getKey().substring(CountingSummarizer.COUNTER_STAT_PREFIX.length()), entry.getValue());
+      }
+    }
+    return ret;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/summary/CountingSummarizer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/summary/CountingSummarizer.java b/core/src/main/java/org/apache/accumulo/core/client/summary/CountingSummarizer.java
new file mode 100644
index 0000000..b3e1b68
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/summary/CountingSummarizer.java
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.summary;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.UnaryOperator;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.commons.lang.mutable.MutableLong;
+
+//checkstyle and formatter are in conflict
+//@formatter:off
+/**
+ * This class counts arbitrary keys while defending against too many keys and keys that are too long.
+ *
+ * <p>
+ * During collection and summarization this class will use the functions from {@link #converter()} and {@link #encoder()}. For each key/value the function from
+ * {@link #converter()} will be called to create zero or more counter objects. A counter associated with each counter object will be incremented, as long as
+ * there are not too many counters and the counter object is not too long.
+ *
+ * <p>
+ * When {@link Summarizer.Collector#summarize(Summarizer.StatisticConsumer)} is called, the function from {@link #encoder()} will be used to convert counter
+ * objects to strings. These strings will be used to emit statistics. Overriding {@link #encoder()} is optional. One reason to override is if the counter object
+ * contains binary or special data. For example, a function that base64 encodes counter objects could be created.
+ *
+ * <p>
+ * If the counter key type is mutable, then consider overriding {@link #copier()}.
+ *
+ * <p>
+ * The function returned by {@link #converter()} will be called frequently and should be very efficient. The function returned by {@link #encoder()} will be
+ * called less frequently and can be more expensive. The reason these two functions exists is to avoid the conversion to string for each key value, if that
+ * conversion is unnecessary.
+ *
+ * <p>
+ * Below is an example implementation that counts column visibilities. This example avoids converting column visibility to string for each key/value. This
+ * example shows the source code for {@link VisibilitySummarizer}.
+ *
+ * <pre>
+ * <code>
+ *   public class VisibilitySummarizer extends CountingSummarizer&lt;ByteSequence&gt; {
+ *     &#064;Override
+ *     protected UnaryOperator&lt;ByteSequence&gt; copier() {
+ *       // ByteSequences are mutable, so override and provide a copy function
+ *       return ArrayByteSequence::new;
+ *     }
+ *
+ *     &#064;Override
+ *     protected Converter&lt;ByteSequence&gt; converter() {
+ *       return (key, val, consumer) -&gt; consumer.accept(key.getColumnVisibilityData());
+ *     }
+ *   }
+ * </code>
+ * </pre>
+ *
+ * @param <K>
+ *          The counter key type. This type must have good implementations of {@link Object#hashCode()} and {@link Object#equals(Object)}.
+ * @see CounterSummary
+ * @since 2.0.0
+ */
+//@formatter:on
+public abstract class CountingSummarizer<K> implements Summarizer {
+
+  /**
+   * A configuration option for specifying the maximum number of unique counters an instance of this summarizer should track. If not specified, a default of
+   * {@value #MAX_COUNTER_DEFAULT} will be used.
+   */
+  public static final String MAX_COUNTERS_OPT = "maxCounters";
+
+  /**
+   * A configuration option for specifying the maximum length of an individual counter key. If not specified, a default of {@value #MAX_CKL_DEFAULT} will be
+   * used.
+   */
+  public static final String MAX_COUNTER_LEN_OPT = "maxCounterLen";
+
+  /**
+   * A configuration option to determine if delete keys should be counted. If set to true then delete keys will not be passed to the {@link Converter} and the
+   * statistic {@value #DELETES_IGNORED_STAT} will track the number of deleted ignored. This options defaults to {@value #INGNORE_DELETES_DEFAULT}.
+   */
+  public static final String INGNORE_DELETES_OPT = "ignoreDeletes";
+
+  /**
+   * This prefixes all counters when emitting statistics in {@link Summarizer.Collector#summarize(Summarizer.StatisticConsumer)}.
+   */
+  public static final String COUNTER_STAT_PREFIX = "c:";
+
+  /**
+   * This is the name of the statistic that tracks how many counters objects were ignored because the number of unique counters was exceeded. The max number of
+   * unique counters is specified by {@link #MAX_COUNTERS_OPT}.
+   */
+  public static final String TOO_MANY_STAT = "tooMany";
+
+  /**
+   * This is the name of the statistic that tracks how many counter objects were ignored because they were too long. The maximum lenght is specified by
+   * {@link #MAX_COUNTER_LEN_OPT}.
+   */
+  public static final String TOO_LONG_STAT = "tooLong";
+
+  /**
+   * This is the name of the statistic that tracks the total number of counter objects emitted by the {@link Converter}. This includes emitted Counter objects
+   * that were ignored.
+   */
+  public static final String EMITTED_STAT = "emitted";
+
+  /**
+   * This is the name of the statistic that tracks the total number of deleted keys seen. This statistic is only incremented when the
+   * {@value #INGNORE_DELETES_OPT} option is set to true.
+   */
+  public static final String DELETES_IGNORED_STAT = "deletesIgnored";
+
+  /**
+   * This tracks the total number of key/values seen by the {@link Summarizer.Collector}
+   */
+  public static final String SEEN_STAT = "seen";
+
+  // this default can not be changed as persisted summary data depends on it. See the documentation about persistence in the Summarizer class javadoc.
+  public static final String MAX_COUNTER_DEFAULT = "1024";
+
+  // this default can not be changed as persisted summary data depends on it
+  public static final String MAX_CKL_DEFAULT = "128";
+
+  // this default can not be changed as persisted summary data depends on it
+  public static final String INGNORE_DELETES_DEFAULT = "true";
+
+  private static final String[] ALL_STATS = new String[] {TOO_LONG_STAT, TOO_MANY_STAT, EMITTED_STAT, SEEN_STAT, DELETES_IGNORED_STAT};
+
+  private int maxCounters;
+  private int maxCounterKeyLen;
+  private boolean ignoreDeletes;
+
+  private void init(SummarizerConfiguration conf) {
+    maxCounters = Integer.parseInt(conf.getOptions().getOrDefault(MAX_COUNTERS_OPT, MAX_COUNTER_DEFAULT));
+    maxCounterKeyLen = Integer.parseInt(conf.getOptions().getOrDefault(MAX_COUNTER_LEN_OPT, MAX_CKL_DEFAULT));
+    ignoreDeletes = Boolean.parseBoolean(conf.getOptions().getOrDefault(INGNORE_DELETES_OPT, INGNORE_DELETES_DEFAULT));
+  }
+
+  /**
+   * A function that converts key values to zero or more counter objects.
+   *
+   * @since 2.0.0
+   */
+  public static interface Converter<K> {
+    /**
+     * @param consumer
+     *          emit counter objects derived from key and value to this consumer
+     */
+    public void convert(Key k, Value v, Consumer<K> consumer);
+  }
+
+  /**
+   *
+   * @return A function that is used to convert each key value to zero or more counter objects. Each function returned should be independent.
+   */
+  protected abstract Converter<K> converter();
+
+  /**
+   * @return A function that is used to convert counter objects to String. The default function calls {@link Object#toString()} on the counter object.
+   */
+  protected Function<K,String> encoder() {
+    return Object::toString;
+  }
+
+  /**
+   * Override this if your key type is mutable and subject to change.
+   *
+   * @return a function that used to copy the counter object. This function is only used when the collector has never seen the counter object before. In this
+   *         case the collector needs to possibly copy the counter object before using as map key. The default implementation is the
+   *         {@link UnaryOperator#identity()} function.
+   */
+  protected UnaryOperator<K> copier() {
+    return UnaryOperator.identity();
+  }
+
+  @Override
+  public Collector collector(SummarizerConfiguration sc) {
+    init(sc);
+    return new Collector() {
+
+      // Map used for computing summary incrementally uses ByteSequence for key which is more efficient than converting String for each Key. The
+      // conversion to String is deferred until the summary is requested.
+
+      private Map<K,MutableLong> counters = new HashMap<>();
+      private long tooMany = 0;
+      private long tooLong = 0;
+      private long seen = 0;
+      private long emitted = 0;
+      private long deleted = 0;
+      private Converter<K> converter = converter();
+      private Function<K,String> encoder = encoder();
+      private UnaryOperator<K> copier = copier();
+
+      private void incrementCounter(K counter) {
+        emitted++;
+
+        MutableLong ml = counters.get(counter);
+        if (ml == null) {
+          if (counters.size() >= maxCounters) {
+            // no need to store this counter in the map and get() it... just use instance variable
+            tooMany++;
+          } else {
+            // we have never seen this key before, check if its too long
+            if (encoder.apply(counter).length() >= maxCounterKeyLen) {
+              tooLong++;
+            } else {
+              counters.put(copier.apply(counter), new MutableLong(1));
+            }
+          }
+        } else {
+          // using mutable long allows calling put() to be avoided
+          ml.increment();
+        }
+      }
+
+      @Override
+      public void accept(Key k, Value v) {
+        seen++;
+        if (ignoreDeletes && k.isDeleted()) {
+          deleted++;
+        } else {
+          converter.convert(k, v, this::incrementCounter);
+        }
+      }
+
+      @Override
+      public void summarize(StatisticConsumer sc) {
+        StringBuilder sb = new StringBuilder(COUNTER_STAT_PREFIX);
+
+        for (Entry<K,MutableLong> entry : counters.entrySet()) {
+          sb.setLength(COUNTER_STAT_PREFIX.length());
+          sb.append(encoder.apply(entry.getKey()));
+          sc.accept(sb.toString(), entry.getValue().longValue());
+        }
+
+        sc.accept(TOO_MANY_STAT, tooMany);
+        sc.accept(TOO_LONG_STAT, tooLong);
+        sc.accept(EMITTED_STAT, emitted);
+        sc.accept(SEEN_STAT, seen);
+        sc.accept(DELETES_IGNORED_STAT, deleted);
+      }
+    };
+  }
+
+  @Override
+  public Combiner combiner(SummarizerConfiguration sc) {
+    init(sc);
+    return new Combiner() {
+
+      @Override
+      public void merge(Map<String,Long> summary1, Map<String,Long> summary2) {
+
+        for (String key : ALL_STATS) {
+          summary1.merge(key, summary2.getOrDefault(key, 0l), Long::sum);
+        }
+
+        for (Entry<String,Long> entry : summary2.entrySet()) {
+          String k2 = entry.getKey();
+          Long v2 = entry.getValue();
+
+          if (k2.startsWith(COUNTER_STAT_PREFIX)) {
+            summary1.merge(k2, v2, Long::sum);
+          }
+        }
+
+        if (summary1.size() - ALL_STATS.length > maxCounters) {
+          // find the keys with the lowest counts to remove
+          List<String> keysToRemove = summary1.entrySet().stream().filter(e -> e.getKey().startsWith(COUNTER_STAT_PREFIX)) // filter out non counters
+              .sorted((e1, e2) -> Long.compare(e2.getValue(), e1.getValue())) // sort descending by count
+              .skip(maxCounters) // skip most frequent
+              .map(e -> e.getKey()).collect(Collectors.toList()); // collect the least frequent counters in a list
+
+          long removedCount = 0;
+          for (String key : keysToRemove) {
+            removedCount += summary1.remove(key);
+          }
+
+          summary1.merge(TOO_MANY_STAT, removedCount, Long::sum);
+        }
+      }
+    };
+  }
+}


[3/9] accumulo git commit: ACCUMULO-4501 ACCUMULO-96 Added Summarization

Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/util/CancelFlagFuture.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/CancelFlagFuture.java b/core/src/main/java/org/apache/accumulo/core/util/CancelFlagFuture.java
new file mode 100644
index 0000000..9da91c0
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/CancelFlagFuture.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.util;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A simple future wrapper that will set an atomic boolean to true if a future is successfully canceled
+ */
+public class CancelFlagFuture<T> implements Future<T> {
+
+  private Future<T> wrappedFuture;
+  private AtomicBoolean cancelFlag;
+
+  public CancelFlagFuture(Future<T> wrappedFuture, AtomicBoolean cancelFlag) {
+    this.wrappedFuture = wrappedFuture;
+    this.cancelFlag = cancelFlag;
+  }
+
+  @Override
+  public boolean cancel(boolean mayInterruptIfRunning) {
+    boolean ret = wrappedFuture.cancel(mayInterruptIfRunning);
+    if (ret) {
+      cancelFlag.set(true);
+    }
+    return ret;
+  }
+
+  @Override
+  public boolean isCancelled() {
+    return wrappedFuture.isCancelled();
+  }
+
+  @Override
+  public boolean isDone() {
+    return wrappedFuture.isDone();
+  }
+
+  @Override
+  public T get() throws InterruptedException, ExecutionException {
+    return wrappedFuture.get();
+  }
+
+  @Override
+  public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
+    return wrappedFuture.get(timeout, unit);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/util/CompletableFutureUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/CompletableFutureUtil.java b/core/src/main/java/org/apache/accumulo/core/util/CompletableFutureUtil.java
new file mode 100644
index 0000000..c417b0f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/CompletableFutureUtil.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiFunction;
+import java.util.function.Supplier;
+
+public class CompletableFutureUtil {
+
+  // create a binary tree of completable future operations, where each node in the tree merges the results of their children when complete
+  public static <T> CompletableFuture<T> merge(List<CompletableFuture<T>> futures, BiFunction<T,T,T> mergeFunc, Supplier<T> nothing) {
+    if (futures.size() == 0) {
+      return CompletableFuture.completedFuture(nothing.get());
+    }
+    while (futures.size() > 1) {
+      ArrayList<CompletableFuture<T>> mergedFutures = new ArrayList<>(futures.size() / 2);
+      for (int i = 0; i < futures.size(); i += 2) {
+        if (i + 1 == futures.size()) {
+          mergedFutures.add(futures.get(i));
+        } else {
+          mergedFutures.add(futures.get(i).thenCombine(futures.get(i + 1), mergeFunc));
+        }
+      }
+
+      futures = mergedFutures;
+    }
+
+    return futures.get(0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/thrift/data.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/data.thrift b/core/src/main/thrift/data.thrift
index 3e5c56c..648c37d 100644
--- a/core/src/main/thrift/data.thrift
+++ b/core/src/main/thrift/data.thrift
@@ -149,6 +149,40 @@ struct TConditionalSession {
         3:i64 ttl;
 }
 
+struct TSummarizerConfiguration {
+  1:string classname
+  2:map<string, string> options
+  3:string configId
+}
+
+struct TSummary {
+   1:map<string, i64> summary
+   2:TSummarizerConfiguration config
+   3:i64 filesContaining
+   4:i64 filesExceeding
+   5:i64 filesLarge
+}
+
+struct TSummaries {
+   1:bool finished
+   2:i64 sessionId
+   3:i64 totalFiles
+   4:i64 deletedFiles
+   5:list<TSummary> summaries
+}
+
+struct TRowRange {
+  1:binary startRow
+  2:binary endRow
+}
+
+struct TSummaryRequest {
+  1:string tableId
+  2:TRowRange bounds
+  3:list<TSummarizerConfiguration> summarizers
+  4:string summarizerPattern
+}
+
 typedef map<TKeyExtent,list<TConditionalMutation>> CMBatch
 
 typedef map<TKeyExtent,list<TMutation>> UpdateBatch

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/thrift/tabletserver.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/tabletserver.thrift b/core/src/main/thrift/tabletserver.thrift
index 7697a2d..b56449f 100644
--- a/core/src/main/thrift/tabletserver.thrift
+++ b/core/src/main/thrift/tabletserver.thrift
@@ -230,6 +230,11 @@ service TabletClientService extends client.ClientService {
   list<ActiveCompaction> getActiveCompactions(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec)
   oneway void removeLogs(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:list<string> filenames)
   list<string> getActiveLogs(1:trace.TInfo tinfo, 2:security.TCredentials credentials)
+
+  data.TSummaries startGetSummaries(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:data.TSummaryRequest request) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+  data.TSummaries startGetSummariesForPartition(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:data.TSummaryRequest request, 4:i32 modulus, 5:i32 remainder) throws (1:client.ThriftSecurityException sec)
+  data.TSummaries startGetSummariesFromFiles(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:data.TSummaryRequest request, 4:map<string,list<data.TRowRange>> files) throws (1:client.ThriftSecurityException sec)
+  data.TSummaries contiuneGetSummaries(1:trace.TInfo tinfo, 2:i64 sessionId) throws (1:NoSuchScanIDException nssi)
 }
 
 typedef i32 TabletID

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
index 1d699c2..496cde3 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
@@ -27,6 +27,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeMap;
+import java.util.function.Predicate;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -37,8 +38,10 @@ import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.client.admin.DiskUsage;
 import org.apache.accumulo.core.client.admin.Locations;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.client.admin.SummaryRetriever;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
@@ -249,6 +252,29 @@ public class TableOperationsHelperTest {
     public Locations locate(String tableName, Collection<Range> ranges) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
       throw new UnsupportedOperationException();
     }
+
+    @Override
+    public SummaryRetriever summaries(String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void addSummarizers(String tableName, SummarizerConfiguration... summarizerConf) throws TableNotFoundException, AccumuloException,
+        AccumuloSecurityException {
+      throw new UnsupportedOperationException();
+
+    }
+
+    @Override
+    public void removeSummarizers(String tableName, Predicate<SummarizerConfiguration> predicate) throws AccumuloException, TableNotFoundException,
+        AccumuloSecurityException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public List<SummarizerConfiguration> listSummarizers(String tableName) throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
+      throw new UnsupportedOperationException();
+    }
   }
 
   protected TableOperationsHelper getHelper() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
index d85db92..367c833 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
@@ -17,12 +17,18 @@
 package org.apache.accumulo.core.client.mapred;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.Collection;
 
 import org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator;
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.CountingSummarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.summarizers.FamilySummarizer;
+import org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
@@ -43,6 +49,9 @@ public class AccumuloFileOutputFormatTest {
     samplerConfig.addOption("hasher", "murmur3_32");
     samplerConfig.addOption("modulus", "109");
 
+    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(VisibilitySummarizer.class).addOption(CountingSummarizer.MAX_COUNTERS_OPT, 2048).build();
+    SummarizerConfiguration sc2 = SummarizerConfiguration.builder(FamilySummarizer.class).addOption(CountingSummarizer.MAX_COUNTERS_OPT, 256).build();
+
     JobConf job = new JobConf();
     AccumuloFileOutputFormat.setReplication(job, a);
     AccumuloFileOutputFormat.setFileBlockSize(job, b);
@@ -50,6 +59,7 @@ public class AccumuloFileOutputFormatTest {
     AccumuloFileOutputFormat.setIndexBlockSize(job, d);
     AccumuloFileOutputFormat.setCompressionType(job, e);
     AccumuloFileOutputFormat.setSampler(job, samplerConfig);
+    AccumuloFileOutputFormat.setSummarizers(job, sc1, sc2);
 
     AccumuloConfiguration acuconf = FileOutputConfigurator.getAccumuloConfiguration(AccumuloFileOutputFormat.class, job);
 
@@ -60,6 +70,11 @@ public class AccumuloFileOutputFormatTest {
     assertEquals("snappy", acuconf.get(Property.TABLE_FILE_COMPRESSION_TYPE));
     assertEquals(new SamplerConfigurationImpl(samplerConfig), SamplerConfigurationImpl.newSamplerConfig(acuconf));
 
+    Collection<SummarizerConfiguration> summarizerConfigs = SummarizerConfiguration.fromTableProperties(acuconf);
+    assertEquals(2, summarizerConfigs.size());
+    assertTrue(summarizerConfigs.contains(sc1));
+    assertTrue(summarizerConfigs.contains(sc2));
+
     a = 17;
     b = 1300l;
     c = 150l;
@@ -85,5 +100,8 @@ public class AccumuloFileOutputFormatTest {
     assertEquals(110l, acuconf.getMemoryInBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX));
     assertEquals("lzo", acuconf.get(Property.TABLE_FILE_COMPRESSION_TYPE));
     assertEquals(new SamplerConfigurationImpl(samplerConfig), SamplerConfigurationImpl.newSamplerConfig(acuconf));
+
+    summarizerConfigs = SummarizerConfiguration.fromTableProperties(acuconf);
+    assertEquals(0, summarizerConfigs.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
index 39d226b..5143453 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
@@ -17,12 +17,18 @@
 package org.apache.accumulo.core.client.mapreduce;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.Collection;
 
 import org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator;
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.CountingSummarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.summarizers.FamilySummarizer;
+import org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
@@ -43,6 +49,9 @@ public class AccumuloFileOutputFormatTest {
     samplerConfig.addOption("hasher", "murmur3_32");
     samplerConfig.addOption("modulus", "109");
 
+    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(VisibilitySummarizer.class).addOption(CountingSummarizer.MAX_COUNTERS_OPT, 2048).build();
+    SummarizerConfiguration sc2 = SummarizerConfiguration.builder(FamilySummarizer.class).addOption(CountingSummarizer.MAX_COUNTERS_OPT, 256).build();
+
     Job job1 = Job.getInstance();
     AccumuloFileOutputFormat.setReplication(job1, a);
     AccumuloFileOutputFormat.setFileBlockSize(job1, b);
@@ -50,6 +59,7 @@ public class AccumuloFileOutputFormatTest {
     AccumuloFileOutputFormat.setIndexBlockSize(job1, d);
     AccumuloFileOutputFormat.setCompressionType(job1, e);
     AccumuloFileOutputFormat.setSampler(job1, samplerConfig);
+    AccumuloFileOutputFormat.setSummarizers(job1, sc1, sc2);
 
     AccumuloConfiguration acuconf = FileOutputConfigurator.getAccumuloConfiguration(AccumuloFileOutputFormat.class, job1.getConfiguration());
 
@@ -60,6 +70,11 @@ public class AccumuloFileOutputFormatTest {
     assertEquals("snappy", acuconf.get(Property.TABLE_FILE_COMPRESSION_TYPE));
     assertEquals(new SamplerConfigurationImpl(samplerConfig), SamplerConfigurationImpl.newSamplerConfig(acuconf));
 
+    Collection<SummarizerConfiguration> summarizerConfigs = SummarizerConfiguration.fromTableProperties(acuconf);
+    assertEquals(2, summarizerConfigs.size());
+    assertTrue(summarizerConfigs.contains(sc1));
+    assertTrue(summarizerConfigs.contains(sc2));
+
     a = 17;
     b = 1300l;
     c = 150l;
@@ -86,5 +101,8 @@ public class AccumuloFileOutputFormatTest {
     assertEquals("lzo", acuconf.get(Property.TABLE_FILE_COMPRESSION_TYPE));
     assertEquals(new SamplerConfigurationImpl(samplerConfig), SamplerConfigurationImpl.newSamplerConfig(acuconf));
 
+    summarizerConfigs = SummarizerConfiguration.fromTableProperties(acuconf);
+    assertEquals(0, summarizerConfigs.size());
+
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java b/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java
index 4993810..3352b0f 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -36,6 +37,11 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.summary.CounterSummary;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.Summary;
+import org.apache.accumulo.core.client.summary.summarizers.FamilySummarizer;
+import org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.ArrayByteSequence;
@@ -83,6 +89,10 @@ public class RFileTest {
   }
 
   private SortedMap<Key,Value> createTestData(int startRow, int rows, int startFamily, int families, int qualifiers) {
+    return createTestData(startRow, rows, startFamily, families, qualifiers, "");
+  }
+
+  private SortedMap<Key,Value> createTestData(int startRow, int rows, int startFamily, int families, int qualifiers, String... vis) {
     TreeMap<Key,Value> testData = new TreeMap<>();
 
     for (int r = 0; r < rows; r++) {
@@ -91,8 +101,10 @@ public class RFileTest {
         String fam = colStr(f + startFamily);
         for (int q = 0; q < qualifiers; q++) {
           String qual = colStr(q);
-          Key k = new Key(row, fam, qual);
-          testData.put(k, new Value((k.hashCode() + "").getBytes()));
+          for (String v : vis) {
+            Key k = new Key(row, fam, qual, v);
+            testData.put(k, new Value((k.hashCode() + "").getBytes()));
+          }
         }
       }
     }
@@ -498,6 +510,148 @@ public class RFileTest {
     scanner.close();
   }
 
+  @Test
+  public void testSummaries() throws Exception {
+    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(VisibilitySummarizer.class).build();
+    SummarizerConfiguration sc2 = SummarizerConfiguration.builder(FamilySummarizer.class).build();
+
+    LocalFileSystem localFs = FileSystem.getLocal(new Configuration());
+    String testFile = createTmpTestFile();
+
+    SortedMap<Key,Value> testData1 = createTestData(0, 100, 0, 4, 1, "A&B", "A&B&C");
+
+    RFileWriter writer = RFile.newWriter().to(testFile).withFileSystem(localFs).withSummarizers(sc1, sc2).build();
+    writer.append(testData1.entrySet());
+    writer.close();
+
+    // verify summary data
+    Collection<Summary> summaries = RFile.summaries().from(testFile).withFileSystem(localFs).read();
+    Assert.assertEquals(2, summaries.size());
+    for (Summary summary : summaries) {
+      Assert.assertEquals(0, summary.getFileStatistics().getInaccurate());
+      Assert.assertEquals(1, summary.getFileStatistics().getTotal());
+      String className = summary.getSummarizerConfiguration().getClassName();
+      CounterSummary counterSummary = new CounterSummary(summary);
+      if (className.equals(FamilySummarizer.class.getName())) {
+        Map<String,Long> counters = counterSummary.getCounters();
+        Map<String,Long> expected = ImmutableMap.of("0000", 200l, "0001", 200l, "0002", 200l, "0003", 200l);
+        Assert.assertEquals(expected, counters);
+      } else if (className.equals(VisibilitySummarizer.class.getName())) {
+        Map<String,Long> counters = counterSummary.getCounters();
+        Map<String,Long> expected = ImmutableMap.of("A&B", 400l, "A&B&C", 400l);
+        Assert.assertEquals(expected, counters);
+      } else {
+        Assert.fail("Unexpected classname " + className);
+      }
+    }
+
+    // check if writing summary data impacted normal rfile functionality
+    Scanner scanner = RFile.newScanner().from(testFile).withFileSystem(localFs).withAuthorizations(new Authorizations("A", "B", "C")).build();
+    Assert.assertEquals(testData1, toMap(scanner));
+    scanner.close();
+
+    String testFile2 = createTmpTestFile();
+    SortedMap<Key,Value> testData2 = createTestData(100, 100, 0, 4, 1, "A&B", "A&B&C");
+    writer = RFile.newWriter().to(testFile2).withFileSystem(localFs).withSummarizers(sc1, sc2).build();
+    writer.append(testData2.entrySet());
+    writer.close();
+
+    // verify reading summaries from multiple files works
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).read();
+    Assert.assertEquals(2, summaries.size());
+    for (Summary summary : summaries) {
+      Assert.assertEquals(0, summary.getFileStatistics().getInaccurate());
+      Assert.assertEquals(2, summary.getFileStatistics().getTotal());
+      String className = summary.getSummarizerConfiguration().getClassName();
+      CounterSummary counterSummary = new CounterSummary(summary);
+      if (className.equals(FamilySummarizer.class.getName())) {
+        Map<String,Long> counters = counterSummary.getCounters();
+        Map<String,Long> expected = ImmutableMap.of("0000", 400l, "0001", 400l, "0002", 400l, "0003", 400l);
+        Assert.assertEquals(expected, counters);
+      } else if (className.equals(VisibilitySummarizer.class.getName())) {
+        Map<String,Long> counters = counterSummary.getCounters();
+        Map<String,Long> expected = ImmutableMap.of("A&B", 800l, "A&B&C", 800l);
+        Assert.assertEquals(expected, counters);
+      } else {
+        Assert.fail("Unexpected classname " + className);
+      }
+    }
+
+    // verify reading a subset of summaries works
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 800l, "A&B&C", 800l), 0);
+
+    // the following test check boundry conditions for start row and end row
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).startRow(rowStr(99)).read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 400l, "A&B&C", 400l), 0);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).startRow(rowStr(98)).read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 800l, "A&B&C", 800l), 1);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).startRow(rowStr(0)).read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 800l, "A&B&C", 800l), 1);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).startRow("#").read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 800l, "A&B&C", 800l), 0);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).startRow(rowStr(100)).read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 400l, "A&B&C", 400l), 1);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).endRow(rowStr(99)).read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 400l, "A&B&C", 400l), 0);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).endRow(rowStr(100)).read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 800l, "A&B&C", 800l), 1);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).endRow(rowStr(199)).read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 800l, "A&B&C", 800l), 0);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).startRow(rowStr(50))
+        .endRow(rowStr(150)).read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 800l, "A&B&C", 800l), 2);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).startRow(rowStr(120))
+        .endRow(rowStr(150)).read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 400l, "A&B&C", 400l), 1);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).startRow(rowStr(50))
+        .endRow(rowStr(199)).read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 800l, "A&B&C", 800l), 1);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).startRow("#").endRow(rowStr(150))
+        .read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 800l, "A&B&C", 800l), 1);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).startRow(rowStr(199)).read();
+    checkSummaries(summaries, ImmutableMap.of(), 0);
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).startRow(rowStr(200)).read();
+    checkSummaries(summaries, ImmutableMap.of(), 0);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).endRow("#").read();
+    checkSummaries(summaries, ImmutableMap.of(), 0);
+
+    summaries = RFile.summaries().from(testFile, testFile2).withFileSystem(localFs).selectSummaries(sc -> sc.equals(sc1)).endRow(rowStr(0)).read();
+    checkSummaries(summaries, ImmutableMap.of("A&B", 400l, "A&B&C", 400l), 1);
+  }
+
+  private void checkSummaries(Collection<Summary> summaries, Map<String,Long> expected, int extra) {
+    Assert.assertEquals(1, summaries.size());
+    for (Summary summary : summaries) {
+      Assert.assertEquals(extra, summary.getFileStatistics().getInaccurate());
+      Assert.assertEquals(extra, summary.getFileStatistics().getExtra());
+      Assert.assertEquals(2, summary.getFileStatistics().getTotal());
+      String className = summary.getSummarizerConfiguration().getClassName();
+      CounterSummary counterSummary = new CounterSummary(summary);
+      if (className.equals(VisibilitySummarizer.class.getName())) {
+        Map<String,Long> counters = counterSummary.getCounters();
+
+        Assert.assertEquals(expected, counters);
+      } else {
+        Assert.fail("Unexpected classname " + className);
+      }
+    }
+  }
+
   @Test(expected = IllegalArgumentException.class)
   public void testOutOfOrder() throws Exception {
     // test that exception declared in API is thrown

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/test/java/org/apache/accumulo/core/client/summary/CountingSummarizerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/summary/CountingSummarizerTest.java b/core/src/test/java/org/apache/accumulo/core/client/summary/CountingSummarizerTest.java
new file mode 100644
index 0000000..06f8d35
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/summary/CountingSummarizerTest.java
@@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.client.summary;
+
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.COUNTER_STAT_PREFIX;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.DELETES_IGNORED_STAT;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.EMITTED_STAT;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.INGNORE_DELETES_OPT;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.MAX_COUNTERS_OPT;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.MAX_COUNTER_LEN_OPT;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.SEEN_STAT;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.TOO_LONG_STAT;
+import static org.apache.accumulo.core.client.summary.CountingSummarizer.TOO_MANY_STAT;
+
+import java.util.Arrays;
+import java.util.HashMap;
+
+import org.apache.accumulo.core.client.summary.CounterSummary;
+import org.apache.accumulo.core.client.summary.CountingSummarizer;
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.Summarizer.Collector;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.summarizers.FamilySummarizer;
+import org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class CountingSummarizerTest {
+
+  public static class MultiSummarizer extends CountingSummarizer<String> {
+    @Override
+    protected Converter<String> converter() {
+      return (k, v, c) -> {
+        c.accept("rp:" + k.getRowData().subSequence(0, 2).toString());
+        c.accept("fp:" + k.getColumnFamilyData().subSequence(0, 2).toString());
+        c.accept("qp:" + k.getColumnQualifierData().subSequence(0, 2).toString());
+      };
+    }
+  }
+
+  @Test
+  public void testMultipleEmit() {
+    SummarizerConfiguration sc = SummarizerConfiguration.builder(MultiSummarizer.class).build();
+    MultiSummarizer countSum = new MultiSummarizer();
+
+    Summarizer.Collector collector = countSum.collector(sc);
+
+    Value val = new Value("abc");
+
+    HashMap<String,Long> expected = new HashMap<>();
+
+    for (String row : new String[] {"ask", "asleep", "some", "soul"}) {
+      for (String fam : new String[] {"hop", "hope", "nope", "noop"}) {
+        for (String qual : new String[] {"mad", "lad", "lab", "map"}) {
+          collector.accept(new Key(row, fam, qual), val);
+
+          expected.merge("rp:" + row.substring(0, 2), 1l, Long::sum);
+          expected.merge("fp:" + fam.substring(0, 2), 1l, Long::sum);
+          expected.merge("qp:" + qual.substring(0, 2), 1l, Long::sum);
+        }
+      }
+    }
+
+    HashMap<String,Long> stats = new HashMap<>();
+    collector.summarize((k, v) -> stats.put(k, v));
+
+    CounterSummary csum = new CounterSummary(stats);
+    Assert.assertEquals(expected, csum.getCounters());
+    Assert.assertEquals(64, csum.getSeen());
+    Assert.assertEquals(3 * 64, csum.getEmitted());
+    Assert.assertEquals(0, csum.getIgnored());
+    Assert.assertEquals(0, csum.getDeletesIgnored());
+  }
+
+  @Test
+  public void testSummarizing() {
+    SummarizerConfiguration sc = SummarizerConfiguration.builder(FamilySummarizer.class).addOptions(MAX_COUNTERS_OPT, "5", MAX_COUNTER_LEN_OPT, "10").build();
+    FamilySummarizer countSum = new FamilySummarizer();
+
+    Value val = new Value("abc");
+
+    Summarizer.Collector collector = countSum.collector(sc);
+    for (String fam : Arrays.asList("f1", "f1", "f1", "f2", "f1", "f70000000000000000000", "f70000000000000000001", "f2", "f3", "f4", "f5", "f6", "f7", "f3",
+        "f7")) {
+      collector.accept(new Key("r", fam), val);
+    }
+
+    Key dk = new Key("r", "f2");
+    dk.setDeleted(true);
+    collector.accept(dk, new Value(""));
+
+    HashMap<String,Long> stats = new HashMap<>();
+    collector.summarize((k, v) -> stats.put(k, v));
+
+    String p = COUNTER_STAT_PREFIX;
+
+    HashMap<String,Long> expected = new HashMap<>();
+    expected.put(p + "f1", 4l);
+    expected.put(p + "f2", 2l);
+    expected.put(p + "f3", 2l);
+    expected.put(p + "f4", 1l);
+    expected.put(p + "f5", 1l);
+    expected.put(TOO_LONG_STAT, 2l);
+    expected.put(TOO_MANY_STAT, 3l);
+    expected.put(SEEN_STAT, 16l);
+    expected.put(EMITTED_STAT, 15l);
+    expected.put(DELETES_IGNORED_STAT, 1l);
+
+    Assert.assertEquals(expected, stats);
+
+    CounterSummary csum = new CounterSummary(stats);
+    Assert.assertEquals(5, csum.getIgnored());
+    Assert.assertEquals(3, csum.getTooMany());
+    Assert.assertEquals(2, csum.getTooLong());
+    Assert.assertEquals(16, csum.getSeen());
+    Assert.assertEquals(15, csum.getEmitted());
+    Assert.assertEquals(1, csum.getDeletesIgnored());
+
+    expected.clear();
+    expected.put("f1", 4l);
+    expected.put("f2", 2l);
+    expected.put("f3", 2l);
+    expected.put("f4", 1l);
+    expected.put("f5", 1l);
+    Assert.assertEquals(expected, csum.getCounters());
+
+  }
+
+  @Test
+  public void testMerge() {
+    SummarizerConfiguration sc = SummarizerConfiguration.builder(VisibilitySummarizer.class).addOption(MAX_COUNTERS_OPT, "5").build();
+    VisibilitySummarizer countSum = new VisibilitySummarizer();
+
+    String p = COUNTER_STAT_PREFIX;
+
+    HashMap<String,Long> sm1 = new HashMap<>();
+    sm1.put(p + "f001", 9l);
+    sm1.put(p + "f002", 4l);
+    sm1.put(p + "f003", 2l);
+    sm1.put(p + "f004", 1l);
+    sm1.put(p + "f005", 19l);
+    sm1.put(EMITTED_STAT, 15l);
+    sm1.put(SEEN_STAT, 5l);
+    sm1.put(DELETES_IGNORED_STAT, 1l);
+
+    HashMap<String,Long> sm2 = new HashMap<>();
+    sm2.put(p + "f001", 1l);
+    sm2.put(p + "f002", 2l);
+    sm2.put(p + "f00a", 7l);
+    sm2.put(p + "f00b", 1l);
+    sm2.put(p + "f00c", 17l);
+    sm2.put(EMITTED_STAT, 18l);
+    sm2.put(SEEN_STAT, 6l);
+    sm2.put(DELETES_IGNORED_STAT, 2l);
+
+    countSum.combiner(sc).merge(sm1, sm2);
+
+    HashMap<String,Long> expected = new HashMap<>();
+    expected.put(p + "f001", 10l);
+    expected.put(p + "f002", 6l);
+    expected.put(p + "f005", 19l);
+    expected.put(p + "f00a", 7l);
+    expected.put(p + "f00c", 17l);
+    expected.put(TOO_LONG_STAT, 0l);
+    expected.put(TOO_MANY_STAT, 4l);
+    expected.put(EMITTED_STAT, 18l + 15l);
+    expected.put(SEEN_STAT, 6l + 5l);
+    expected.put(DELETES_IGNORED_STAT, 3l);
+
+    Assert.assertEquals(expected, sm1);
+
+    sm2.clear();
+    sm2.put(p + "f001", 19l);
+    sm2.put(p + "f002", 2l);
+    sm2.put(p + "f003", 3l);
+    sm2.put(p + "f00b", 13l);
+    sm2.put(p + "f00c", 2l);
+    sm2.put(TOO_LONG_STAT, 1l);
+    sm2.put(TOO_MANY_STAT, 3l);
+    sm2.put(EMITTED_STAT, 21l);
+    sm2.put(SEEN_STAT, 7l);
+    sm2.put(DELETES_IGNORED_STAT, 5l);
+
+    countSum.combiner(sc).merge(sm1, sm2);
+
+    expected.clear();
+    expected.put(p + "f001", 29l);
+    expected.put(p + "f002", 8l);
+    expected.put(p + "f005", 19l);
+    expected.put(p + "f00b", 13l);
+    expected.put(p + "f00c", 19l);
+    expected.put(TOO_LONG_STAT, 1l);
+    expected.put(TOO_MANY_STAT, 17l);
+    expected.put(EMITTED_STAT, 21l + 18 + 15);
+    expected.put(SEEN_STAT, 7l + 6 + 5);
+    expected.put(DELETES_IGNORED_STAT, 8l);
+  }
+
+  @Test
+  public void testCountDeletes() {
+    SummarizerConfiguration sc = SummarizerConfiguration.builder(FamilySummarizer.class).addOptions(INGNORE_DELETES_OPT, "false").build();
+    FamilySummarizer countSum = new FamilySummarizer();
+
+    Key k1 = new Key("r1", "f1");
+    Key k2 = new Key("r1", "f1");
+    k2.setDeleted(true);
+    Key k3 = new Key("r1", "f2");
+
+    Collector collector = countSum.collector(sc);
+    collector.accept(k1, new Value(""));
+    collector.accept(k2, new Value(""));
+    collector.accept(k3, new Value(""));
+
+    String p = COUNTER_STAT_PREFIX;
+
+    HashMap<String,Long> expected = new HashMap<>();
+    expected.put(p + "f1", 2l);
+    expected.put(p + "f2", 1l);
+    expected.put(TOO_LONG_STAT, 0l);
+    expected.put(TOO_MANY_STAT, 0l);
+    expected.put(SEEN_STAT, 3l);
+    expected.put(EMITTED_STAT, 3l);
+    expected.put(DELETES_IGNORED_STAT, 0l);
+
+    HashMap<String,Long> stats = new HashMap<>();
+    collector.summarize(stats::put);
+    Assert.assertEquals(expected, stats);
+
+    CounterSummary csum = new CounterSummary(stats);
+    Assert.assertEquals(0, csum.getIgnored());
+    Assert.assertEquals(0, csum.getTooMany());
+    Assert.assertEquals(0, csum.getTooLong());
+    Assert.assertEquals(3, csum.getSeen());
+    Assert.assertEquals(3, csum.getEmitted());
+    Assert.assertEquals(0, csum.getDeletesIgnored());
+
+    expected.clear();
+    expected.put("f1", 2l);
+    expected.put("f2", 1l);
+    Assert.assertEquals(expected, csum.getCounters());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/test/java/org/apache/accumulo/core/summary/SummaryCollectionTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/summary/SummaryCollectionTest.java b/core/src/test/java/org/apache/accumulo/core/summary/SummaryCollectionTest.java
new file mode 100644
index 0000000..95702d3
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/summary/SummaryCollectionTest.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.summary;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.Summary;
+import org.apache.accumulo.core.client.summary.Summary.FileStatistics;
+import org.apache.accumulo.core.client.summary.summarizers.FamilySummarizer;
+import org.apache.accumulo.core.summary.SummaryCollection.FileSummary;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class SummaryCollectionTest {
+  @Test
+  public void testDeleted() {
+    SummarizerConfiguration conf = SummarizerConfiguration.builder(FamilySummarizer.class).build();
+
+    HashMap<String,Long> stats = new HashMap<String,Long>();
+    stats.put("c:foo", 9L);
+    FileSummary fs1 = new FileSummary(conf, stats, false);
+    SummaryCollection sc1 = new SummaryCollection(Collections.singleton(fs1));
+
+    stats = new HashMap<String,Long>();
+    stats.put("c:foo", 5L);
+    stats.put("c:bar", 3L);
+    FileSummary fs2 = new FileSummary(conf, stats, true);
+    SummaryCollection sc2 = new SummaryCollection(Collections.singleton(fs2));
+
+    SummaryCollection sc3 = new SummaryCollection(Collections.emptyList());
+
+    SummaryCollection sc4 = new SummaryCollection(Collections.emptyList(), true);
+
+    SummarizerFactory factory = new SummarizerFactory();
+    SummaryCollection mergeSc = new SummaryCollection();
+    for (SummaryCollection sc : Arrays.asList(sc1, sc2, sc3, sc4, sc4)) {
+      mergeSc.merge(sc, factory);
+    }
+
+    for (SummaryCollection sc : Arrays.asList(mergeSc, new SummaryCollection(mergeSc.toThrift()))) {
+      List<Summary> summaries = sc.getSummaries();
+      Assert.assertEquals(1, summaries.size());
+      Summary summary = summaries.get(0);
+      FileStatistics filestats = summary.getFileStatistics();
+      Assert.assertEquals(5, filestats.getTotal());
+      Assert.assertEquals(1, filestats.getExtra());
+      Assert.assertEquals(0, filestats.getLarge());
+      Assert.assertEquals(1, filestats.getMissing());
+      Assert.assertEquals(2, filestats.getDeleted());
+      Assert.assertEquals(4, filestats.getInaccurate());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/test/java/org/apache/accumulo/core/util/CompletableFutureUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/util/CompletableFutureUtilTest.java b/core/src/test/java/org/apache/accumulo/core/util/CompletableFutureUtilTest.java
new file mode 100644
index 0000000..9de580d
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/util/CompletableFutureUtilTest.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.util;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class CompletableFutureUtilTest {
+  @Test
+  public void testMerge() throws Exception {
+    ExecutorService es = Executors.newFixedThreadPool(3);
+    try {
+      for (int n : new int[] {1, 2, 3, 997, 1000}) {
+        List<CompletableFuture<Integer>> futures = new ArrayList<>();
+        for (int i = 1; i <= n; i++) {
+          final int num = i;
+          futures.add(CompletableFuture.supplyAsync(() -> num, es));
+        }
+
+        CompletableFuture<Integer> mergedFutures = CompletableFutureUtil.merge(futures, Integer::sum, () -> 0);
+        Assert.assertEquals(n * (n + 1) / 2, mergedFutures.get().intValue());
+      }
+
+      // test zero
+      CompletableFuture<Integer> mergedFutures = CompletableFutureUtil.merge(Collections.emptyList(), Integer::sum, () -> 0);
+      Assert.assertEquals(0, mergedFutures.get().intValue());
+    } finally {
+      es.shutdown();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/docs/src/main/asciidoc/accumulo_user_manual.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/main/asciidoc/accumulo_user_manual.asciidoc b/docs/src/main/asciidoc/accumulo_user_manual.asciidoc
index fe5403c..979d943 100644
--- a/docs/src/main/asciidoc/accumulo_user_manual.asciidoc
+++ b/docs/src/main/asciidoc/accumulo_user_manual.asciidoc
@@ -63,6 +63,8 @@ include::chapters/kerberos.txt[]
 
 include::chapters/sampling.txt[]
 
+include::chapters/summaries.txt[]
+
 include::chapters/administration.txt[]
 
 include::chapters/multivolume.txt[]

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/docs/src/main/asciidoc/chapters/summaries.txt
----------------------------------------------------------------------
diff --git a/docs/src/main/asciidoc/chapters/summaries.txt b/docs/src/main/asciidoc/chapters/summaries.txt
new file mode 100644
index 0000000..08d8011
--- /dev/null
+++ b/docs/src/main/asciidoc/chapters/summaries.txt
@@ -0,0 +1,232 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+== Summary Statistics
+
+=== Overview
+
+Accumulo has the ability to generate summary statistics about data in a table
+using user defined functions.  Currently these statistics are only generated for
+data written to files.  Data recently written to Accumulo that is still in
+memory will not contribute to summary statistics.
+
+This feature can be used to inform a user about what data is in their table.
+Summary statistics can also be used by compaction strategies to make decisions
+about which files to compact.  
+
+Summary data is stored in each file Accumulo produces.  Accumulo can gather
+summary information from across a cluster merging it along the way.  In order
+for this to be fast the, summary information should fit in cache.  There is a
+dedicated cache for summary data on each tserver with a configurable size.  In
+order for summary data to fit in cache, it should probably be small.
+
+For information on writing a custom summarizer see the javadoc for
++org.apache.accumulo.core.client.summary.Summarizer+.  The package
++org.apache.accumulo.core.client.summary.summarizers+ contains summarizer
+implementations that ship with Accumulo and can be configured for use.
+
+=== Inaccuracies
+
+Summary data can be inaccurate when files are missing summary data or when
+files have extra summary data. Files can contain data outside of a tablets
+boundaries. This can happen as result of bulk imported files and tablet splits.
+When this happens, those files could contain extra summary information.
+Accumulo offsets this some by storing summary information for multiple row
+ranges per a file.  However, the ranges are not granular enough to completely
+offset extra data.
+
+Any source of inaccuracies is reported when summary information is requested.
+In the shell examples below this can be seen on the +File Statistics+ line.
+For files missing summary information, the compact command in the shell has a
++--sf-no-summary+ option.  This options compacts files that do not have the
+summary information configured for the table.  The compact command also has the
++--sf-extra-summary+ option which will compact files with extra summary
+information.
+
+=== Configuring
+
+The following tablet server and table properties configure summarization.
+
+* <<appendices/config.txt#_tserver_cache_summary_size>>
+* <<appendices/config.txt#_tserver_summary_partition_threads>>
+* <<appendices/config.txt#_tserver_summary_remote_threads>>
+* <<appendices/config.txt#_tserver_summary_retrieval_threads>>
+* <<appendices/config.txt#TABLE_SUMMARIZER_PREFIX>>
+* <<appendices/config.txt#_table_file_summary_maxsize>>
+
+=== Permissions
+
+Because summary data may be derived from sensitive data, requesting summary data
+requires a special permission.  User must have the table permission
++GET_SUMMARIES+ in order to retrieve summary data.
+
+
+=== Bulk import
+
+When generating rfiles to bulk import into Accumulo, those rfiles can contain
+summary data.  To use this feature, look at the javadoc on the
++AccumuloFileOutputFormat.setSummarizers(...)+ method.  Also,
++org.apache.accumulo.core.client.rfile.RFile+ has options for creating RFiles
+with embedded summary data.
+
+=== Examples
+
+This example walks through using summarizers in the Accumulo shell.  Below a
+table is created and some data is inserted to summarize.
+
+ root@uno> createtable summary_test
+ root@uno summary_test> setauths -u root -s PI,GEO,TIME
+ root@uno summary_test> insert 3b503bd name last Doe
+ root@uno summary_test> insert 3b503bd name first John
+ root@uno summary_test> insert 3b503bd contact address "123 Park Ave, NY, NY" -l PI&GEO
+ root@uno summary_test> insert 3b503bd date birth "1/11/1942" -l PI&TIME
+ root@uno summary_test> insert 3b503bd date married "5/11/1962" -l PI&TIME
+ root@uno summary_test> insert 3b503bd contact home_phone 1-123-456-7890 -l PI
+ root@uno summary_test> insert d5d18dd contact address "50 Lake Shore Dr, Chicago, IL" -l PI&GEO
+ root@uno summary_test> insert d5d18dd name first Jane
+ root@uno summary_test> insert d5d18dd name last Doe
+ root@uno summary_test> insert d5d18dd date birth 8/15/1969 -l PI&TIME
+ root@uno summary_test> scan -s PI,GEO,TIME
+ 3b503bd contact:address [PI&GEO]    123 Park Ave, NY, NY
+ 3b503bd contact:home_phone [PI]    1-123-456-7890
+ 3b503bd date:birth [PI&TIME]    1/11/1942
+ 3b503bd date:married [PI&TIME]    5/11/1962
+ 3b503bd name:first []    John
+ 3b503bd name:last []    Doe
+ d5d18dd contact:address [PI&GEO]    50 Lake Shore Dr, Chicago, IL
+ d5d18dd date:birth [PI&TIME]    8/15/1969
+ d5d18dd name:first []    Jane
+ d5d18dd name:last []    Doe
+
+After inserting the data, summaries are requested below.  No summaries are returned.
+
+ root@uno summary_test> summaries
+
+The visibility summarizer is configured below and the table is flushed.
+Flushing the table creates a file creating summary data in the process. The
+summary data returned counts how many times each column visibility occurred.
+The statistics with a +c:+ prefix are visibilities.  The others are generic
+statistics created by the CountingSummarizer that VisibilitySummarizer extends. 
+
+ root@uno summary_test> config -t summary_test -s table.summarizer.vis=org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer
+ root@uno summary_test> summaries
+ root@uno summary_test> flush -w
+ 2017-02-24 19:54:46,090 [shell.Shell] INFO : Flush of table summary_test completed.
+ root@uno summary_test> summaries
+  Summarizer         : org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer vis {}
+  File Statistics    : [total:1, missing:0, extra:0, large:0]
+  Summary Statistics : 
+     c:                                                           = 4
+     c:PI                                                         = 1
+     c:PI&GEO                                                     = 2
+     c:PI&TIME                                                    = 3
+     emitted                                                      = 10
+     seen                                                         = 10
+     tooLong                                                      = 0
+     tooMany                                                      = 0
+
+VisibilitySummarizer has an option +maxCounters+ that determines the max number
+of column visibilites it will track.  Below this option is set and compaction
+is forced to regenerate summary data.  The new summary data only has three
+visibilites and now the +tooMany+ statistic is 4.  This is the number of
+visibilites that were not counted.
+
+ root@uno summary_test> config -t summary_test -s table.summarizer.vis.opt.maxCounters=3
+ root@uno summary_test> compact -w
+ 2017-02-24 19:54:46,267 [shell.Shell] INFO : Compacting table ...
+ 2017-02-24 19:54:47,127 [shell.Shell] INFO : Compaction of table summary_test completed for given range
+ root@uno summary_test> summaries
+  Summarizer         : org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer vis {maxCounters=3}
+  File Statistics    : [total:1, missing:0, extra:0, large:0]
+  Summary Statistics : 
+     c:PI                                                         = 1
+     c:PI&GEO                                                     = 2
+     c:PI&TIME                                                    = 3
+     emitted                                                      = 10
+     seen                                                         = 10
+     tooLong                                                      = 0
+     tooMany                                                      = 4
+
+Another summarizer is configured below that tracks the number of deletes.  Also
+a compaction strategy that uses this summary data is configured.  The
++TooManyDeletesCompactionStrategy+ will force a compaction of the tablet when
+the ratio of deletes to non-deletes is over 25%.  This threshold is
+configurable.  Below a delete is added and its reflected in the statistics.  In
+this case there is 1 delete and 10 non-deletes, not enough to force a
+compaction of the tablet.
+
+....
+root@uno summary_test> config -t summary_test -s table.summarizer.del=org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer
+root@uno summary_test> compact -w
+2017-02-24 19:54:47,282 [shell.Shell] INFO : Compacting table ...
+2017-02-24 19:54:49,236 [shell.Shell] INFO : Compaction of table summary_test completed for given range
+root@uno summary_test> config -t summary_test -s table.compaction.major.ratio=10
+root@uno summary_test> config -t summary_test -s table.majc.compaction.strategy=org.apache.accumulo.tserver.compaction.strategies.TooManyDeletesCompactionStrategy
+root@uno summary_test> deletemany -r d5d18dd -c date -f
+[DELETED] d5d18dd date:birth [PI&TIME]
+root@uno summary_test> flush -w
+2017-02-24 19:54:49,686 [shell.Shell] INFO : Flush of table summary_test completed.
+root@uno summary_test> summaries
+ Summarizer         : org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer vis {maxCounters=3}
+ File Statistics    : [total:2, missing:0, extra:0, large:0]
+ Summary Statistics : 
+    c:PI                                                         = 1
+    c:PI&GEO                                                     = 2
+    c:PI&TIME                                                    = 4
+    emitted                                                      = 11
+    seen                                                         = 11
+    tooLong                                                      = 0
+    tooMany                                                      = 4
+
+ Summarizer         : org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer del {}
+ File Statistics    : [total:2, missing:0, extra:0, large:0]
+ Summary Statistics : 
+    deletes                                                      = 1
+    total                                                        = 11
+....
+
+Some more deletes are added and the table is flushed below.  This results in 4
+deletes and 10 non-deletes, which triggers a full compaction.  A full
+compaction of all files is the only time when delete markers are dropped.  The
+compaction ratio was set to 10 above to show that the number of files did not
+trigger the compaction.   After the compaction there no deletes 6 non-deletes.
+
+....
+root@uno summary_test> deletemany -r d5d18dd -f
+[DELETED] d5d18dd contact:address [PI&GEO]
+[DELETED] d5d18dd name:first []
+[DELETED] d5d18dd name:last []
+root@uno summary_test> flush -w
+2017-02-24 19:54:52,800 [shell.Shell] INFO : Flush of table summary_test completed.
+root@uno summary_test> summaries
+ Summarizer         : org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer vis {maxCounters=3}
+ File Statistics    : [total:1, missing:0, extra:0, large:0]
+ Summary Statistics : 
+    c:PI                                                         = 1
+    c:PI&GEO                                                     = 1
+    c:PI&TIME                                                    = 2
+    emitted                                                      = 6
+    seen                                                         = 6
+    tooLong                                                      = 0
+    tooMany                                                      = 2
+
+ Summarizer         : org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer del {}
+ File Statistics    : [total:1, missing:0, extra:0, large:0]
+ Summary Statistics : 
+    deletes                                                      = 0
+    total                                                        = 6
+root@uno summary_test>   
+....
+

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
index 3b19aeb..1c670bc 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
@@ -142,8 +142,7 @@ public class StandaloneAccumuloCluster implements AccumuloCluster {
 
   @Override
   public StandaloneClusterControl getClusterControl() {
-    return new StandaloneClusterControl(accumuloHome, clientAccumuloConfDir, serverAccumuloConfDir,
-                                        clientCmdPrefix, serverCmdPrefix);
+    return new StandaloneClusterControl(accumuloHome, clientAccumuloConfDir, serverAccumuloConfDir, clientCmdPrefix, serverCmdPrefix);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
index bf1ccc7..45028e9 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
@@ -63,7 +63,8 @@ public class StandaloneClusterControl implements ClusterControl {
 
   protected String accumuloServicePath, accumuloPath, accumuloUtilPath;
 
-  public StandaloneClusterControl(String accumuloHome, String clientAccumuloConfDir, String serverAccumuloConfDir, String clientCmdPrefix, String serverCmdPrefix) {
+  public StandaloneClusterControl(String accumuloHome, String clientAccumuloConfDir, String serverAccumuloConfDir, String clientCmdPrefix,
+      String serverCmdPrefix) {
     this.options = new RemoteShellOptions();
     this.accumuloHome = accumuloHome;
     this.clientAccumuloConfDir = clientAccumuloConfDir;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
index 20763bb..2693c05 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
@@ -139,7 +139,8 @@ public class MiniAccumuloConfigImpl {
       mergeProp(Property.TSERV_PORTSEARCH.getKey(), "true");
       mergeProp(Property.TSERV_DATACACHE_SIZE.getKey(), "10M");
       mergeProp(Property.TSERV_INDEXCACHE_SIZE.getKey(), "10M");
-      mergeProp(Property.TSERV_MAXMEM.getKey(), "50M");
+      mergeProp(Property.TSERV_SUMMARYCACHE_SIZE.getKey(), "10M");
+      mergeProp(Property.TSERV_MAXMEM.getKey(), "40M");
       mergeProp(Property.TSERV_WALOG_MAX_SIZE.getKey(), "100M");
       mergeProp(Property.TSERV_NATIVEMAP_ENABLED.getKey(), "false");
       // since there is a small amount of memory, check more frequently for majc... setting may not be needed in 1.5

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index c4edc96..fd7746a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -845,4 +845,9 @@ public class SecurityOperation {
     authenticate(credentials);
     return hasSystemPermission(credentials, SystemPermission.OBTAIN_DELEGATION_TOKEN, false);
   }
+
+  public boolean canGetSummaries(TCredentials credentials, String tableId, String namespaceId) throws ThriftSecurityException {
+    authenticate(credentials);
+    return hasTablePermission(credentials, tableId, namespaceId, TablePermission.GET_SUMMARIES, false);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 47a0d18..7187e3d 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -50,6 +50,8 @@ import java.util.concurrent.BlockingDeque;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -76,7 +78,9 @@ import org.apache.accumulo.core.client.impl.Translator.TKeyExtentTranslator;
 import org.apache.accumulo.core.client.impl.Translator.TRangeTranslator;
 import org.apache.accumulo.core.client.impl.Translators;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
@@ -102,7 +106,11 @@ import org.apache.accumulo.core.data.thrift.TKeyExtent;
 import org.apache.accumulo.core.data.thrift.TKeyValue;
 import org.apache.accumulo.core.data.thrift.TMutation;
 import org.apache.accumulo.core.data.thrift.TRange;
+import org.apache.accumulo.core.data.thrift.TRowRange;
+import org.apache.accumulo.core.data.thrift.TSummaries;
+import org.apache.accumulo.core.data.thrift.TSummaryRequest;
 import org.apache.accumulo.core.data.thrift.UpdateErrors;
+import org.apache.accumulo.core.file.blockfile.cache.BlockCache;
 import org.apache.accumulo.core.iterators.IterationInterruptedException;
 import org.apache.accumulo.core.master.thrift.BulkImportState;
 import org.apache.accumulo.core.master.thrift.Compacting;
@@ -119,6 +127,9 @@ import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.summary.Gatherer;
+import org.apache.accumulo.core.summary.Gatherer.FileSystemResolver;
+import org.apache.accumulo.core.summary.SummaryCollection;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
@@ -235,6 +246,7 @@ import org.apache.accumulo.tserver.session.MultiScanSession;
 import org.apache.accumulo.tserver.session.ScanSession;
 import org.apache.accumulo.tserver.session.Session;
 import org.apache.accumulo.tserver.session.SessionManager;
+import org.apache.accumulo.tserver.session.SummarySession;
 import org.apache.accumulo.tserver.session.UpdateSession;
 import org.apache.accumulo.tserver.tablet.BulkImportCacheCleaner;
 import org.apache.accumulo.tserver.tablet.CommitSession;
@@ -1790,6 +1802,109 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
       log.warn("Garbage collector is attempting to remove logs through the tablet server");
       log.warn("This is probably because your file Garbage Collector is an older version than your tablet servers.\n" + "Restart your file Garbage Collector.");
     }
+
+    private TSummaries getSummaries(Future<SummaryCollection> future) throws TimeoutException {
+      try {
+        SummaryCollection sc = future.get(MAX_TIME_TO_WAIT_FOR_SCAN_RESULT_MILLIS, TimeUnit.MILLISECONDS);
+        return sc.toThrift();
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new RuntimeException(e);
+      } catch (ExecutionException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    private TSummaries handleTimeout(long sessionId) {
+      long timeout = TabletServer.this.getConfiguration().getTimeInMillis(Property.TSERV_CLIENT_TIMEOUT);
+      sessionManager.removeIfNotAccessed(sessionId, timeout);
+      return new TSummaries(false, sessionId, -1, -1, null);
+    }
+
+    private TSummaries startSummaryOperation(TCredentials credentials, Future<SummaryCollection> future) {
+      try {
+        return getSummaries(future);
+      } catch (TimeoutException e) {
+        long sid = sessionManager.createSession(new SummarySession(credentials, future), false);
+        while (sid == 0) {
+          sessionManager.removeSession(sid);
+          sid = sessionManager.createSession(new SummarySession(credentials, future), false);
+        }
+        return handleTimeout(sid);
+      }
+    }
+
+    @Override
+    public TSummaries startGetSummaries(TInfo tinfo, TCredentials credentials, TSummaryRequest request) throws ThriftSecurityException,
+        ThriftTableOperationException, NoSuchScanIDException, TException {
+      String namespaceId;
+      try {
+        namespaceId = Tables.getNamespaceId(TabletServer.this.getInstance(), request.getTableId());
+      } catch (TableNotFoundException e1) {
+        throw new ThriftTableOperationException(request.getTableId(), null, null, TableOperationExceptionType.NOTFOUND, null);
+      }
+
+      if (!security.canGetSummaries(credentials, request.getTableId(), namespaceId)) {
+        throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+      }
+
+      ServerConfigurationFactory factory = TabletServer.this.getServerConfigurationFactory();
+      ExecutorService es = resourceManager.getSummaryPartitionExecutor();
+      Future<SummaryCollection> future = new Gatherer(TabletServer.this, request, factory.getTableConfiguration(request.getTableId())).gather(es);
+
+      return startSummaryOperation(credentials, future);
+    }
+
+    @Override
+    public TSummaries startGetSummariesForPartition(TInfo tinfo, TCredentials credentials, TSummaryRequest request, int modulus, int remainder)
+        throws ThriftSecurityException, NoSuchScanIDException, TException {
+      // do not expect users to call this directly, expect other tservers to call this method
+      if (!security.canPerformSystemActions(credentials)) {
+        throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+      }
+
+      ServerConfigurationFactory factory = TabletServer.this.getServerConfigurationFactory();
+      ExecutorService spe = resourceManager.getSummaryRemoteExecutor();
+      Future<SummaryCollection> future = new Gatherer(TabletServer.this, request, factory.getTableConfiguration(request.getTableId())).processPartition(spe,
+          modulus, remainder);
+
+      return startSummaryOperation(credentials, future);
+    }
+
+    @Override
+    public TSummaries startGetSummariesFromFiles(TInfo tinfo, TCredentials credentials, TSummaryRequest request, Map<String,List<TRowRange>> files)
+        throws ThriftSecurityException, NoSuchScanIDException, TException {
+      // do not expect users to call this directly, expect other tservers to call this method
+      if (!security.canPerformSystemActions(credentials)) {
+        throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+      }
+
+      ExecutorService srp = resourceManager.getSummaryRetrievalExecutor();
+      TableConfiguration tableCfg = confFactory.getTableConfiguration(request.getTableId());
+      BlockCache summaryCache = resourceManager.getSummaryCache();
+      BlockCache indexCache = resourceManager.getIndexCache();
+      FileSystemResolver volMgr = p -> fs.getVolumeByPath(p).getFileSystem();
+      Future<SummaryCollection> future = new Gatherer(TabletServer.this, request, tableCfg).processFiles(volMgr, files, summaryCache, indexCache, srp);
+
+      return startSummaryOperation(credentials, future);
+    }
+
+    @Override
+    public TSummaries contiuneGetSummaries(TInfo tinfo, long sessionId) throws NoSuchScanIDException, TException {
+      SummarySession session = (SummarySession) sessionManager.getSession(sessionId);
+      if (session == null) {
+        throw new NoSuchScanIDException();
+      }
+
+      Future<SummaryCollection> future = session.getFuture();
+      try {
+        TSummaries tsums = getSummaries(future);
+        sessionManager.removeSession(sessionId);
+        return tsums;
+      } catch (TimeoutException e) {
+        return handleTimeout(sessionId);
+      }
+    }
   }
 
   private class SplitRunner implements Runnable {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
index 3cd7bfa..411ffa5 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.tserver;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.Objects.requireNonNull;
 
 import java.io.IOException;
@@ -65,7 +66,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
-import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 /**
  * ResourceManager is responsible for managing the resources of all tablets within a tablet server.
@@ -86,12 +86,13 @@ public class TabletServerResourceManager {
   private final ExecutorService assignMetaDataPool;
   private final ExecutorService readAheadThreadPool;
   private final ExecutorService defaultReadAheadThreadPool;
+  private final ExecutorService summaryRetrievalPool;
+  private final ExecutorService summaryParitionPool;
+  private final ExecutorService summaryRemotePool;
   private final Map<String,ExecutorService> threadPools = new TreeMap<>();
 
   private final ConcurrentHashMap<KeyExtent,RunnableStartedAt> activeAssignments;
 
-  private final VolumeManager fs;
-
   private final FileManager fileManager;
 
   private final MemoryManager memoryManager;
@@ -100,6 +101,7 @@ public class TabletServerResourceManager {
 
   private final BlockCache _dCache;
   private final BlockCache _iCache;
+  private final BlockCache _sCache;
   private final TabletServer tserver;
   private final ServerConfigurationFactory conf;
 
@@ -141,6 +143,14 @@ public class TabletServerResourceManager {
     return createEs(max, name, new LinkedBlockingQueue<Runnable>());
   }
 
+  private ExecutorService createIdlingEs(Property max, String name, long timeout, TimeUnit timeUnit) {
+    LinkedBlockingQueue<Runnable> queue = new LinkedBlockingQueue<Runnable>();
+    int maxThreads = conf.getConfiguration().getCount(max);
+    ThreadPoolExecutor tp = new ThreadPoolExecutor(maxThreads, maxThreads, timeout, timeUnit, queue, new NamingThreadFactory(name));
+    tp.allowCoreThreadTimeOut(true);
+    return addEs(max, name, tp);
+  }
+
   private ExecutorService createEs(Property max, String name, BlockingQueue<Runnable> queue) {
     int maxThreads = conf.getConfiguration().getCount(max);
     ThreadPoolExecutor tp = new ThreadPoolExecutor(maxThreads, maxThreads, 0L, TimeUnit.MILLISECONDS, queue, new NamingThreadFactory(name));
@@ -154,7 +164,6 @@ public class TabletServerResourceManager {
   public TabletServerResourceManager(TabletServer tserver, VolumeManager fs) {
     this.tserver = tserver;
     this.conf = tserver.getServerConfigurationFactory();
-    this.fs = fs;
     final AccumuloConfiguration acuConf = conf.getConfiguration();
 
     long maxMemory = acuConf.getMemoryInBytes(Property.TSERV_MAXMEM);
@@ -163,15 +172,18 @@ public class TabletServerResourceManager {
     long blockSize = acuConf.getMemoryInBytes(Property.TSERV_DEFAULT_BLOCKSIZE);
     long dCacheSize = acuConf.getMemoryInBytes(Property.TSERV_DATACACHE_SIZE);
     long iCacheSize = acuConf.getMemoryInBytes(Property.TSERV_INDEXCACHE_SIZE);
+    long sCacheSize = acuConf.getMemoryInBytes(Property.TSERV_SUMMARYCACHE_SIZE);
     long totalQueueSize = acuConf.getMemoryInBytes(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX);
 
     String policy = acuConf.get(Property.TSERV_CACHE_POLICY);
     if (policy.equalsIgnoreCase("LRU")) {
       _iCache = new LruBlockCache(iCacheSize, blockSize);
       _dCache = new LruBlockCache(dCacheSize, blockSize);
+      _sCache = new LruBlockCache(sCacheSize, blockSize);
     } else if (policy.equalsIgnoreCase("TinyLFU")) {
       _iCache = new TinyLfuBlockCache(iCacheSize, blockSize);
       _dCache = new TinyLfuBlockCache(dCacheSize, blockSize);
+      _sCache = new TinyLfuBlockCache(sCacheSize, blockSize);
     } else {
       throw new IllegalArgumentException("Unknown Block cache policy " + policy);
     }
@@ -179,14 +191,14 @@ public class TabletServerResourceManager {
     Runtime runtime = Runtime.getRuntime();
     if (usingNativeMap) {
       // Still check block cache sizes when using native maps.
-      if (dCacheSize + iCacheSize + totalQueueSize > runtime.maxMemory()) {
+      if (dCacheSize + iCacheSize + sCacheSize + totalQueueSize > runtime.maxMemory()) {
         throw new IllegalArgumentException(String.format("Block cache sizes %,d and mutation queue size %,d is too large for this JVM configuration %,d",
-            dCacheSize + iCacheSize, totalQueueSize, runtime.maxMemory()));
+            dCacheSize + iCacheSize + sCacheSize, totalQueueSize, runtime.maxMemory()));
       }
-    } else if (maxMemory + dCacheSize + iCacheSize + totalQueueSize > runtime.maxMemory()) {
+    } else if (maxMemory + dCacheSize + iCacheSize + sCacheSize + totalQueueSize > runtime.maxMemory()) {
       throw new IllegalArgumentException(String.format(
           "Maximum tablet server map memory %,d block cache sizes %,d and mutation queue size %,d is too large for this JVM configuration %,d", maxMemory,
-          dCacheSize + iCacheSize, totalQueueSize, runtime.maxMemory()));
+          dCacheSize + iCacheSize + sCacheSize, totalQueueSize, runtime.maxMemory()));
     }
     runtime.gc();
 
@@ -222,6 +234,10 @@ public class TabletServerResourceManager {
     readAheadThreadPool = createEs(Property.TSERV_READ_AHEAD_MAXCONCURRENT, "tablet read ahead");
     defaultReadAheadThreadPool = createEs(Property.TSERV_METADATA_READ_AHEAD_MAXCONCURRENT, "metadata tablets read ahead");
 
+    summaryRetrievalPool = createIdlingEs(Property.TSERV_SUMMARY_RETRIEVAL_THREADS, "summary file retriever", 60, TimeUnit.SECONDS);
+    summaryRemotePool = createIdlingEs(Property.TSERV_SUMMARY_REMOTE_THREADS, "summary remote", 60, TimeUnit.SECONDS);
+    summaryParitionPool = createIdlingEs(Property.TSERV_SUMMARY_PARTITION_THREADS, "summary partition", 60, TimeUnit.SECONDS);
+
     int maxOpenFiles = acuConf.getCount(Property.TSERV_SCAN_MAX_OPENFILES);
 
     fileManager = new FileManager(tserver, fs, maxOpenFiles, _dCache, _iCache);
@@ -657,7 +673,7 @@ public class TabletServerResourceManager {
       CompactionStrategy strategy = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_COMPACTION_STRATEGY, CompactionStrategy.class,
           new DefaultCompactionStrategy());
       strategy.init(Property.getCompactionStrategyOptions(tableConf));
-      MajorCompactionRequest request = new MajorCompactionRequest(extent, reason, TabletServerResourceManager.this.fs, tableConf);
+      MajorCompactionRequest request = new MajorCompactionRequest(extent, reason, tableConf);
       request.setFiles(tabletFiles);
       try {
         return strategy.shouldCompact(request);
@@ -760,4 +776,19 @@ public class TabletServerResourceManager {
     return _dCache;
   }
 
+  public BlockCache getSummaryCache() {
+    return _sCache;
+  }
+
+  public ExecutorService getSummaryRetrievalExecutor() {
+    return summaryRetrievalPool;
+  }
+
+  public ExecutorService getSummaryPartitionExecutor() {
+    return summaryParitionPool;
+  }
+
+  public ExecutorService getSummaryRemoteExecutor() {
+    return summaryRemotePool;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/CompactionStrategy.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/CompactionStrategy.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/CompactionStrategy.java
index 0e0089a..85a4a13 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/CompactionStrategy.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/CompactionStrategy.java
@@ -32,7 +32,6 @@ import java.util.Map;
  * compaction thread.
  */
 public abstract class CompactionStrategy {
-
   /**
    * The settings for the compaction strategy pulled from zookeeper. The <tt>table.compacations.major.strategy.opts</tt> part of the setting will be removed.
    */


[6/9] accumulo git commit: ACCUMULO-4501 ACCUMULO-96 Added Summarization

Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummaryRequest.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummaryRequest.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummaryRequest.java
new file mode 100644
index 0000000..78c242e
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TSummaryRequest.java
@@ -0,0 +1,760 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.10.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.10.0)")
+public class TSummaryRequest implements org.apache.thrift.TBase<TSummaryRequest, TSummaryRequest._Fields>, java.io.Serializable, Cloneable, Comparable<TSummaryRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TSummaryRequest");
+
+  private static final org.apache.thrift.protocol.TField TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableId", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField BOUNDS_FIELD_DESC = new org.apache.thrift.protocol.TField("bounds", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+  private static final org.apache.thrift.protocol.TField SUMMARIZERS_FIELD_DESC = new org.apache.thrift.protocol.TField("summarizers", org.apache.thrift.protocol.TType.LIST, (short)3);
+  private static final org.apache.thrift.protocol.TField SUMMARIZER_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("summarizerPattern", org.apache.thrift.protocol.TType.STRING, (short)4);
+
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TSummaryRequestStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TSummaryRequestTupleSchemeFactory();
+
+  public java.lang.String tableId; // required
+  public TRowRange bounds; // required
+  public java.util.List<TSummarizerConfiguration> summarizers; // required
+  public java.lang.String summarizerPattern; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TABLE_ID((short)1, "tableId"),
+    BOUNDS((short)2, "bounds"),
+    SUMMARIZERS((short)3, "summarizers"),
+    SUMMARIZER_PATTERN((short)4, "summarizerPattern");
+
+    private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+    static {
+      for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TABLE_ID
+          return TABLE_ID;
+        case 2: // BOUNDS
+          return BOUNDS;
+        case 3: // SUMMARIZERS
+          return SUMMARIZERS;
+        case 4: // SUMMARIZER_PATTERN
+          return SUMMARIZER_PATTERN;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(java.lang.String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final java.lang.String _fieldName;
+
+    _Fields(short thriftId, java.lang.String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public java.lang.String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("tableId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.BOUNDS, new org.apache.thrift.meta_data.FieldMetaData("bounds", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRowRange.class)));
+    tmpMap.put(_Fields.SUMMARIZERS, new org.apache.thrift.meta_data.FieldMetaData("summarizers", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSummarizerConfiguration.class))));
+    tmpMap.put(_Fields.SUMMARIZER_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("summarizerPattern", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TSummaryRequest.class, metaDataMap);
+  }
+
+  public TSummaryRequest() {
+  }
+
+  public TSummaryRequest(
+    java.lang.String tableId,
+    TRowRange bounds,
+    java.util.List<TSummarizerConfiguration> summarizers,
+    java.lang.String summarizerPattern)
+  {
+    this();
+    this.tableId = tableId;
+    this.bounds = bounds;
+    this.summarizers = summarizers;
+    this.summarizerPattern = summarizerPattern;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TSummaryRequest(TSummaryRequest other) {
+    if (other.isSetTableId()) {
+      this.tableId = other.tableId;
+    }
+    if (other.isSetBounds()) {
+      this.bounds = new TRowRange(other.bounds);
+    }
+    if (other.isSetSummarizers()) {
+      java.util.List<TSummarizerConfiguration> __this__summarizers = new java.util.ArrayList<TSummarizerConfiguration>(other.summarizers.size());
+      for (TSummarizerConfiguration other_element : other.summarizers) {
+        __this__summarizers.add(new TSummarizerConfiguration(other_element));
+      }
+      this.summarizers = __this__summarizers;
+    }
+    if (other.isSetSummarizerPattern()) {
+      this.summarizerPattern = other.summarizerPattern;
+    }
+  }
+
+  public TSummaryRequest deepCopy() {
+    return new TSummaryRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.tableId = null;
+    this.bounds = null;
+    this.summarizers = null;
+    this.summarizerPattern = null;
+  }
+
+  public java.lang.String getTableId() {
+    return this.tableId;
+  }
+
+  public TSummaryRequest setTableId(java.lang.String tableId) {
+    this.tableId = tableId;
+    return this;
+  }
+
+  public void unsetTableId() {
+    this.tableId = null;
+  }
+
+  /** Returns true if field tableId is set (has been assigned a value) and false otherwise */
+  public boolean isSetTableId() {
+    return this.tableId != null;
+  }
+
+  public void setTableIdIsSet(boolean value) {
+    if (!value) {
+      this.tableId = null;
+    }
+  }
+
+  public TRowRange getBounds() {
+    return this.bounds;
+  }
+
+  public TSummaryRequest setBounds(TRowRange bounds) {
+    this.bounds = bounds;
+    return this;
+  }
+
+  public void unsetBounds() {
+    this.bounds = null;
+  }
+
+  /** Returns true if field bounds is set (has been assigned a value) and false otherwise */
+  public boolean isSetBounds() {
+    return this.bounds != null;
+  }
+
+  public void setBoundsIsSet(boolean value) {
+    if (!value) {
+      this.bounds = null;
+    }
+  }
+
+  public int getSummarizersSize() {
+    return (this.summarizers == null) ? 0 : this.summarizers.size();
+  }
+
+  public java.util.Iterator<TSummarizerConfiguration> getSummarizersIterator() {
+    return (this.summarizers == null) ? null : this.summarizers.iterator();
+  }
+
+  public void addToSummarizers(TSummarizerConfiguration elem) {
+    if (this.summarizers == null) {
+      this.summarizers = new java.util.ArrayList<TSummarizerConfiguration>();
+    }
+    this.summarizers.add(elem);
+  }
+
+  public java.util.List<TSummarizerConfiguration> getSummarizers() {
+    return this.summarizers;
+  }
+
+  public TSummaryRequest setSummarizers(java.util.List<TSummarizerConfiguration> summarizers) {
+    this.summarizers = summarizers;
+    return this;
+  }
+
+  public void unsetSummarizers() {
+    this.summarizers = null;
+  }
+
+  /** Returns true if field summarizers is set (has been assigned a value) and false otherwise */
+  public boolean isSetSummarizers() {
+    return this.summarizers != null;
+  }
+
+  public void setSummarizersIsSet(boolean value) {
+    if (!value) {
+      this.summarizers = null;
+    }
+  }
+
+  public java.lang.String getSummarizerPattern() {
+    return this.summarizerPattern;
+  }
+
+  public TSummaryRequest setSummarizerPattern(java.lang.String summarizerPattern) {
+    this.summarizerPattern = summarizerPattern;
+    return this;
+  }
+
+  public void unsetSummarizerPattern() {
+    this.summarizerPattern = null;
+  }
+
+  /** Returns true if field summarizerPattern is set (has been assigned a value) and false otherwise */
+  public boolean isSetSummarizerPattern() {
+    return this.summarizerPattern != null;
+  }
+
+  public void setSummarizerPatternIsSet(boolean value) {
+    if (!value) {
+      this.summarizerPattern = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, java.lang.Object value) {
+    switch (field) {
+    case TABLE_ID:
+      if (value == null) {
+        unsetTableId();
+      } else {
+        setTableId((java.lang.String)value);
+      }
+      break;
+
+    case BOUNDS:
+      if (value == null) {
+        unsetBounds();
+      } else {
+        setBounds((TRowRange)value);
+      }
+      break;
+
+    case SUMMARIZERS:
+      if (value == null) {
+        unsetSummarizers();
+      } else {
+        setSummarizers((java.util.List<TSummarizerConfiguration>)value);
+      }
+      break;
+
+    case SUMMARIZER_PATTERN:
+      if (value == null) {
+        unsetSummarizerPattern();
+      } else {
+        setSummarizerPattern((java.lang.String)value);
+      }
+      break;
+
+    }
+  }
+
+  public java.lang.Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TABLE_ID:
+      return getTableId();
+
+    case BOUNDS:
+      return getBounds();
+
+    case SUMMARIZERS:
+      return getSummarizers();
+
+    case SUMMARIZER_PATTERN:
+      return getSummarizerPattern();
+
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new java.lang.IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TABLE_ID:
+      return isSetTableId();
+    case BOUNDS:
+      return isSetBounds();
+    case SUMMARIZERS:
+      return isSetSummarizers();
+    case SUMMARIZER_PATTERN:
+      return isSetSummarizerPattern();
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(java.lang.Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TSummaryRequest)
+      return this.equals((TSummaryRequest)that);
+    return false;
+  }
+
+  public boolean equals(TSummaryRequest that) {
+    if (that == null)
+      return false;
+    if (this == that)
+      return true;
+
+    boolean this_present_tableId = true && this.isSetTableId();
+    boolean that_present_tableId = true && that.isSetTableId();
+    if (this_present_tableId || that_present_tableId) {
+      if (!(this_present_tableId && that_present_tableId))
+        return false;
+      if (!this.tableId.equals(that.tableId))
+        return false;
+    }
+
+    boolean this_present_bounds = true && this.isSetBounds();
+    boolean that_present_bounds = true && that.isSetBounds();
+    if (this_present_bounds || that_present_bounds) {
+      if (!(this_present_bounds && that_present_bounds))
+        return false;
+      if (!this.bounds.equals(that.bounds))
+        return false;
+    }
+
+    boolean this_present_summarizers = true && this.isSetSummarizers();
+    boolean that_present_summarizers = true && that.isSetSummarizers();
+    if (this_present_summarizers || that_present_summarizers) {
+      if (!(this_present_summarizers && that_present_summarizers))
+        return false;
+      if (!this.summarizers.equals(that.summarizers))
+        return false;
+    }
+
+    boolean this_present_summarizerPattern = true && this.isSetSummarizerPattern();
+    boolean that_present_summarizerPattern = true && that.isSetSummarizerPattern();
+    if (this_present_summarizerPattern || that_present_summarizerPattern) {
+      if (!(this_present_summarizerPattern && that_present_summarizerPattern))
+        return false;
+      if (!this.summarizerPattern.equals(that.summarizerPattern))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int hashCode = 1;
+
+    hashCode = hashCode * 8191 + ((isSetTableId()) ? 131071 : 524287);
+    if (isSetTableId())
+      hashCode = hashCode * 8191 + tableId.hashCode();
+
+    hashCode = hashCode * 8191 + ((isSetBounds()) ? 131071 : 524287);
+    if (isSetBounds())
+      hashCode = hashCode * 8191 + bounds.hashCode();
+
+    hashCode = hashCode * 8191 + ((isSetSummarizers()) ? 131071 : 524287);
+    if (isSetSummarizers())
+      hashCode = hashCode * 8191 + summarizers.hashCode();
+
+    hashCode = hashCode * 8191 + ((isSetSummarizerPattern()) ? 131071 : 524287);
+    if (isSetSummarizerPattern())
+      hashCode = hashCode * 8191 + summarizerPattern.hashCode();
+
+    return hashCode;
+  }
+
+  @Override
+  public int compareTo(TSummaryRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = java.lang.Boolean.valueOf(isSetTableId()).compareTo(other.isSetTableId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTableId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableId, other.tableId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetBounds()).compareTo(other.isSetBounds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetBounds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.bounds, other.bounds);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetSummarizers()).compareTo(other.isSetSummarizers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSummarizers()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.summarizers, other.summarizers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetSummarizerPattern()).compareTo(other.isSetSummarizerPattern());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSummarizerPattern()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.summarizerPattern, other.summarizerPattern);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    scheme(iprot).read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    scheme(oprot).write(oprot, this);
+  }
+
+  @Override
+  public java.lang.String toString() {
+    java.lang.StringBuilder sb = new java.lang.StringBuilder("TSummaryRequest(");
+    boolean first = true;
+
+    sb.append("tableId:");
+    if (this.tableId == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tableId);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("bounds:");
+    if (this.bounds == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.bounds);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("summarizers:");
+    if (this.summarizers == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.summarizers);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("summarizerPattern:");
+    if (this.summarizerPattern == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.summarizerPattern);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (bounds != null) {
+      bounds.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TSummaryRequestStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public TSummaryRequestStandardScheme getScheme() {
+      return new TSummaryRequestStandardScheme();
+    }
+  }
+
+  private static class TSummaryRequestStandardScheme extends org.apache.thrift.scheme.StandardScheme<TSummaryRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TSummaryRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TABLE_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tableId = iprot.readString();
+              struct.setTableIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // BOUNDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.bounds = new TRowRange();
+              struct.bounds.read(iprot);
+              struct.setBoundsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // SUMMARIZERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list122 = iprot.readListBegin();
+                struct.summarizers = new java.util.ArrayList<TSummarizerConfiguration>(_list122.size);
+                TSummarizerConfiguration _elem123;
+                for (int _i124 = 0; _i124 < _list122.size; ++_i124)
+                {
+                  _elem123 = new TSummarizerConfiguration();
+                  _elem123.read(iprot);
+                  struct.summarizers.add(_elem123);
+                }
+                iprot.readListEnd();
+              }
+              struct.setSummarizersIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // SUMMARIZER_PATTERN
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.summarizerPattern = iprot.readString();
+              struct.setSummarizerPatternIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TSummaryRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.tableId != null) {
+        oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
+        oprot.writeString(struct.tableId);
+        oprot.writeFieldEnd();
+      }
+      if (struct.bounds != null) {
+        oprot.writeFieldBegin(BOUNDS_FIELD_DESC);
+        struct.bounds.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.summarizers != null) {
+        oprot.writeFieldBegin(SUMMARIZERS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.summarizers.size()));
+          for (TSummarizerConfiguration _iter125 : struct.summarizers)
+          {
+            _iter125.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.summarizerPattern != null) {
+        oprot.writeFieldBegin(SUMMARIZER_PATTERN_FIELD_DESC);
+        oprot.writeString(struct.summarizerPattern);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TSummaryRequestTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public TSummaryRequestTupleScheme getScheme() {
+      return new TSummaryRequestTupleScheme();
+    }
+  }
+
+  private static class TSummaryRequestTupleScheme extends org.apache.thrift.scheme.TupleScheme<TSummaryRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TSummaryRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet optionals = new java.util.BitSet();
+      if (struct.isSetTableId()) {
+        optionals.set(0);
+      }
+      if (struct.isSetBounds()) {
+        optionals.set(1);
+      }
+      if (struct.isSetSummarizers()) {
+        optionals.set(2);
+      }
+      if (struct.isSetSummarizerPattern()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
+      if (struct.isSetTableId()) {
+        oprot.writeString(struct.tableId);
+      }
+      if (struct.isSetBounds()) {
+        struct.bounds.write(oprot);
+      }
+      if (struct.isSetSummarizers()) {
+        {
+          oprot.writeI32(struct.summarizers.size());
+          for (TSummarizerConfiguration _iter126 : struct.summarizers)
+          {
+            _iter126.write(oprot);
+          }
+        }
+      }
+      if (struct.isSetSummarizerPattern()) {
+        oprot.writeString(struct.summarizerPattern);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TSummaryRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet incoming = iprot.readBitSet(4);
+      if (incoming.get(0)) {
+        struct.tableId = iprot.readString();
+        struct.setTableIdIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.bounds = new TRowRange();
+        struct.bounds.read(iprot);
+        struct.setBoundsIsSet(true);
+      }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TList _list127 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.summarizers = new java.util.ArrayList<TSummarizerConfiguration>(_list127.size);
+          TSummarizerConfiguration _elem128;
+          for (int _i129 = 0; _i129 < _list127.size; ++_i129)
+          {
+            _elem128 = new TSummarizerConfiguration();
+            _elem128.read(iprot);
+            struct.summarizers.add(_elem128);
+          }
+        }
+        struct.setSummarizersIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.summarizerPattern = iprot.readString();
+        struct.setSummarizerPatternIsSet(true);
+      }
+    }
+  }
+
+  private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+    return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+  }
+  private static void unusedMethod() {}
+}
+

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
index 6e5728a..4808da3 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
@@ -97,12 +97,12 @@ public class BloomFilterLayer {
     private boolean closed = false;
     private long length = -1;
 
-    Writer(FileSKVWriter writer, AccumuloConfiguration acuconf) {
+    Writer(FileSKVWriter writer, AccumuloConfiguration acuconf, boolean useAccumuloStart) {
       this.writer = writer;
-      initBloomFilter(acuconf);
+      initBloomFilter(acuconf, useAccumuloStart);
     }
 
-    private synchronized void initBloomFilter(AccumuloConfiguration acuconf) {
+    private synchronized void initBloomFilter(AccumuloConfiguration acuconf, boolean useAccumuloStart) {
 
       numKeys = acuconf.getCount(Property.TABLE_BLOOM_SIZE);
       // vector size should be <code>-kn / (ln(1 - c^(1/k)))</code> bits for
@@ -121,7 +121,9 @@ public class BloomFilterLayer {
         String context = acuconf.get(Property.TABLE_CLASSPATH);
         String classname = acuconf.get(Property.TABLE_BLOOM_KEY_FUNCTOR);
         Class<? extends KeyFunctor> clazz;
-        if (context != null && !context.equals(""))
+        if (!useAccumuloStart)
+          clazz = Writer.class.getClassLoader().loadClass(classname).asSubclass(KeyFunctor.class);
+        else if (context != null && !context.equals(""))
           clazz = AccumuloVFSClassLoader.getContextManager().loadClass(context, classname, KeyFunctor.class);
         else
           clazz = AccumuloVFSClassLoader.loadClass(classname, KeyFunctor.class);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java b/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
index c7d8248..e36b30f 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
@@ -23,6 +23,7 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.file.map.MapFileOperations;
 import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.file.rfile.RFileOperations;
+import org.apache.accumulo.core.summary.SummaryWriter;
 import org.apache.hadoop.fs.Path;
 
 class DispatchingFileFactory extends FileOperations {
@@ -73,10 +74,10 @@ class DispatchingFileFactory extends FileOperations {
   protected FileSKVWriter openWriter(OpenWriterOperation options) throws IOException {
     FileSKVWriter writer = findFileFactory(options).openWriter(options);
     if (options.getTableConfiguration().getBoolean(Property.TABLE_BLOOM_ENABLED)) {
-      return new BloomFilterLayer.Writer(writer, options.getTableConfiguration());
-    } else {
-      return writer;
+      writer = new BloomFilterLayer.Writer(writer, options.getTableConfiguration(), options.isAccumuloStartEnabled());
     }
+
+    return SummaryWriter.wrap(writer, options.getTableConfiguration(), options.isAccumuloStartEnabled());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
index 10bb784..67757bc 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
@@ -292,6 +292,7 @@ public abstract class FileOperations {
       NeedsFileOrOuputStream<OpenWriterOperationBuilder> {
     private String compression;
     private FSDataOutputStream outputStream;
+    private boolean enableAccumuloStart = true;
 
     @Override
     public NeedsTableConfiguration<OpenWriterOperationBuilder> forOutputStream(String extenstion, FSDataOutputStream outputStream, Configuration fsConf) {
@@ -301,6 +302,16 @@ public abstract class FileOperations {
       return this;
     }
 
+    public boolean isAccumuloStartEnabled() {
+      return enableAccumuloStart;
+    }
+
+    @Override
+    public OpenWriterOperation setAccumuloStartEnabled(boolean enableAccumuloStart) {
+      this.enableAccumuloStart = enableAccumuloStart;
+      return this;
+    }
+
     @Override
     public OpenWriterOperation withCompression(String compression) {
       this.compression = compression;
@@ -337,6 +348,13 @@ public abstract class FileOperations {
     /** Set the compression type. */
     public OpenWriterOperationBuilder withCompression(String compression);
 
+    /**
+     * Classes may be instantiated as part of a write operation. For example if BloomFilters, Samplers, or Summarizers are used then classes are loaded. When
+     * running in a tserver, Accumulo start should be used to load classes. When running in a client process, Accumulo start should not be used. This method
+     * makes it possible to specify if Accumulo Start should be used to load classes. Calling this method is optional and the default is true.
+     */
+    public OpenWriterOperationBuilder setAccumuloStartEnabled(boolean enableAccumuloStart);
+
     /** Construct the writer. */
     public FileSKVWriter build() throws IOException;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
index 00ebb7a..d17528c 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
 import org.apache.accumulo.core.file.rfile.RFile.Reader;
+import org.apache.accumulo.core.summary.SummaryReader;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.start.spi.KeywordExecutable;
 import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
@@ -58,6 +59,8 @@ public class PrintInfo implements KeywordExecutable {
     boolean histogram = false;
     @Parameter(names = {"--useSample"}, description = "Use sample data for --dump, --vis, --histogram options")
     boolean useSample = false;
+    @Parameter(names = {"--summary"}, description = "Print summary data in file")
+    boolean printSummary = false;
     @Parameter(names = {"--keyStats"}, description = "print key length statistics for index and all data")
     boolean keyStats = false;
     @Parameter(description = " <file> { <file> ... }")
@@ -210,6 +213,10 @@ public class PrintInfo implements KeywordExecutable {
         }
       }
 
+      if (opts.printSummary) {
+        SummaryReader.print(iter, System.out);
+      }
+
       iter.close();
 
       if (opts.vis || opts.hash) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
index 96d31ce..ec721ba 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
@@ -88,7 +88,7 @@ public class RFileOperations extends FileOperations {
     Sampler sampler = null;
 
     if (samplerConfig != null) {
-      sampler = SamplerFactory.newSampler(samplerConfig, acuconf);
+      sampler = SamplerFactory.newSampler(samplerConfig, acuconf, options.isAccumuloStartEnabled());
     }
 
     String compression = options.getCompression();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java
new file mode 100644
index 0000000..7e92b64
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.metadata.schema;
+
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN;
+
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.IsolatedScanner;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.ClientContext;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.FetchedColumns;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.ColumnFQ;
+import org.apache.hadoop.io.Text;
+
+import com.google.common.base.Preconditions;
+
+public class MetadataScanner {
+
+  public static interface SourceOptions {
+    TableOptions from(Scanner scanner);
+
+    TableOptions from(ClientContext ctx);
+  }
+
+  public static interface TableOptions {
+    ColumnOptions overRootTable();
+
+    ColumnOptions overMetadataTable();
+
+    ColumnOptions overUserTableId(String tableId);
+
+    ColumnOptions overUserTableId(String tableId, Text startRow, Text endRow);
+  }
+
+  public static interface ColumnOptions {
+    public ColumnOptions fetchFiles();
+
+    public ColumnOptions fetchLocation();
+
+    public ColumnOptions fetchPrev();
+
+    public ColumnOptions fetchLast();
+
+    public Iterable<TabletMetadata> build() throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
+  }
+
+  private static class TabletMetadataIterator implements Iterator<TabletMetadata> {
+
+    private boolean sawLast = false;
+    private Iterator<TabletMetadata> iter;
+    private Text endRow;
+
+    TabletMetadataIterator(Iterator<TabletMetadata> source, Text endRow) {
+      this.iter = source;
+      this.endRow = endRow;
+    }
+
+    @Override
+    public boolean hasNext() {
+      return !sawLast && iter.hasNext();
+    }
+
+    @Override
+    public TabletMetadata next() {
+      if (sawLast) {
+        throw new NoSuchElementException();
+      }
+      TabletMetadata next = iter.next();
+      if (next.getExtent().contains(endRow)) {
+        sawLast = true;
+      }
+      return next;
+    }
+  }
+
+  private static class Builder implements SourceOptions, TableOptions, ColumnOptions {
+
+    private List<Text> families = new ArrayList<>();
+    private List<ColumnFQ> qualifiers = new ArrayList<>();
+    private Scanner scanner;
+    private ClientContext ctx;
+    private String table;
+    private String userTableId;
+    private EnumSet<FetchedColumns> fetchedCols = EnumSet.noneOf(FetchedColumns.class);
+    private Text startRow;
+    private Text endRow;
+
+    @Override
+    public ColumnOptions fetchFiles() {
+      fetchedCols.add(FetchedColumns.FILES);
+      families.add(DataFileColumnFamily.NAME);
+      return this;
+    }
+
+    @Override
+    public ColumnOptions fetchLocation() {
+      fetchedCols.add(FetchedColumns.LOCATION);
+      families.add(CurrentLocationColumnFamily.NAME);
+      families.add(FutureLocationColumnFamily.NAME);
+      return this;
+    }
+
+    @Override
+    public ColumnOptions fetchPrev() {
+      fetchedCols.add(FetchedColumns.PREV_ROW);
+      qualifiers.add(PREV_ROW_COLUMN);
+      return this;
+    }
+
+    @Override
+    public ColumnOptions fetchLast() {
+      fetchedCols.add(FetchedColumns.LAST);
+      families.add(LastLocationColumnFamily.NAME);
+      return this;
+    }
+
+    @Override
+    public Iterable<TabletMetadata> build() throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+      if (ctx != null) {
+        scanner = new IsolatedScanner(ctx.getConnector().createScanner(table, Authorizations.EMPTY));
+      } else if (!(scanner instanceof IsolatedScanner)) {
+        scanner = new IsolatedScanner(scanner);
+      }
+
+      if (userTableId != null) {
+        scanner.setRange(new KeyExtent(userTableId, null, startRow).toMetadataRange());
+      }
+
+      for (Text fam : families) {
+        scanner.fetchColumnFamily(fam);
+      }
+
+      for (ColumnFQ col : qualifiers) {
+        col.fetch(scanner);
+      }
+
+      if (families.size() == 0 && qualifiers.size() == 0) {
+        fetchedCols = EnumSet.allOf(FetchedColumns.class);
+      }
+
+      Iterable<TabletMetadata> tmi = TabletMetadata.convert(scanner, fetchedCols);
+
+      if (endRow != null) {
+        // create an iterable that will stop at the tablet which contains the endRow
+        return new Iterable<TabletMetadata>() {
+          @Override
+          public Iterator<TabletMetadata> iterator() {
+            return new TabletMetadataIterator(tmi.iterator(), endRow);
+          }
+        };
+      } else {
+        return tmi;
+      }
+
+    }
+
+    @Override
+    public ColumnOptions overRootTable() {
+      this.table = RootTable.NAME;
+      return this;
+    }
+
+    @Override
+    public ColumnOptions overMetadataTable() {
+      this.table = MetadataTable.NAME;
+      return this;
+    }
+
+    @Override
+    public ColumnOptions overUserTableId(String tableId) {
+      Preconditions.checkArgument(!tableId.equals(RootTable.ID) && !tableId.equals(MetadataTable.ID));
+
+      this.table = MetadataTable.NAME;
+      this.userTableId = tableId;
+      return this;
+    }
+
+    @Override
+    public TableOptions from(Scanner scanner) {
+      this.scanner = scanner;
+      return this;
+    }
+
+    @Override
+    public TableOptions from(ClientContext ctx) {
+      this.ctx = ctx;
+      return this;
+    }
+
+    @Override
+    public ColumnOptions overUserTableId(String tableId, Text startRow, Text endRow) {
+      this.table = MetadataTable.NAME;
+      this.userTableId = tableId;
+      this.startRow = startRow;
+      this.endRow = endRow;
+      return this;
+    }
+
+  }
+
+  public static SourceOptions builder() {
+    return new Builder();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
new file mode 100644
index 0000000..af5f814
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.metadata.schema;
+
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN;
+
+import java.util.EnumSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Objects;
+
+import org.apache.accumulo.core.client.RowIterator;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
+import org.apache.hadoop.io.Text;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableList.Builder;
+import com.google.common.collect.Iterators;
+import com.google.common.net.HostAndPort;
+
+public class TabletMetadata {
+
+  private String tableId;
+  private Text prevEndRow;
+  private Text endRow;
+  private Location location;
+  private List<String> files;
+  private EnumSet<FetchedColumns> fetchedColumns;
+  private KeyExtent extent;
+  private Location last;
+
+  public static enum LocationType {
+    CURRENT, FUTURE, LAST
+  }
+
+  public static enum FetchedColumns {
+    LOCATION, PREV_ROW, FILES, LAST
+  }
+
+  public static class Location {
+    private final String server;
+    private final String session;
+    private final LocationType lt;
+
+    Location(String server, String session, LocationType lt) {
+      this.server = server;
+      this.session = session;
+      this.lt = lt;
+    }
+
+    public HostAndPort getHostAndPort() {
+      return HostAndPort.fromString(server);
+    }
+
+    public String getSession() {
+      return session;
+    }
+
+    public LocationType getLocationType() {
+      return lt;
+    }
+  }
+
+  public String getTableId() {
+    return tableId;
+  }
+
+  public KeyExtent getExtent() {
+    if (extent == null) {
+      extent = new KeyExtent(getTableId(), getEndRow(), getPrevEndRow());
+    }
+    return extent;
+  }
+
+  public Text getPrevEndRow() {
+    Preconditions.checkState(fetchedColumns.contains(FetchedColumns.PREV_ROW), "Requested prev row when it was not fetched");
+    return prevEndRow;
+  }
+
+  public Text getEndRow() {
+    return endRow;
+  }
+
+  public Location getLocation() {
+    Preconditions.checkState(fetchedColumns.contains(FetchedColumns.LOCATION), "Requested location when it was not fetched");
+    return location;
+  }
+
+  public Location getLast() {
+    Preconditions.checkState(fetchedColumns.contains(FetchedColumns.LAST), "Requested last when it was not fetched");
+    return last;
+  }
+
+  public List<String> getFiles() {
+    Preconditions.checkState(fetchedColumns.contains(FetchedColumns.FILES), "Requested files when it was not fetched");
+    return files;
+  }
+
+  public static TabletMetadata convertRow(Iterator<Entry<Key,Value>> rowIter, EnumSet<FetchedColumns> fetchedColumns) {
+    Objects.requireNonNull(rowIter);
+
+    TabletMetadata te = new TabletMetadata();
+
+    Builder<String> filesBuilder = ImmutableList.builder();
+    ByteSequence row = null;
+
+    while (rowIter.hasNext()) {
+      Entry<Key,Value> kv = rowIter.next();
+      Key k = kv.getKey();
+      Value v = kv.getValue();
+      Text fam = k.getColumnFamily();
+
+      if (row == null) {
+        row = k.getRowData();
+        KeyExtent ke = new KeyExtent(k.getRow(), (Text) null);
+        te.endRow = ke.getEndRow();
+        te.tableId = ke.getTableId();
+      } else if (!row.equals(k.getRowData())) {
+        throw new IllegalArgumentException("Input contains more than one row : " + row + " " + k.getRowData());
+      }
+
+      if (PREV_ROW_COLUMN.hasColumns(k)) {
+        te.prevEndRow = KeyExtent.decodePrevEndRow(v);
+      }
+
+      if (fam.equals(DataFileColumnFamily.NAME)) {
+        filesBuilder.add(k.getColumnQualifier().toString());
+      } else if (fam.equals(CurrentLocationColumnFamily.NAME)) {
+        if (te.location != null) {
+          throw new IllegalArgumentException("Input contains more than one location " + te.location + " " + v);
+        }
+        te.location = new Location(v.toString(), k.getColumnQualifierData().toString(), LocationType.CURRENT);
+      } else if (fam.equals(FutureLocationColumnFamily.NAME)) {
+        if (te.location != null) {
+          throw new IllegalArgumentException("Input contains more than one location " + te.location + " " + v);
+        }
+        te.location = new Location(v.toString(), k.getColumnQualifierData().toString(), LocationType.FUTURE);
+      } else if (fam.equals(LastLocationColumnFamily.NAME)) {
+        te.last = new Location(v.toString(), k.getColumnQualifierData().toString(), LocationType.LAST);
+      }
+    }
+
+    te.files = filesBuilder.build();
+    te.fetchedColumns = fetchedColumns;
+    return te;
+  }
+
+  public static Iterable<TabletMetadata> convert(Scanner input, EnumSet<FetchedColumns> fetchedColumns) {
+    return new Iterable<TabletMetadata>() {
+      @Override
+      public Iterator<TabletMetadata> iterator() {
+        RowIterator rowIter = new RowIterator(input);
+        return Iterators.transform(rowIter, ri -> convertRow(ri, fetchedColumns));
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java b/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java
index d3e2fe7..7c622b0 100644
--- a/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java
@@ -17,8 +17,6 @@
 
 package org.apache.accumulo.core.sample.impl;
 
-import static com.google.common.base.Preconditions.checkArgument;
-
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -171,16 +169,6 @@ public class SamplerConfigurationImpl implements Writable {
     return className + " " + options;
   }
 
-  public static void checkDisjoint(Map<String,String> props, SamplerConfiguration samplerConfiguration) {
-    if (props.isEmpty() || samplerConfiguration == null) {
-      return;
-    }
-
-    Map<String,String> sampleProps = new SamplerConfigurationImpl(samplerConfiguration).toTablePropertiesMap();
-
-    checkArgument(Collections.disjoint(props.keySet(), sampleProps.keySet()), "Properties and derived sampler properties are not disjoint");
-  }
-
   public static TSamplerConfiguration toThrift(SamplerConfiguration samplerConfig) {
     if (samplerConfig == null)
       return null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerFactory.java b/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerFactory.java
index d70f3af..0cf75ae 100644
--- a/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerFactory.java
@@ -25,11 +25,13 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 
 public class SamplerFactory {
-  public static Sampler newSampler(SamplerConfigurationImpl config, AccumuloConfiguration acuconf) throws IOException {
+  public static Sampler newSampler(SamplerConfigurationImpl config, AccumuloConfiguration acuconf, boolean useAccumuloStart) throws IOException {
     String context = acuconf.get(Property.TABLE_CLASSPATH);
 
     Class<? extends Sampler> clazz;
     try {
+      if (!useAccumuloStart)
+        clazz = SamplerFactory.class.getClassLoader().loadClass(config.getClassName()).asSubclass(Sampler.class);
       if (context != null && !context.equals(""))
         clazz = AccumuloVFSClassLoader.getContextManager().loadClass(context, config.getClassName(), Sampler.class);
       else
@@ -45,4 +47,8 @@ public class SamplerFactory {
       throw new RuntimeException(e);
     }
   }
+
+  public static Sampler newSampler(SamplerConfigurationImpl config, AccumuloConfiguration acuconf) throws IOException {
+    return newSampler(config, acuconf, true);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/security/TablePermission.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/TablePermission.java b/core/src/main/java/org/apache/accumulo/core/security/TablePermission.java
index a80be9a..d721ebc 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/TablePermission.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/TablePermission.java
@@ -33,11 +33,12 @@ public enum TablePermission {
   BULK_IMPORT((byte) 4),
   ALTER_TABLE((byte) 5),
   GRANT((byte) 6),
-  DROP_TABLE((byte) 7);
+  DROP_TABLE((byte) 7),
+  GET_SUMMARIES((byte) 8);
 
   final private byte permID;
 
-  final private static TablePermission mapping[] = new TablePermission[8];
+  final private static TablePermission mapping[] = new TablePermission[9];
   static {
     for (TablePermission perm : TablePermission.values())
       mapping[perm.permID] = perm;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java b/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
new file mode 100644
index 0000000..8389051
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
@@ -0,0 +1,631 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.summary;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.ClientContext;
+import org.apache.accumulo.core.client.impl.ServerClient;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.core.data.thrift.TRowRange;
+import org.apache.accumulo.core.data.thrift.TSummaries;
+import org.apache.accumulo.core.data.thrift.TSummaryRequest;
+import org.apache.accumulo.core.file.blockfile.cache.BlockCache;
+import org.apache.accumulo.core.metadata.schema.MetadataScanner;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
+import org.apache.accumulo.core.trace.Tracer;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.util.CancelFlagFuture;
+import org.apache.accumulo.core.util.CompletableFutureUtil;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.hash.Hashing;
+import com.google.common.net.HostAndPort;
+
+/**
+ * This class implements using multiple tservers to gather summaries.
+ *
+ * Below is a rough outline of the RPC process.
+ *
+ * <ol>
+ * <li>Clients pick a random tserver and make an RPC to remotely execute {@link #gather(ExecutorService)}.
+ * <li> {@link #gather(ExecutorService)} will call make RPC calls to multiple tservers to remotely execute {@link #processPartition(ExecutorService, int, int)}
+ * <li> {@link #processPartition(ExecutorService, int, int)} will make RPC calls to multiple tserver to remotely execute
+ * <li> {@link #processFiles(FileSystemResolver, Map, BlockCache, BlockCache, ExecutorService)}
+ * </ol>
+ */
+public class Gatherer {
+
+  private static final Logger log = LoggerFactory.getLogger(Gatherer.class);
+
+  private ClientContext ctx;
+  private String tableId;
+  private SummarizerFactory factory;
+  private Text startRow = null;
+  private Text endRow = null;
+  private Range clipRange;
+  private Predicate<SummarizerConfiguration> summarySelector;
+
+  private TSummaryRequest request;
+
+  private String summarizerPattern;
+
+  private Set<SummarizerConfiguration> summaries;
+
+  public Gatherer(ClientContext context, TSummaryRequest request, AccumuloConfiguration tableConfig) {
+    this.ctx = context;
+    this.tableId = request.tableId;
+    this.startRow = ByteBufferUtil.toText(request.bounds.startRow);
+    this.endRow = ByteBufferUtil.toText(request.bounds.endRow);
+    this.clipRange = new Range(startRow, false, endRow, true);
+    this.summaries = request.getSummarizers().stream().map(SummarizerConfigurationUtil::fromThrift).collect(Collectors.toSet());
+    this.request = request;
+
+    this.summarizerPattern = request.getSummarizerPattern();
+
+    if (summarizerPattern != null) {
+      Pattern pattern = Pattern.compile(summarizerPattern);
+      // The way conf is converted to string below is documented in the API, so consider this when making changes!
+      summarySelector = conf -> pattern.matcher(conf.getClassName() + " " + new TreeMap<>(conf.getOptions())).matches();
+      if (!summaries.isEmpty()) {
+        summarySelector = summarySelector.or(conf -> summaries.contains(conf));
+      }
+    } else if (!summaries.isEmpty()) {
+      summarySelector = conf -> summaries.contains(conf);
+    } else {
+      summarySelector = conf -> true;
+    }
+
+    this.factory = new SummarizerFactory(tableConfig);
+  }
+
+  private TSummaryRequest getRequest() {
+    return request;
+  }
+
+  /**
+   * @param fileSelector
+   *          only returns files that match this predicate
+   * @return A map of the form : {@code map<tserver location, map<path, list<range>>} . The ranges associated with a file represent the tablets that use the
+   *         file.
+   */
+  private Map<String,Map<String,List<TRowRange>>> getFilesGroupedByLocation(Predicate<String> fileSelector) throws TableNotFoundException, AccumuloException,
+      AccumuloSecurityException {
+
+    Iterable<TabletMetadata> tmi = MetadataScanner.builder().from(ctx).overUserTableId(tableId, startRow, endRow).fetchFiles().fetchLocation().fetchLast()
+        .fetchPrev().build();
+
+    // get a subset of files
+    Map<String,List<TabletMetadata>> files = new HashMap<>();
+    for (TabletMetadata tm : tmi) {
+      for (String file : tm.getFiles()) {
+        if (fileSelector.test(file)) {
+          // TODO push this filtering to server side and possibly use batch scanner
+          files.computeIfAbsent(file, s -> new ArrayList<>()).add(tm);
+        }
+      }
+    }
+
+    // group by location, then file
+
+    Map<String,Map<String,List<TRowRange>>> locations = new HashMap<>();
+
+    List<String> tservers = null;
+
+    for (Entry<String,List<TabletMetadata>> entry : files.entrySet()) {
+
+      String location = entry.getValue().stream().filter(tm -> tm.getLocation() != null) // filter tablets w/o a location
+          .map(tm -> tm.getLocation().getHostAndPort().toString()) // convert to host:port strings
+          .min(String::compareTo) // find minimum host:port
+          .orElse(entry.getValue().stream().filter(tm -> tm.getLast() != null) // if no locations, then look at last locations
+              .map(tm -> tm.getLast().getHostAndPort().toString()) // convert to host:port strings
+              .min(String::compareTo).orElse(null)); // find minimum last location or return null
+
+      if (location == null) {
+        if (tservers == null) {
+          tservers = ctx.getConnector().instanceOperations().getTabletServers();
+          Collections.sort(tservers);
+        }
+
+        // When no location, the approach below will consistently choose the same tserver for the same file (as long as the set of tservers is stable).
+        int idx = Math.abs(Hashing.murmur3_32().hashString(entry.getKey()).asInt()) % tservers.size();
+        location = tservers.get(idx);
+      }
+
+      List<Range> merged = Range.mergeOverlapping(Lists.transform(entry.getValue(), tm -> tm.getExtent().toDataRange())); // merge contiguous ranges
+      List<TRowRange> ranges = merged.stream().map(r -> toClippedExtent(r).toThrift()).collect(Collectors.toList()); // clip ranges to queried range
+
+      locations.computeIfAbsent(location, s -> new HashMap<>()).put(entry.getKey(), ranges);
+    }
+
+    return locations;
+  }
+
+  private <K,V> Iterable<Map<K,V>> partition(Map<K,V> map, int max) {
+
+    if (map.size() < max) {
+      return Collections.singletonList(map);
+    }
+
+    return new Iterable<Map<K,V>>() {
+      @Override
+      public Iterator<Map<K,V>> iterator() {
+        Iterator<Entry<K,V>> esi = map.entrySet().iterator();
+
+        return new Iterator<Map<K,V>>() {
+          @Override
+          public boolean hasNext() {
+            return esi.hasNext();
+          }
+
+          @Override
+          public Map<K,V> next() {
+            Map<K,V> workingMap = new HashMap<>(max);
+            while (esi.hasNext() && workingMap.size() < max) {
+              Entry<K,V> entry = esi.next();
+              workingMap.put(entry.getKey(), entry.getValue());
+            }
+            return workingMap;
+          }
+        };
+      }
+    };
+  }
+
+  private static class ProcessedFiles {
+    final SummaryCollection summaries;
+    final Set<String> failedFiles;
+
+    public ProcessedFiles() {
+      this.summaries = new SummaryCollection();
+      this.failedFiles = new HashSet<>();
+    }
+
+    public ProcessedFiles(SummaryCollection summaries, SummarizerFactory factory) {
+      this();
+      this.summaries.merge(summaries, factory);
+    }
+
+    static ProcessedFiles merge(ProcessedFiles pf1, ProcessedFiles pf2, SummarizerFactory factory) {
+      ProcessedFiles ret = new ProcessedFiles();
+      ret.failedFiles.addAll(pf1.failedFiles);
+      ret.failedFiles.addAll(pf2.failedFiles);
+      ret.summaries.merge(pf1.summaries, factory);
+      ret.summaries.merge(pf2.summaries, factory);
+      return ret;
+    }
+  }
+
+  private class FilesProcessor implements Supplier<ProcessedFiles> {
+
+    HostAndPort location;
+    Map<String,List<TRowRange>> allFiles;
+    private TInfo tinfo;
+    private AtomicBoolean cancelFlag;
+
+    public FilesProcessor(TInfo tinfo, HostAndPort location, Map<String,List<TRowRange>> allFiles, AtomicBoolean cancelFlag) {
+      this.location = location;
+      this.allFiles = allFiles;
+      this.tinfo = tinfo;
+      this.cancelFlag = cancelFlag;
+    }
+
+    @Override
+    public ProcessedFiles get() {
+      ProcessedFiles pfiles = new ProcessedFiles();
+
+      Client client = null;
+      try {
+        client = ThriftUtil.getTServerClient(location, ctx);
+        // partition files into smaller chunks so that not too many are sent to a tserver at once
+        for (Map<String,List<TRowRange>> files : partition(allFiles, 500)) {
+          if (pfiles.failedFiles.size() > 0) {
+            // there was a previous failure on this tserver, so just fail the rest of the files
+            pfiles.failedFiles.addAll(files.keySet());
+            continue;
+          }
+
+          try {
+            TSummaries tSums = client.startGetSummariesFromFiles(tinfo, ctx.rpcCreds(), getRequest(), files);
+            while (!tSums.finished && !cancelFlag.get()) {
+              tSums = client.contiuneGetSummaries(tinfo, tSums.sessionId);
+            }
+
+            pfiles.summaries.merge(new SummaryCollection(tSums), factory);
+          } catch (TApplicationException tae) {
+            throw new RuntimeException(tae);
+          } catch (TTransportException e) {
+            pfiles.failedFiles.addAll(files.keySet());
+            continue;
+          } catch (TException e) {
+            throw new RuntimeException(e);
+          }
+
+        }
+      } catch (TTransportException e1) {
+        pfiles.failedFiles.addAll(allFiles.keySet());
+      } finally {
+        ThriftUtil.returnClient(client);
+      }
+
+      if (cancelFlag.get()) {
+        throw new RuntimeException("Operation canceled");
+      }
+
+      return pfiles;
+    }
+  }
+
+  private class PartitionFuture implements Future<SummaryCollection> {
+
+    private CompletableFuture<ProcessedFiles> future;
+    private int modulus;
+    private int remainder;
+    private ExecutorService execSrv;
+    private TInfo tinfo;
+    private AtomicBoolean cancelFlag = new AtomicBoolean(false);
+
+    PartitionFuture(TInfo tinfo, ExecutorService execSrv, int modulus, int remainder) {
+      this.tinfo = tinfo;
+      this.execSrv = execSrv;
+      this.modulus = modulus;
+      this.remainder = remainder;
+    }
+
+    private synchronized void initiateProcessing(ProcessedFiles previousWork) {
+      try {
+        Predicate<String> fileSelector = file -> Math.abs(Hashing.murmur3_32().hashString(file).asInt()) % modulus == remainder;
+        if (previousWork != null) {
+          fileSelector = fileSelector.and(file -> previousWork.failedFiles.contains(file));
+        }
+        Map<String,Map<String,List<TRowRange>>> filesGBL;
+        filesGBL = getFilesGroupedByLocation(fileSelector);
+
+        List<CompletableFuture<ProcessedFiles>> futures = new ArrayList<>();
+        if (previousWork != null) {
+          futures.add(CompletableFuture.completedFuture(new ProcessedFiles(previousWork.summaries, factory)));
+        }
+
+        for (Entry<String,Map<String,List<TRowRange>>> entry : filesGBL.entrySet()) {
+          HostAndPort location = HostAndPort.fromString(entry.getKey());
+          Map<String,List<TRowRange>> allFiles = entry.getValue();
+
+          futures.add(CompletableFuture.supplyAsync(new FilesProcessor(tinfo, location, allFiles, cancelFlag), execSrv));
+        }
+
+        future = CompletableFutureUtil.merge(futures, (pf1, pf2) -> ProcessedFiles.merge(pf1, pf2, factory), ProcessedFiles::new);
+
+        // when all processing is done, check for failed files... and if found starting processing again
+        future.thenRun(() -> updateFuture());
+      } catch (Exception e) {
+        future = CompletableFuture.completedFuture(new ProcessedFiles());
+        // force future to have this exception
+        future.obtrudeException(e);
+      }
+    }
+
+    private ProcessedFiles _get() {
+      try {
+        return future.get();
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new RuntimeException(e);
+      } catch (ExecutionException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    private synchronized CompletableFuture<ProcessedFiles> updateFuture() {
+      if (future.isDone()) {
+        if (!future.isCancelled() && !future.isCompletedExceptionally()) {
+          ProcessedFiles pf = _get();
+          if (pf.failedFiles.size() > 0) {
+            initiateProcessing(pf);
+          }
+        }
+      }
+
+      return future;
+    }
+
+    synchronized void initiateProcessing() {
+      Preconditions.checkState(future == null);
+      initiateProcessing(null);
+    }
+
+    @Override
+    public synchronized boolean cancel(boolean mayInterruptIfRunning) {
+      boolean canceled = future.cancel(mayInterruptIfRunning);
+      if (canceled) {
+        cancelFlag.set(true);
+      }
+      return canceled;
+    }
+
+    @Override
+    public synchronized boolean isCancelled() {
+      return future.isCancelled();
+    }
+
+    @Override
+    public synchronized boolean isDone() {
+      updateFuture();
+      if (future.isDone()) {
+        if (future.isCancelled() || future.isCompletedExceptionally()) {
+          return true;
+        }
+
+        ProcessedFiles pf = _get();
+        if (pf.failedFiles.size() == 0) {
+          return true;
+        } else {
+          updateFuture();
+        }
+      }
+
+      return false;
+    }
+
+    @Override
+    public SummaryCollection get() throws InterruptedException, ExecutionException {
+      CompletableFuture<ProcessedFiles> futureRef = updateFuture();
+      ProcessedFiles processedFiles = futureRef.get();
+      while (processedFiles.failedFiles.size() > 0) {
+        futureRef = updateFuture();
+        processedFiles = futureRef.get();
+      }
+      return processedFiles.summaries;
+    }
+
+    @Override
+    public SummaryCollection get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
+      long nanosLeft = unit.toNanos(timeout);
+      long t1, t2;
+      CompletableFuture<ProcessedFiles> futureRef = updateFuture();
+      t1 = System.nanoTime();
+      ProcessedFiles processedFiles = futureRef.get(Long.max(1, nanosLeft), TimeUnit.NANOSECONDS);
+      t2 = System.nanoTime();
+      nanosLeft -= (t2 - t1);
+      while (processedFiles.failedFiles.size() > 0) {
+        futureRef = updateFuture();
+        t1 = System.nanoTime();
+        processedFiles = futureRef.get(Long.max(1, nanosLeft), TimeUnit.NANOSECONDS);
+        t2 = System.nanoTime();
+        nanosLeft -= (t2 - t1);
+      }
+      return processedFiles.summaries;
+    }
+
+  }
+
+  /**
+   * This methods reads a subset of file paths into memory and groups them by location. Then it request sumaries for files from each location/tablet server.
+   */
+  public Future<SummaryCollection> processPartition(ExecutorService execSrv, int modulus, int remainder) {
+    PartitionFuture future = new PartitionFuture(Tracer.traceInfo(), execSrv, modulus, remainder);
+    future.initiateProcessing();
+    return future;
+  }
+
+  public static interface FileSystemResolver {
+    FileSystem get(Path file);
+  }
+
+  /**
+   * This method will read summaries from a set of files.
+   */
+  public Future<SummaryCollection> processFiles(FileSystemResolver volMgr, Map<String,List<TRowRange>> files, BlockCache summaryCache, BlockCache indexCache,
+      ExecutorService srp) {
+    List<CompletableFuture<SummaryCollection>> futures = new ArrayList<>();
+    for (Entry<String,List<TRowRange>> entry : files.entrySet()) {
+      futures.add(CompletableFuture.supplyAsync(() -> {
+        List<RowRange> rrl = Lists.transform(entry.getValue(), RowRange::new);
+        return getSummaries(volMgr, entry.getKey(), rrl, summaryCache, indexCache);
+      }, srp));
+    }
+
+    return CompletableFutureUtil.merge(futures, (sc1, sc2) -> SummaryCollection.merge(sc1, sc2, factory), SummaryCollection::new);
+  }
+
+  private int countFiles() throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    // TODO use a batch scanner + iterator to parallelize counting files
+    Iterable<TabletMetadata> tmi = MetadataScanner.builder().from(ctx).overUserTableId(tableId, startRow, endRow).fetchFiles().fetchPrev().build();
+    return (int) StreamSupport.stream(tmi.spliterator(), false).mapToInt(tm -> tm.getFiles().size()).sum();
+  }
+
+  private class GatherRequest implements Supplier<SummaryCollection> {
+
+    private int remainder;
+    private int modulus;
+    private TInfo tinfo;
+    private AtomicBoolean cancelFlag;
+
+    GatherRequest(TInfo tinfo, int remainder, int modulus, AtomicBoolean cancelFlag) {
+      this.remainder = remainder;
+      this.modulus = modulus;
+      this.tinfo = tinfo;
+      this.cancelFlag = cancelFlag;
+    }
+
+    @Override
+    public SummaryCollection get() {
+      TSummaryRequest req = getRequest();
+
+      TSummaries tSums;
+      try {
+        tSums = ServerClient.execute(ctx, new TabletClientService.Client.Factory(), client -> {
+          TSummaries tsr = client.startGetSummariesForPartition(tinfo, ctx.rpcCreds(), req, modulus, remainder);
+          while (!tsr.finished && !cancelFlag.get()) {
+            tsr = client.contiuneGetSummaries(tinfo, tsr.sessionId);
+          }
+          return tsr;
+        });
+      } catch (AccumuloException | AccumuloSecurityException e) {
+        throw new RuntimeException(e);
+      }
+
+      if (cancelFlag.get()) {
+        throw new RuntimeException("Operation canceled");
+      }
+
+      return new SummaryCollection(tSums);
+    }
+  }
+
+  public Future<SummaryCollection> gather(ExecutorService es) {
+    int numFiles;
+    try {
+      numFiles = countFiles();
+    } catch (TableNotFoundException | AccumuloException | AccumuloSecurityException e) {
+      throw new RuntimeException(e);
+    }
+
+    log.debug("Gathering summaries from {} files", numFiles);
+
+    if (numFiles == 0) {
+      return CompletableFuture.completedFuture(new SummaryCollection());
+    }
+
+    // have each tablet server process ~100K files
+    int numRequest = Math.max(numFiles / 100_000, 1);
+
+    List<CompletableFuture<SummaryCollection>> futures = new ArrayList<>();
+
+    AtomicBoolean cancelFlag = new AtomicBoolean(false);
+
+    TInfo tinfo = Tracer.traceInfo();
+    for (int i = 0; i < numRequest; i++) {
+      futures.add(CompletableFuture.supplyAsync(new GatherRequest(tinfo, i, numRequest, cancelFlag), es));
+    }
+
+    Future<SummaryCollection> future = CompletableFutureUtil.merge(futures, (sc1, sc2) -> SummaryCollection.merge(sc1, sc2, factory), SummaryCollection::new);
+    return new CancelFlagFuture<>(future, cancelFlag);
+  }
+
+  private static Text removeTrailingZeroFromRow(Key k) {
+    if (k != null) {
+      Text t = new Text();
+      ByteSequence row = k.getRowData();
+      Preconditions.checkArgument(row.length() >= 1 && row.byteAt(row.length() - 1) == 0);
+      t.set(row.getBackingArray(), row.offset(), row.length() - 1);
+      return t;
+    } else {
+      return null;
+    }
+  }
+
+  private RowRange toClippedExtent(Range r) {
+    r = clipRange.clip(r);
+
+    Text startRow = removeTrailingZeroFromRow(r.getStartKey());
+    Text endRow = removeTrailingZeroFromRow(r.getEndKey());
+
+    return new RowRange(startRow, endRow);
+  }
+
+  public static class RowRange {
+    private Text startRow;
+    private Text endRow;
+
+    public RowRange(KeyExtent ke) {
+      this.startRow = ke.getPrevEndRow();
+      this.endRow = ke.getEndRow();
+    }
+
+    public RowRange(TRowRange trr) {
+      this.startRow = ByteBufferUtil.toText(trr.startRow);
+      this.endRow = ByteBufferUtil.toText(trr.endRow);
+    }
+
+    public RowRange(Text startRow, Text endRow) {
+      this.startRow = startRow;
+      this.endRow = endRow;
+    }
+
+    public Range toRange() {
+      return new Range(startRow, false, endRow, true);
+    }
+
+    public TRowRange toThrift() {
+      return new TRowRange(TextUtil.getByteBuffer(startRow), TextUtil.getByteBuffer(endRow));
+    }
+
+    public Text getStartRow() {
+      return startRow;
+    }
+
+    public Text getEndRow() {
+      return endRow;
+    }
+
+    public String toString() {
+      return startRow + " " + endRow;
+    }
+  }
+
+  private SummaryCollection getSummaries(FileSystemResolver volMgr, String file, List<RowRange> ranges, BlockCache summaryCache, BlockCache indexCache) {
+    Path path = new Path(file);
+    Configuration conf = CachedConfiguration.getInstance();
+    return SummaryReader.load(volMgr.get(path), conf, ctx.getConfiguration(), factory, path, summarySelector, summaryCache, indexCache).getSummaries(ranges);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/summary/SummarizerConfigurationUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/SummarizerConfigurationUtil.java b/core/src/main/java/org/apache/accumulo/core/summary/SummarizerConfigurationUtil.java
new file mode 100644
index 0000000..a67b8c2
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummarizerConfigurationUtil.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.summary;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration.Builder;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.thrift.TSummarizerConfiguration;
+
+public class SummarizerConfigurationUtil {
+
+  public static Map<String,String> toTablePropertiesMap(List<SummarizerConfiguration> summarizers) {
+    if (summarizers.size() == 0) {
+      return Collections.emptyMap();
+    }
+
+    Map<String,String> props = new HashMap<>();
+
+    for (SummarizerConfiguration sconf : summarizers) {
+      String cid = sconf.getPropertyId();
+      String prefix = Property.TABLE_SUMMARIZER_PREFIX.getKey() + cid;
+
+      if (props.containsKey(prefix)) {
+        throw new IllegalArgumentException("Duplicate summarizer config id : " + cid);
+      }
+
+      props.put(prefix, sconf.getClassName());
+      Set<Entry<String,String>> es = sconf.getOptions().entrySet();
+      StringBuilder sb = new StringBuilder(prefix + ".opt.");
+      int resetLen = sb.length();
+      for (Entry<String,String> entry : es) {
+        sb.append(entry.getKey());
+        props.put(sb.toString(), entry.getValue());
+        sb.setLength(resetLen);
+      }
+    }
+
+    return props;
+  }
+
+  public static List<SummarizerConfiguration> getSummarizerConfigs(Iterable<Entry<String,String>> props) {
+    TreeMap<String,String> filteredMap = new TreeMap<>();
+    for (Entry<String,String> entry : props) {
+      if (entry.getKey().startsWith(Property.TABLE_SUMMARIZER_PREFIX.getKey())) {
+        filteredMap.put(entry.getKey(), entry.getValue());
+      }
+    }
+
+    return getSummarizerConfigsFiltered(filteredMap);
+  }
+
+  public static List<SummarizerConfiguration> getSummarizerConfigs(AccumuloConfiguration aconf) {
+    Map<String,String> sprops = aconf.getAllPropertiesWithPrefix(Property.TABLE_SUMMARIZER_PREFIX);
+    return getSummarizerConfigsFiltered(new TreeMap<>(sprops));
+  }
+
+  private static List<SummarizerConfiguration> getSummarizerConfigsFiltered(SortedMap<String,String> sprops) {
+    if (sprops.size() == 0) {
+      return Collections.emptyList();
+    }
+
+    SummarizerConfiguration.Builder builder = null;
+
+    List<SummarizerConfiguration> configs = new ArrayList<>();
+
+    final int preLen = Property.TABLE_SUMMARIZER_PREFIX.getKey().length();
+    for (Entry<String,String> entry : sprops.entrySet()) {
+      String k = entry.getKey().substring(preLen);
+
+      String[] tokens = k.split("\\.");
+
+      String id = tokens[0];
+      if (tokens.length == 1) {
+        if (builder != null) {
+          configs.add(builder.build());
+        }
+
+        builder = SummarizerConfiguration.builder(entry.getValue()).setPropertyId(id);
+
+      } else if (tokens.length == 3 || tokens[1].equals("opt")) {
+        builder.addOption(tokens[2], entry.getValue());
+      } else {
+        throw new IllegalArgumentException("Unable to parse summarizer property : " + k);
+      }
+    }
+
+    configs.add(builder.build());
+
+    return configs;
+  }
+
+  public static TSummarizerConfiguration toThrift(SummarizerConfiguration sc) {
+    return new TSummarizerConfiguration(sc.getClassName(), sc.getOptions(), sc.getPropertyId());
+  }
+
+  public static SummarizerConfiguration fromThrift(TSummarizerConfiguration config) {
+    Builder builder = SummarizerConfiguration.builder(config.getClassname());
+    builder.setPropertyId(config.getConfigId());
+    builder.addOptions(config.getOptions());
+    return builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/summary/SummarizerFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/SummarizerFactory.java b/core/src/main/java/org/apache/accumulo/core/summary/SummarizerFactory.java
new file mode 100644
index 0000000..bba41b5
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummarizerFactory.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.summary;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+
+public class SummarizerFactory {
+  private ClassLoader classloader;
+  private String context;
+
+  public SummarizerFactory() {
+    this.classloader = SummarizerFactory.class.getClassLoader();
+  }
+
+  public SummarizerFactory(ClassLoader classloader) {
+    this.classloader = classloader;
+  }
+
+  public SummarizerFactory(AccumuloConfiguration tableConfig) {
+    this.context = tableConfig.get(Property.TABLE_CLASSPATH);
+  }
+
+  private Summarizer newSummarizer(String classname) throws ClassNotFoundException, IOException, InstantiationException, IllegalAccessException {
+    if (classloader != null) {
+      return classloader.loadClass(classname).asSubclass(Summarizer.class).newInstance();
+    } else {
+      if (context != null && !context.equals(""))
+        return AccumuloVFSClassLoader.getContextManager().loadClass(context, classname, Summarizer.class).newInstance();
+      else
+        return AccumuloVFSClassLoader.loadClass(classname, Summarizer.class).newInstance();
+    }
+  }
+
+  public Summarizer getSummarizer(SummarizerConfiguration conf) {
+    try {
+      Summarizer summarizer = newSummarizer(conf.getClassName());
+      return summarizer;
+    } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}


[8/9] accumulo git commit: ACCUMULO-4501 ACCUMULO-96 Added Summarization

Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/summary/Summarizer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/summary/Summarizer.java b/core/src/main/java/org/apache/accumulo/core/client/summary/Summarizer.java
new file mode 100644
index 0000000..fdb194b
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/summary/Summarizer.java
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.client.summary;
+
+import java.util.Map;
+
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.rfile.RFile;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+//checkstyle and the formatter are in conflict, so turn off the formatter
+//@formatter:off
+/**
+ * <p>
+ * Instances of this interface can be configured for Accumulo tables. When Accumulo compacts files, it will use this Factory to create {@link Collector} and
+ * {@link Combiner} objects to generate summary information about the data in the file.
+ *
+ * <p>
+ * In order to merge summary information from multiple files, Accumulo will use this factory to create a {@link Combiner} object.
+ *
+ * <p>
+ * Below is an example of a very simple summarizer that will compute the number of deletes, total number of keys, min timestamp and max timestamp.
+ *
+ * <pre>
+ * <code>
+ *   public class BasicSummarizer implements Summarizer {
+ *
+ *     public static final String DELETES_STAT = &quot;deletes&quot;;
+ *     public static final String MIN_STAMP_STAT = &quot;minStamp&quot;;
+ *     public static final String MAX_STAMP_STAT = &quot;maxStamp&quot;;
+ *     public static final String TOTAL_STAT = &quot;total&quot;;
+ *
+ *     &#064;Override
+ *     public Collector collector(SummarizerConfiguration sc) {
+ *       return new Collector() {
+ *
+ *         private long minStamp = Long.MAX_VALUE;
+ *         private long maxStamp = Long.MIN_VALUE;
+ *         private long deletes = 0;
+ *         private long total = 0;
+ *
+ *         &#064;Override
+ *         public void accept(Key k, Value v) {
+ *           if (k.getTimestamp() &lt; minStamp) {
+ *             minStamp = k.getTimestamp();
+ *           }
+ *
+ *           if (k.getTimestamp() &gt; maxStamp) {
+ *             maxStamp = k.getTimestamp();
+ *           }
+ *
+ *           if (k.isDeleted()) {
+ *             deletes++;
+ *           }
+ *
+ *           total++;
+ *         }
+ *
+ *         &#064;Override
+ *         public void summarize(StatisticConsumer sc) {
+ *           sc.accept(MIN_STAMP_STAT, minStamp);
+ *           sc.accept(MAX_STAMP_STAT, maxStamp);
+ *           sc.accept(DELETES_STAT, deletes);
+ *           sc.accept(TOTAL_STAT, total);
+ *         }
+ *       };
+ *     }
+ *
+ *     &#064;Override
+ *     public Combiner combiner(SummarizerConfiguration sc) {
+ *       return (stats1, stats2) -&gt; {
+ *         stats1.merge(DELETES_STAT, stats2.get(DELETES_STAT), Long::sum);
+ *         stats1.merge(TOTAL_STAT, stats2.get(TOTAL_STAT), Long::sum);
+ *         stats1.merge(MIN_STAMP_STAT, stats2.get(MIN_STAMP_STAT), Long::min);
+ *         stats1.merge(MAX_STAMP_STAT, stats2.get(MAX_STAMP_STAT), Long::max);
+ *       };
+ *     }
+ *   }
+ * </code>
+ * </pre>
+ *
+ * <p>
+ * Below is an example summarizer that counts the log of the value length.
+ *
+ * <pre>
+ * <code>
+ * public class ValueLogLengthSummarizer implements Summarizer {
+ *
+ *  &#64;Override
+ *  public Collector collector(SummarizerConfiguration sc) {
+ *
+ *    return new Collector(){
+ *
+ *      long[] counts = new long[32];
+ *
+ *      &#64;Override
+ *      public void accept(Key k, Value v) {
+ *        int idx;
+ *        if(v.getSize() == 0)
+ *          idx = 0;
+ *        else
+ *          idx = IntMath.log2(v.getSize(), RoundingMode.UP);  //IntMath is from Guava
+ *
+ *        counts[idx]++;
+ *      }
+ *
+ *      &#64;Override
+ *      public void summarize(StatisticConsumer sc) {
+ *        for (int i = 0; i &lt; counts.length; i++) {
+ *          if(counts[i] &gt; 0) {
+ *            sc.accept(""+(1&lt;&lt;i), counts[i]);
+ *          }
+ *        }
+ *      }
+ *    };
+ *  }
+ *
+ *  &#64;Override
+ *  public Combiner combiner(SummarizerConfiguration sc) {
+ *    return (m1, m2) -&gt; m2.forEach((k,v) -&gt; m1.merge(k, v, Long::sum));
+ *  }
+ * }
+ * </code>
+ * </pre>
+ *
+ * <p>
+ * The reason a Summarizer is a factory for a Collector and Combiner is to make it very clear in the API that Accumulo uses them independently at different
+ * times. Therefore its not advisable to share internal state between the Collector and Combiner. The example implementation shows that the Collectors design
+ * allows for very efficient collection of specialized summary information. Creating {@link String} + {@link Long} pairs is deferred until the summarize method
+ * is called.
+ *
+ * <p>
+ * Summary data can be used by Compaction Strategies to decide which files to compact.
+ *
+ * <p>
+ * Summary data is persisted, so ideally the same summarizer class with the same options should always produce the same results.  If you need to change the behavior
+ * of a summarizer, then consider doing this by adding a new option.  If the same summarizer is configured twice with different options, then Accumulo will store and
+ * merge each one separately.  This can allow old and new behavior to coexists simultaneously.
+ *
+ * @since 2.0.0
+ *
+ * @see TableOperations#summaries(String)
+ * @see TableOperations#addSummarizers(String, SummarizerConfiguration...)
+ * @see TableOperations#listSummarizers(String)
+ * @see TableOperations#removeSummarizers(String, java.util.function.Predicate)
+ * @see RFile#summaries()
+ * @see SummarizerConfiguration
+ */
+ //@formatter:on
+public interface Summarizer {
+
+  public static interface StatisticConsumer {
+    public void accept(String statistic, long value);
+  }
+
+  /**
+   * When Accumulo calls methods in this interface, it will call {@link #accept(Key, Value)} zero or more times and then call
+   * {@link #summarize(Summarizer.StatisticConsumer)} once. After calling {@link #summarize(Summarizer.StatisticConsumer)}, it will not use the collector again.
+   *
+   * @since 2.0.0
+   */
+  public static interface Collector {
+    /**
+     * During compactions, Accumulo passes each Key Value written to the file to this method.
+     */
+    void accept(Key k, Value v);
+
+    /**
+     * After Accumulo has written some Key Values, it will call this method to generate some statistics about what was previously passed to
+     * {@link #accept(Key, Value)}.
+     *
+     * <p>
+     * In order for summary data to be useful for decision making about data, it needs to be quickly accessible. In order to be quickly accessible, it needs to
+     * fit in the tablet server cache as described in {@link TableOperations#summaries(String)} and the compaction strategy documentation. Therefore its
+     * advisable to generate small summaries. If the summary data generated is too large it will not be stored. The maximum summary size is set using the per
+     * table property {@code table.file.summary.maxSize}. The number of files that exceeded the summary size is reported by
+     * {@link Summary.FileStatistics#getLarge()}.
+     *
+     * @param sc
+     *          Emit statistics to this Object.
+     */
+    public void summarize(StatisticConsumer sc);
+  }
+
+  /**
+   * A Combiner is used to merge statistics emitted from {@link Collector#summarize(StatisticConsumer)} and from previous invocations of itself.
+   *
+   * @since 2.0.0
+   */
+  public static interface Combiner {
+    /**
+     * This method should merge the statistics in the second map into the first map. Both maps may have statistics produced by a {@link Collector} or previous
+     * calls to this method.
+     *
+     * <p>
+     * If first map is too large after this call, then it may not be stored. See the comment on {@link Collector#summarize(StatisticConsumer)}
+     */
+    public void merge(Map<String,Long> statistics1, Map<String,Long> statistics2);
+  }
+
+  /**
+   * Factory method that creates a {@link Collector} based on configuration. Each {@link Collector} created by this method should be independent and have its
+   * own internal state. Accumulo uses a Collector to generate summary statistics about a sequence of key values written to a file.
+   */
+  public Collector collector(SummarizerConfiguration sc);
+
+  /**
+   * Factory method that creates a {@link Combiner}. Accumulo will only use the created Combiner to merge data from {@link Collector}s created using the same
+   * {@link SummarizerConfiguration}.
+   */
+  public Combiner combiner(SummarizerConfiguration sc);
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/summary/SummarizerConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/summary/SummarizerConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/summary/SummarizerConfiguration.java
new file mode 100644
index 0000000..ec98695
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/summary/SummarizerConfiguration.java
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.summary;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.summary.SummarizerConfigurationUtil;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+
+/**
+ * This class encapsulates the configuration needed to instantiate a {@link Summarizer}. It also provides methods and documentation for setting the table
+ * properties that configure a Summarizer.
+ *
+ * @since 2.0.0
+ */
+public class SummarizerConfiguration {
+
+  private final String className;
+  private final Map<String,String> options;
+  private int hashCode = 0;
+  private final String configId;
+
+  private SummarizerConfiguration(String className, String configId, Map<String,String> options) {
+    this.className = className;
+    this.options = ImmutableMap.copyOf(options);
+
+    if (configId == null) {
+      ArrayList<String> keys = new ArrayList<>(this.options.keySet());
+      Collections.sort(keys);
+      Hasher hasher = Hashing.murmur3_32().newHasher();
+      hasher.putString(className);
+      for (String key : keys) {
+        hasher.putString(key);
+        hasher.putString(options.get(key));
+      }
+
+      this.configId = hasher.hash().toString();
+    } else {
+      this.configId = configId;
+    }
+  }
+
+  /**
+   * @return the name of a class that implements @link {@link Summarizer}.
+   */
+  public String getClassName() {
+    return className;
+  }
+
+  /**
+   * @return custom options for a {link @Summarizer}
+   */
+  public Map<String,String> getOptions() {
+    return options;
+  }
+
+  /**
+   * The propertyId is used to when creating table properties for a summarizer. Its not used for equality or hashCode for this class.
+   */
+  public String getPropertyId() {
+    return configId;
+  }
+
+  @Override
+  public String toString() {
+    return className + " " + configId + " " + options;
+  }
+
+  /**
+   * Compares the classname and options to determine equality.
+   */
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof SummarizerConfiguration) {
+      SummarizerConfiguration osc = (SummarizerConfiguration) o;
+      return className.equals(osc.className) && options.equals(osc.options);
+    }
+
+    return false;
+  }
+
+  /**
+   * Hashes the classname and options to create a hashcode.
+   */
+  @Override
+  public int hashCode() {
+    if (hashCode == 0) {
+      hashCode = 31 * options.hashCode() + className.hashCode();
+    }
+    return hashCode;
+  }
+
+  /**
+   * Converts this configuration to Accumulo per table properties. The returned map has the following key values. The {@code <configId>} below is from
+   * {@link #getPropertyId()}. The {@code <optionKey>} and {@code <optionValue>} below are derived from the key values of {@link #getOptions()}.
+   *
+   * <pre>
+   * {@code
+   *   table.summarizer.<configId>=<classname>
+   *   table.summarizer.<configId>.opt.<optionKey1>=<optionValue1>
+   *   table.summarizer.<configId>.opt.<optionKey2>=<optionValue2>
+   *      .
+   *      .
+   *      .
+   *   table.summarizer.<configId>.opt.<optionKeyN>=<optionValueN>
+   * }
+   * </pre>
+   */
+  public Map<String,String> toTableProperties() {
+    return SummarizerConfigurationUtil.toTablePropertiesMap(Collections.singletonList(this));
+  }
+
+  /**
+   * Encodes each configuration in the same way as {@link #toTableProperties()}.
+   *
+   * @throws IllegalArgumentException
+   *           when there are duplicate values for {@link #getPropertyId()}
+   */
+  public static Map<String,String> toTableProperties(SummarizerConfiguration... configurations) {
+    return SummarizerConfigurationUtil.toTablePropertiesMap(Arrays.asList(configurations));
+  }
+
+  /**
+   * Encodes each configuration in the same way as {@link #toTableProperties()}.
+   *
+   * @throws IllegalArgumentException
+   *           when there are duplicate values for {@link #getPropertyId()}
+   */
+  public static Map<String,String> toTableProperties(Collection<SummarizerConfiguration> configurations) {
+    return SummarizerConfigurationUtil.toTablePropertiesMap(new ArrayList<SummarizerConfiguration>(configurations));
+  }
+
+  /**
+   * Decodes table properties with the prefix {@code table.summarizer} into {@link SummarizerConfiguration} objects. Table properties with prefixes other than
+   * {@code table.summarizer} are ignored.
+   */
+  public static Collection<SummarizerConfiguration> fromTableProperties(Map<String,String> props) {
+    return fromTableProperties(props.entrySet());
+  }
+
+  /**
+   * @see #fromTableProperties(Map)
+   */
+  public static Collection<SummarizerConfiguration> fromTableProperties(Iterable<Entry<String,String>> props) {
+    return SummarizerConfigurationUtil.getSummarizerConfigs(props);
+  }
+
+  public static class Builder {
+    private String className;
+    private ImmutableMap.Builder<String,String> imBuilder;
+    private String configId = null;
+
+    private Builder(String className) {
+      this.className = className;
+      this.imBuilder = ImmutableMap.builder();
+    }
+
+    /**
+     * Sets the id used when generating table properties. Setting this is optional. If not set, an id is generated using hashing that will likely be unique.
+     *
+     * @param propId
+     *          This id is used when converting a {@link SummarizerConfiguration} to table properties. Since tables can have multiple summarizers, make sure its
+     *          unique.
+     *
+     * @see SummarizerConfiguration#toTableProperties()
+     */
+    public Builder setPropertyId(String propId) {
+      Preconditions.checkArgument(propId.matches("\\w+"), "Config Id %s is not alphanum", propId);
+      this.configId = propId;
+      return this;
+    }
+
+    /**
+     * Adds an option that Summarizers can use when constructing Collectors and Combiners.
+     *
+     * @return this
+     *
+     * @see SummarizerConfiguration#getOptions()
+     */
+    public Builder addOption(String key, String value) {
+      Preconditions.checkArgument(key.matches("\\w+"), "Option Id %s is not alphanum", key);
+      imBuilder.put(key, value);
+      return this;
+    }
+
+    /**
+     * Adds an option that Summarizers can use when constructing Collectors and Combiners.
+     *
+     * @return this
+     *
+     * @see SummarizerConfiguration#getOptions()
+     */
+    public Builder addOption(String key, long value) {
+      return addOption(key, Long.toString(value));
+    }
+
+    /**
+     * Convenience method for adding multiple options. The following
+     *
+     * <pre>
+     * {@code builder.addOptions("opt1","val1","opt2","val2","opt3","val3")}
+     * </pre>
+     *
+     * <p>
+     * is equivalent to
+     *
+     * <pre>
+     * {@code
+     *   builder.addOption("opt1","val1");
+     *   builder.addOption("opt2","val2");
+     *   builder.addOption("opt3","val3");
+     * }
+     * </pre>
+     *
+     * @param keyValuePairs
+     *          This array must have an even and positive number of elements.
+     * @return this
+     * @see SummarizerConfiguration#getOptions()
+     */
+    public Builder addOptions(String... keyValuePairs) {
+      Preconditions.checkArgument(keyValuePairs.length % 2 == 0 && keyValuePairs.length > 0, "Require an even, positive number of arguments, got %s",
+          keyValuePairs.length);
+      for (int i = 0; i < keyValuePairs.length; i += 2) {
+        addOption(keyValuePairs[i], keyValuePairs[i + 1]);
+      }
+      return this;
+    }
+
+    /**
+     * @param options
+     *          Each entry in the map is passed to {@link #addOption(String, String)}
+     * @return this
+     *
+     * @see SummarizerConfiguration#getOptions()
+     */
+    public Builder addOptions(Map<String,String> options) {
+      options.entrySet().forEach(e -> addOption(e.getKey(), e.getValue()));
+      return this;
+    }
+
+    public SummarizerConfiguration build() {
+      return new SummarizerConfiguration(className, configId, imBuilder.build());
+    }
+  }
+
+  /**
+   * Call this method to initiate a chain of fluent method calls to a create an immutable {@link SummarizerConfiguration}
+   *
+   * @param className
+   *          The fully qualified name of a class that implements {@link Summarizer}.
+   */
+  public static Builder builder(String className) {
+    return new Builder(className);
+  }
+
+  /**
+   * @see #builder(String)
+   */
+  public static Builder builder(Class<? extends Summarizer> clazz) {
+    return new Builder(clazz.getName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/summary/Summary.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/summary/Summary.java b/core/src/main/java/org/apache/accumulo/core/client/summary/Summary.java
new file mode 100644
index 0000000..8a6a9aa
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/summary/Summary.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.client.summary;
+
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * This class encapsulates summary statistics, information about how those statistics were generated, and information about files the statistics were obtained
+ * from.
+ *
+ * @see Summarizer
+ * @since 2.0.0
+ */
+public class Summary {
+
+  /**
+   * @since 2.0.0
+   */
+  public static class FileStatistics {
+    private final long total;
+    private final long missing;
+    private final long extra;
+    private final long large;
+    private final long deleted;
+
+    private FileStatistics(long total, long missing, long extra, long large, long deleted) {
+      this.total = total;
+      this.missing = missing;
+      this.extra = extra;
+      this.large = large;
+      this.deleted = deleted;
+    }
+
+    /**
+     * @return The total number of files from which summary information was obtained.
+     */
+    public long getTotal() {
+      return total;
+    }
+
+    /**
+     * @return The number of files that did not contain the requested summary information. When this is non-zero, it means that summary counts may be
+     *         incomplete. In the Accumulo shell, the compact command has a -{@code -sf-no-summary} option to compact files missing summary information. The
+     *         compaction will create the summary information. This could be done over a range of the table to avoid doing the entire table at once.
+     */
+    public long getMissing() {
+      return missing;
+    }
+
+    /**
+     * @return The number of files that had summary information outside of a tablet or query range boundaries. When this is non-zero, it means that summary
+     *         counts may be artificially inflated or contain extraneous information. In the Accumulo shell, the compact command has a -{@code -sf-extra-summary}
+     *         option to compact files with extra summary information.
+     */
+    public long getExtra() {
+      return extra;
+    }
+
+    /**
+     * @return The number of files that an attempt was made to generate summaries, but the summarizer generated a summary that was larger than the configured
+     *         maximum. For these files no summary statistics are stored. Only the fact that summarization was attempted and failed is stored.
+     * @see Summarizer.Collector#summarize(org.apache.accumulo.core.client.summary.Summarizer.StatisticConsumer)
+     */
+    public long getLarge() {
+      return large;
+    }
+
+    /**
+     * @return The number of files that were deleted after the summary retrieval operations started. This is a rare race condition where a compaction causes a
+     *         file to be deleted while retrieving summaries. When this happens, the file that replaced the deleted file can not be used because it may contain
+     *         duplication summary information for other files. Avoiding this race condition would be expensive, so reporting it was chosen. If this condition
+     *         must be avoided, then compactions must be stopped. Compactions could be stopped on a cloned table to avoid this.
+     */
+    public long getDeleted() {
+      return deleted;
+    }
+
+    /**
+     * @return The total number of files that had some kind of issue which would cause summary statistics to be inaccurate. This is the sum of
+     *         {@link #getMissing()}, {@link #getExtra()}, {{@link #getLarge()}, and {@link #getDeleted()}.
+     */
+    public long getInaccurate() {
+      return getMissing() + getExtra() + getLarge() + getDeleted();
+    }
+
+    @Override
+    public String toString() {
+      return String.format("[total:%,d, missing:%,d, extra:%,d, large:%,d, deleted:%,d]", total, missing, extra, large, deleted);
+    }
+  }
+
+  private final ImmutableMap<String,Long> statistics;
+  private final SummarizerConfiguration config;
+  private final FileStatistics fileStats;
+
+  public Summary(Map<String,Long> summary, SummarizerConfiguration config, long totalFiles, long filesMissingSummary, long filesWithExtra, long filesWithLarge,
+      long deletedFiles) {
+    this.statistics = ImmutableMap.copyOf(summary);
+    this.config = config;
+    this.fileStats = new FileStatistics(totalFiles, filesMissingSummary, filesWithExtra, filesWithLarge, deletedFiles);
+  }
+
+  /**
+   * @return Statistics about the files from which summary statistics were obtained.
+   */
+  public FileStatistics getFileStatistics() {
+    return fileStats;
+  }
+
+  /**
+   * @return The configuration used to generate and combine the summary statistics
+   */
+  public SummarizerConfiguration getSummarizerConfiguration() {
+    return config;
+  }
+
+  /**
+   * @return An immutable map of the statistics that were generated and merged by the specified {@link Summarizer}.
+   */
+  public Map<String,Long> getStatistics() {
+    return statistics;
+  }
+
+  @Override
+  public String toString() {
+    return "config : " + config + " filestats : " + fileStats + " statistics : " + statistics;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/DeletesSummarizer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/DeletesSummarizer.java b/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/DeletesSummarizer.java
new file mode 100644
index 0000000..1e94298
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/DeletesSummarizer.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.client.summary.summarizers;
+
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+
+/**
+ * This summarizer tracks the total number of delete Keys seen and the total number of keys seen.
+ *
+ * <p>
+ * This summarizer is used by org.apache.accumulo.tserver.compaction.strategies.TooManyDeletesCompactionStrategy to make compaction decisions based on the
+ * number of deletes.
+ *
+ * @since 2.0.0
+ * @see TableOperations#addSummarizers(String, org.apache.accumulo.core.client.summary.SummarizerConfiguration...)
+ */
+public class DeletesSummarizer implements Summarizer {
+
+  /**
+   * The name of the statistics for the number of deletes.
+   */
+  public static final String DELETES_STAT = "deletes";
+
+  /**
+   * The name of the statistics for the total number of keys.
+   */
+  public static final String TOTAL_STAT = "total";
+
+  @Override
+  public Collector collector(SummarizerConfiguration sc) {
+    return new Collector() {
+
+      long total = 0;
+      long deletes = 0;
+
+      @Override
+      public void accept(Key k, Value v) {
+        total++;
+        if (k.isDeleted()) {
+          deletes++;
+        }
+      }
+
+      @Override
+      public void summarize(StatisticConsumer sc) {
+        sc.accept(DELETES_STAT, deletes);
+        sc.accept(TOTAL_STAT, total);
+      }
+    };
+  }
+
+  @Override
+  public Combiner combiner(SummarizerConfiguration sc) {
+    return (m1, m2) -> m2.forEach((k, v) -> m1.merge(k, v, Long::sum));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/FamilySummarizer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/FamilySummarizer.java b/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/FamilySummarizer.java
new file mode 100644
index 0000000..9452530
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/FamilySummarizer.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.summary.summarizers;
+
+import java.util.function.UnaryOperator;
+
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.summary.CountingSummarizer;
+import org.apache.accumulo.core.data.ArrayByteSequence;
+import org.apache.accumulo.core.data.ByteSequence;
+
+/**
+ * Counts column column families. Leverages super class to defend against too many. This class is useful for discovering what column families are present when
+ * the expected number of families is small.
+ *
+ * @since 2.0.0
+ *
+ * @see TableOperations#addSummarizers(String, org.apache.accumulo.core.client.summary.SummarizerConfiguration...)
+ * @see TableOperations#summaries(String)
+ */
+public class FamilySummarizer extends CountingSummarizer<ByteSequence> {
+
+  @Override
+  protected UnaryOperator<ByteSequence> copier() {
+    return ArrayByteSequence::new;
+  }
+
+  @Override
+  protected Converter<ByteSequence> converter() {
+    return (k, v, c) -> c.accept(k.getColumnFamilyData());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/VisibilitySummarizer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/VisibilitySummarizer.java b/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/VisibilitySummarizer.java
new file mode 100644
index 0000000..c8f76d0
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/VisibilitySummarizer.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.client.summary.summarizers;
+
+import java.util.function.UnaryOperator;
+
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.summary.CountingSummarizer;
+import org.apache.accumulo.core.data.ArrayByteSequence;
+import org.apache.accumulo.core.data.ByteSequence;
+
+/**
+ * Counts column visibility labels. Leverages super class to defend against too many. This class is useful for discovering what column visibilities are present
+ * when the expected number of visibilities is small.
+ *
+ * @since 2.0.0
+ *
+ * @see TableOperations#addSummarizers(String, org.apache.accumulo.core.client.summary.SummarizerConfiguration...)
+ * @see TableOperations#summaries(String)
+ */
+public class VisibilitySummarizer extends CountingSummarizer<ByteSequence> {
+
+  @Override
+  protected UnaryOperator<ByteSequence> copier() {
+    return ArrayByteSequence::new;
+  }
+
+  @Override
+  protected Converter<ByteSequence> converter() {
+    return (k, v, c) -> c.accept(k.getColumnVisibilityData());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/compaction/CompactionSettings.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/compaction/CompactionSettings.java b/core/src/main/java/org/apache/accumulo/core/compaction/CompactionSettings.java
index 1c5369e..8e65f1c 100644
--- a/core/src/main/java/org/apache/accumulo/core/compaction/CompactionSettings.java
+++ b/core/src/main/java/org/apache/accumulo/core/compaction/CompactionSettings.java
@@ -21,6 +21,8 @@ import java.util.Map;
 
 public enum CompactionSettings {
 
+  SF_NO_SUMMARY(new NullType()),
+  SF_EXTRA_SUMMARY(new NullType()),
   SF_NO_SAMPLE(new NullType()),
   SF_GT_ESIZE_OPT(new SizeType()),
   SF_LT_ESIZE_OPT(new SizeType()),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 6ff2bed..7298db1 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -244,6 +244,7 @@ public enum Property {
   TSERV_CACHE_POLICY("tserver.cache.policy", "LRU", PropertyType.STRING, "Specifies the eviction policy of the file data caches (LRU or TinyLFU)."),
   TSERV_DATACACHE_SIZE("tserver.cache.data.size", "128M", PropertyType.MEMORY, "Specifies the size of the cache for file data blocks."),
   TSERV_INDEXCACHE_SIZE("tserver.cache.index.size", "512M", PropertyType.MEMORY, "Specifies the size of the cache for file indices."),
+  TSERV_SUMMARYCACHE_SIZE("tserver.cache.summary.size", "128M", PropertyType.MEMORY, "Specifies the size of the cache for summary data on each tablet server."),
   TSERV_PORTSEARCH("tserver.port.search", "false", PropertyType.BOOLEAN, "if the ports above are in use, search higher ports until one is available"),
   TSERV_CLIENTPORT("tserver.port.client", "9997", PropertyType.PORT, "The port used for handling client connections on the tablet servers"),
   @Deprecated
@@ -359,6 +360,14 @@ public enum Property {
       "The number of threads available to load tablets. Recoveries are still performed serially."),
   TSERV_SLOW_FLUSH_MILLIS("tserver.slow.flush.time", "100ms", PropertyType.TIMEDURATION,
       "If a flush to the write-ahead log takes longer than this period of time, debugging information will written, and may result in a log rollover."),
+  TSERV_SUMMARY_PARTITION_THREADS("tserver.summary.partition.threads", "10", PropertyType.COUNT,
+      "Summary data must be retrieved from files.  For a large number of files, the files are broken into partitions of 100K files.  This setting determines "
+          + "how many of these groups of 100K files will be processed concurrently."),
+  TSERV_SUMMARY_REMOTE_THREADS("tserver.summary.remote.threads", "128", PropertyType.COUNT,
+      "For a partitioned group of 100K files, those files are grouped by tablet server.  Then a remote tablet server is asked to gather summary data.  This "
+          + "setting determines how many concurrent request are made per partition."),
+  TSERV_SUMMARY_RETRIEVAL_THREADS("tserver.summary.retrieval.threads", "10", PropertyType.COUNT,
+      "The number of threads on each tablet server available to retrieve summary data, that is not currently in cache, from RFiles."),
 
   // accumulo garbage collector properties
   GC_PREFIX("gc.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of the accumulo garbage collector."),
@@ -461,6 +470,9 @@ public enum Property {
       "Determines the max # of files each tablet in a table can have. When adjusting this property you may want to consider adjusting"
           + " table.compaction.major.ratio also. Setting this property to 0 will make it default to tserver.scan.files.open.max-1, this will prevent a"
           + " tablet from having more files than can be opened. Setting this property low may throttle ingest and increase query performance."),
+  TABLE_FILE_SUMMARY_MAX_SIZE("table.file.summary.maxSize", "256K", PropertyType.MEMORY, "The maximum size summary that will be stored. The number of"
+      + " files that had summary data exceeding this threshold is reported by Summary.getFileStatistics().getLarge().  When adjusting this"
+      + " consider the expected number files with summaries on each tablet server and the summary cache size."),
   @Deprecated
   TABLE_WALOG_ENABLED("table.walog.enabled", "true", PropertyType.BOOLEAN, "This setting is deprecated.  Use table.durability=none instead."),
   TABLE_BLOOM_ENABLED("table.bloom.enabled", "false", PropertyType.BOOLEAN, "Use bloom filters on this table."),
@@ -547,6 +559,13 @@ public enum Property {
   TABLE_SUSPEND_DURATION("table.suspend.duration", "0s", PropertyType.TIMEDURATION,
       "For tablets belonging to this table: When a tablet server dies, allow the tablet server this duration to revive before reassigning its tablets"
           + "to other tablet servers."),
+  TABLE_SUMMARIZER_PREFIX(
+      "table.summarizer.",
+      null,
+      PropertyType.PREFIX,
+      "Prefix for configuring summarizers for a table.  Using this prefix multiple summarizers can be configured with options for each one. Each summarizer configured "
+          + "should have a unique id, this id can be anything. To add a summarizer set table.summarizer.<unique id>=<summarizer class name>.  If the summarizer has options, "
+          + "then for each option set table.summarizer.<unique id>.opt.<key>=<value>."),
 
   // VFS ClassLoader properties
   VFS_CLASSLOADER_SYSTEM_CLASSPATH_PROPERTY(AccumuloVFSClassLoader.VFS_CLASSLOADER_SYSTEM_CLASSPATH_PROPERTY, "", PropertyType.STRING,
@@ -814,7 +833,8 @@ public enum Property {
     return validTableProperties.contains(key) || key.startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey())
         || key.startsWith(Property.TABLE_ITERATOR_PREFIX.getKey()) || key.startsWith(Property.TABLE_LOCALITY_GROUP_PREFIX.getKey())
         || key.startsWith(Property.TABLE_COMPACTION_STRATEGY_PREFIX.getKey()) || key.startsWith(Property.TABLE_REPLICATION_TARGET.getKey())
-        || key.startsWith(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey()) || key.startsWith(TABLE_SAMPLER_OPTS.getKey());
+        || key.startsWith(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey()) || key.startsWith(TABLE_SAMPLER_OPTS.getKey())
+        || key.startsWith(TABLE_SUMMARIZER_PREFIX.getKey());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
index 2c458f0..9726090 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.core.conf;
 import static java.util.Objects.requireNonNull;
 
 import java.util.Arrays;
+import java.util.Objects;
 import java.util.function.Function;
 import java.util.function.Predicate;
 import java.util.regex.Matcher;
@@ -29,6 +30,8 @@ import org.apache.accumulo.core.util.Pair;
 import org.apache.commons.lang.math.IntRange;
 import org.apache.hadoop.fs.Path;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Types of {@link Property} values. Each type has a short name, a description, and a regex which valid values match. All of these fields are optional.
  */
@@ -89,11 +92,12 @@ public enum PropertyType {
   URI("uri", x -> true, "A valid URI");
 
   private String shortname, format;
-  private Predicate<String> predicate;
+  // made this transient because findbugs was complaining
+  private transient Predicate<String> predicate;
 
   private PropertyType(String shortname, Predicate<String> predicate, String formatDescription) {
     this.shortname = shortname;
-    this.predicate = predicate;
+    this.predicate = Objects.requireNonNull(predicate);
     this.format = formatDescription;
   }
 
@@ -117,6 +121,7 @@ public enum PropertyType {
    * @return true if value is valid or null, or if this type has no regex
    */
   public boolean isValidFormat(String value) {
+    Preconditions.checkState(predicate != null, "Predicate was null, maybe this enum was serialized????");
     return predicate.test(value);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
index 5d16541..bf0ae28 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
@@ -20,6 +20,7 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.io.Serializable;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 
 import org.apache.accumulo.core.util.ByteBufferUtil;
 
@@ -101,6 +102,23 @@ public class ArrayByteSequence extends ByteSequence implements Serializable {
     }
   }
 
+  private static byte[] copy(ByteSequence bs) {
+    if (bs.isBackedByArray()) {
+      return Arrays.copyOfRange(bs.getBackingArray(), bs.offset(), bs.offset() + bs.length());
+    } else {
+      return bs.toArray();
+    }
+  }
+
+  /**
+   * Copy constructor. Copies contents of byteSequence.
+   *
+   * @since 2.0.0
+   */
+  public ArrayByteSequence(ByteSequence byteSequence) {
+    this(copy(byteSequence));
+  }
+
   @Override
   public byte byteAt(int i) {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/data/thrift/TRowRange.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TRowRange.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TRowRange.java
new file mode 100644
index 0000000..5d1c062
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TRowRange.java
@@ -0,0 +1,521 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.10.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.10.0)")
+public class TRowRange implements org.apache.thrift.TBase<TRowRange, TRowRange._Fields>, java.io.Serializable, Cloneable, Comparable<TRowRange> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowRange");
+
+  private static final org.apache.thrift.protocol.TField START_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("startRow", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField END_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("endRow", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TRowRangeStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TRowRangeTupleSchemeFactory();
+
+  public java.nio.ByteBuffer startRow; // required
+  public java.nio.ByteBuffer endRow; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    START_ROW((short)1, "startRow"),
+    END_ROW((short)2, "endRow");
+
+    private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+    static {
+      for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // START_ROW
+          return START_ROW;
+        case 2: // END_ROW
+          return END_ROW;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(java.lang.String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final java.lang.String _fieldName;
+
+    _Fields(short thriftId, java.lang.String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public java.lang.String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.START_ROW, new org.apache.thrift.meta_data.FieldMetaData("startRow", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    tmpMap.put(_Fields.END_ROW, new org.apache.thrift.meta_data.FieldMetaData("endRow", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRowRange.class, metaDataMap);
+  }
+
+  public TRowRange() {
+  }
+
+  public TRowRange(
+    java.nio.ByteBuffer startRow,
+    java.nio.ByteBuffer endRow)
+  {
+    this();
+    this.startRow = org.apache.thrift.TBaseHelper.copyBinary(startRow);
+    this.endRow = org.apache.thrift.TBaseHelper.copyBinary(endRow);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TRowRange(TRowRange other) {
+    if (other.isSetStartRow()) {
+      this.startRow = org.apache.thrift.TBaseHelper.copyBinary(other.startRow);
+    }
+    if (other.isSetEndRow()) {
+      this.endRow = org.apache.thrift.TBaseHelper.copyBinary(other.endRow);
+    }
+  }
+
+  public TRowRange deepCopy() {
+    return new TRowRange(this);
+  }
+
+  @Override
+  public void clear() {
+    this.startRow = null;
+    this.endRow = null;
+  }
+
+  public byte[] getStartRow() {
+    setStartRow(org.apache.thrift.TBaseHelper.rightSize(startRow));
+    return startRow == null ? null : startRow.array();
+  }
+
+  public java.nio.ByteBuffer bufferForStartRow() {
+    return org.apache.thrift.TBaseHelper.copyBinary(startRow);
+  }
+
+  public TRowRange setStartRow(byte[] startRow) {
+    this.startRow = startRow == null ? (java.nio.ByteBuffer)null : java.nio.ByteBuffer.wrap(startRow.clone());
+    return this;
+  }
+
+  public TRowRange setStartRow(java.nio.ByteBuffer startRow) {
+    this.startRow = org.apache.thrift.TBaseHelper.copyBinary(startRow);
+    return this;
+  }
+
+  public void unsetStartRow() {
+    this.startRow = null;
+  }
+
+  /** Returns true if field startRow is set (has been assigned a value) and false otherwise */
+  public boolean isSetStartRow() {
+    return this.startRow != null;
+  }
+
+  public void setStartRowIsSet(boolean value) {
+    if (!value) {
+      this.startRow = null;
+    }
+  }
+
+  public byte[] getEndRow() {
+    setEndRow(org.apache.thrift.TBaseHelper.rightSize(endRow));
+    return endRow == null ? null : endRow.array();
+  }
+
+  public java.nio.ByteBuffer bufferForEndRow() {
+    return org.apache.thrift.TBaseHelper.copyBinary(endRow);
+  }
+
+  public TRowRange setEndRow(byte[] endRow) {
+    this.endRow = endRow == null ? (java.nio.ByteBuffer)null : java.nio.ByteBuffer.wrap(endRow.clone());
+    return this;
+  }
+
+  public TRowRange setEndRow(java.nio.ByteBuffer endRow) {
+    this.endRow = org.apache.thrift.TBaseHelper.copyBinary(endRow);
+    return this;
+  }
+
+  public void unsetEndRow() {
+    this.endRow = null;
+  }
+
+  /** Returns true if field endRow is set (has been assigned a value) and false otherwise */
+  public boolean isSetEndRow() {
+    return this.endRow != null;
+  }
+
+  public void setEndRowIsSet(boolean value) {
+    if (!value) {
+      this.endRow = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, java.lang.Object value) {
+    switch (field) {
+    case START_ROW:
+      if (value == null) {
+        unsetStartRow();
+      } else {
+        if (value instanceof byte[]) {
+          setStartRow((byte[])value);
+        } else {
+          setStartRow((java.nio.ByteBuffer)value);
+        }
+      }
+      break;
+
+    case END_ROW:
+      if (value == null) {
+        unsetEndRow();
+      } else {
+        if (value instanceof byte[]) {
+          setEndRow((byte[])value);
+        } else {
+          setEndRow((java.nio.ByteBuffer)value);
+        }
+      }
+      break;
+
+    }
+  }
+
+  public java.lang.Object getFieldValue(_Fields field) {
+    switch (field) {
+    case START_ROW:
+      return getStartRow();
+
+    case END_ROW:
+      return getEndRow();
+
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new java.lang.IllegalArgumentException();
+    }
+
+    switch (field) {
+    case START_ROW:
+      return isSetStartRow();
+    case END_ROW:
+      return isSetEndRow();
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(java.lang.Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TRowRange)
+      return this.equals((TRowRange)that);
+    return false;
+  }
+
+  public boolean equals(TRowRange that) {
+    if (that == null)
+      return false;
+    if (this == that)
+      return true;
+
+    boolean this_present_startRow = true && this.isSetStartRow();
+    boolean that_present_startRow = true && that.isSetStartRow();
+    if (this_present_startRow || that_present_startRow) {
+      if (!(this_present_startRow && that_present_startRow))
+        return false;
+      if (!this.startRow.equals(that.startRow))
+        return false;
+    }
+
+    boolean this_present_endRow = true && this.isSetEndRow();
+    boolean that_present_endRow = true && that.isSetEndRow();
+    if (this_present_endRow || that_present_endRow) {
+      if (!(this_present_endRow && that_present_endRow))
+        return false;
+      if (!this.endRow.equals(that.endRow))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int hashCode = 1;
+
+    hashCode = hashCode * 8191 + ((isSetStartRow()) ? 131071 : 524287);
+    if (isSetStartRow())
+      hashCode = hashCode * 8191 + startRow.hashCode();
+
+    hashCode = hashCode * 8191 + ((isSetEndRow()) ? 131071 : 524287);
+    if (isSetEndRow())
+      hashCode = hashCode * 8191 + endRow.hashCode();
+
+    return hashCode;
+  }
+
+  @Override
+  public int compareTo(TRowRange other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = java.lang.Boolean.valueOf(isSetStartRow()).compareTo(other.isSetStartRow());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetStartRow()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, other.startRow);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetEndRow()).compareTo(other.isSetEndRow());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetEndRow()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.endRow, other.endRow);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    scheme(iprot).read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    scheme(oprot).write(oprot, this);
+  }
+
+  @Override
+  public java.lang.String toString() {
+    java.lang.StringBuilder sb = new java.lang.StringBuilder("TRowRange(");
+    boolean first = true;
+
+    sb.append("startRow:");
+    if (this.startRow == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.startRow, sb);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("endRow:");
+    if (this.endRow == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.endRow, sb);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TRowRangeStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public TRowRangeStandardScheme getScheme() {
+      return new TRowRangeStandardScheme();
+    }
+  }
+
+  private static class TRowRangeStandardScheme extends org.apache.thrift.scheme.StandardScheme<TRowRange> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TRowRange struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // START_ROW
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.startRow = iprot.readBinary();
+              struct.setStartRowIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // END_ROW
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.endRow = iprot.readBinary();
+              struct.setEndRowIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TRowRange struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.startRow != null) {
+        oprot.writeFieldBegin(START_ROW_FIELD_DESC);
+        oprot.writeBinary(struct.startRow);
+        oprot.writeFieldEnd();
+      }
+      if (struct.endRow != null) {
+        oprot.writeFieldBegin(END_ROW_FIELD_DESC);
+        oprot.writeBinary(struct.endRow);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TRowRangeTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public TRowRangeTupleScheme getScheme() {
+      return new TRowRangeTupleScheme();
+    }
+  }
+
+  private static class TRowRangeTupleScheme extends org.apache.thrift.scheme.TupleScheme<TRowRange> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TRowRange struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet optionals = new java.util.BitSet();
+      if (struct.isSetStartRow()) {
+        optionals.set(0);
+      }
+      if (struct.isSetEndRow()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetStartRow()) {
+        oprot.writeBinary(struct.startRow);
+      }
+      if (struct.isSetEndRow()) {
+        oprot.writeBinary(struct.endRow);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TRowRange struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.startRow = iprot.readBinary();
+        struct.setStartRowIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.endRow = iprot.readBinary();
+        struct.setEndRowIsSet(true);
+      }
+    }
+  }
+
+  private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+    return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+  }
+  private static void unusedMethod() {}
+}
+


[5/9] accumulo git commit: ACCUMULO-4501 ACCUMULO-96 Added Summarization

Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/summary/SummaryCollection.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/SummaryCollection.java b/core/src/main/java/org/apache/accumulo/core/summary/SummaryCollection.java
new file mode 100644
index 0000000..cc688c9
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummaryCollection.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.summary;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.Summary;
+import org.apache.accumulo.core.data.thrift.TSummaries;
+import org.apache.accumulo.core.data.thrift.TSummarizerConfiguration;
+import org.apache.accumulo.core.data.thrift.TSummary;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This class facilitates merging, storing, and serializing (to/from thrift) intermediate summary information.
+ */
+public class SummaryCollection {
+
+  private static class MergedSummary {
+    Map<String,Long> summary;
+    long filesContaining;
+    long filesExceedingBoundry;
+    long filesLarge;
+
+    public MergedSummary(FileSummary entry) {
+      this.summary = entry.summary;
+      this.filesContaining = 1;
+      this.filesExceedingBoundry = entry.exceededBoundry ? 1 : 0;
+      this.filesLarge = entry.exceededMaxSize ? 1 : 0;
+    }
+
+    public MergedSummary(TSummary tSummary) {
+      this.summary = new HashMap<>(tSummary.getSummary());
+      this.filesContaining = tSummary.getFilesContaining();
+      this.filesExceedingBoundry = tSummary.getFilesExceeding();
+      this.filesLarge = tSummary.getFilesLarge();
+    }
+
+    public void merge(MergedSummary other, SummarizerConfiguration config, SummarizerFactory factory) {
+
+      if (summary == null && other.summary != null) {
+        summary = new HashMap<>(other.summary);
+      } else if (summary != null && other.summary != null) {
+        Summarizer summarizer = factory.getSummarizer(config);
+        summarizer.combiner(config).merge(summary, other.summary);
+      }
+
+      filesContaining += other.filesContaining;
+      filesExceedingBoundry += other.filesExceedingBoundry;
+      filesLarge += other.filesLarge;
+    }
+
+    public TSummary toThrift(SummarizerConfiguration key) {
+      TSummarizerConfiguration tsumConf = SummarizerConfigurationUtil.toThrift(key);
+      return new TSummary(summary, tsumConf, filesContaining, filesExceedingBoundry, filesLarge);
+    }
+
+  }
+
+  private Map<SummarizerConfiguration,MergedSummary> mergedSummaries;
+  private long totalFiles;
+  private long deletedFiles;
+
+  public SummaryCollection() {
+    mergedSummaries = new HashMap<>();
+    totalFiles = 0;
+  }
+
+  public SummaryCollection(TSummaries tsums) {
+    mergedSummaries = new HashMap<>();
+    for (TSummary tSummary : tsums.getSummaries()) {
+      SummarizerConfiguration sconf = SummarizerConfigurationUtil.fromThrift(tSummary.getConfig());
+      mergedSummaries.put(sconf, new MergedSummary(tSummary));
+    }
+
+    totalFiles = tsums.getTotalFiles();
+    deletedFiles = tsums.getDeletedFiles();
+  }
+
+  SummaryCollection(Collection<FileSummary> initialEntries) {
+    this(initialEntries, false);
+  }
+
+  SummaryCollection(Collection<FileSummary> initialEntries, boolean deleted) {
+    if (deleted) {
+      Preconditions.checkArgument(initialEntries.size() == 0);
+    }
+    mergedSummaries = new HashMap<>();
+    for (FileSummary entry : initialEntries) {
+      mergedSummaries.put(entry.conf, new MergedSummary(entry));
+    }
+    totalFiles = 1;
+    this.deletedFiles = deleted ? 1 : 0;
+  }
+
+  static class FileSummary {
+
+    private SummarizerConfiguration conf;
+    private Map<String,Long> summary;
+    private boolean exceededBoundry;
+    private boolean exceededMaxSize;
+
+    FileSummary(SummarizerConfiguration conf, Map<String,Long> summary, boolean exceededBoundry) {
+      this.conf = conf;
+      this.summary = summary;
+      this.exceededBoundry = exceededBoundry;
+      this.exceededMaxSize = false;
+    }
+
+    FileSummary(SummarizerConfiguration conf) {
+      this.conf = conf;
+      this.summary = new HashMap<>();
+      ;
+      this.exceededBoundry = false;
+      this.exceededMaxSize = true;
+    }
+  }
+
+  public void merge(SummaryCollection other, SummarizerFactory factory) {
+    for (Entry<SummarizerConfiguration,MergedSummary> entry : other.mergedSummaries.entrySet()) {
+      MergedSummary ms = mergedSummaries.get(entry.getKey());
+      if (ms == null) {
+        mergedSummaries.put(entry.getKey(), entry.getValue());
+      } else {
+        ms.merge(entry.getValue(), entry.getKey(), factory);
+      }
+    }
+
+    this.totalFiles += other.totalFiles;
+    this.deletedFiles += other.deletedFiles;
+  }
+
+  public static SummaryCollection merge(SummaryCollection sc1, SummaryCollection sc2, SummarizerFactory factory) {
+    SummaryCollection ret = new SummaryCollection();
+    ret.merge(sc1, factory);
+    ret.merge(sc2, factory);
+    return ret;
+  }
+
+  public List<Summary> getSummaries() {
+    ArrayList<Summary> ret = new ArrayList<>(mergedSummaries.size());
+
+    for (Entry<SummarizerConfiguration,MergedSummary> entry : mergedSummaries.entrySet()) {
+      SummarizerConfiguration config = entry.getKey();
+      MergedSummary ms = entry.getValue();
+
+      ret.add(new Summary(ms.summary, config, totalFiles, (totalFiles - deletedFiles) - ms.filesContaining, ms.filesExceedingBoundry, ms.filesLarge,
+          deletedFiles));
+    }
+
+    return ret;
+  }
+
+  public long getTotalFiles() {
+    return totalFiles;
+  }
+
+  public TSummaries toThrift() {
+    List<TSummary> summaries = new ArrayList<>(mergedSummaries.size());
+    for (Entry<SummarizerConfiguration,MergedSummary> entry : mergedSummaries.entrySet()) {
+      summaries.add(entry.getValue().toThrift(entry.getKey()));
+    }
+
+    return new TSummaries(true, -1l, totalFiles, deletedFiles, summaries);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/summary/SummaryInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/SummaryInfo.java b/core/src/main/java/org/apache/accumulo/core/summary/SummaryInfo.java
new file mode 100644
index 0000000..7b9ebe4
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummaryInfo.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.summary;
+
+import java.util.Map;
+
+import org.apache.hadoop.io.Text;
+
+class SummaryInfo {
+
+  final Map<String,Long> summary;
+  final Text lastRow;
+  final int count;
+
+  SummaryInfo(Text row, Map<String,Long> summary, int count) {
+    this.lastRow = row;
+    this.summary = summary;
+    this.count = count;
+  }
+
+  SummaryInfo(byte[] row, Map<String,Long> summary, int count) {
+    this.lastRow = new Text(row);
+    this.summary = summary;
+    this.count = count;
+  }
+
+  Text getLastRow() {
+    return lastRow;
+  }
+
+  Map<String,Long> getSummary() {
+    return summary;
+  }
+
+  int getCount() {
+    return count;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java b/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
new file mode 100644
index 0000000..9b2b5d9
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.summary;
+
+import java.io.DataInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Predicate;
+
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.file.blockfile.cache.BlockCache;
+import org.apache.accumulo.core.file.blockfile.cache.CacheEntry;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.rfile.RFile.Reader;
+import org.apache.accumulo.core.file.rfile.bcfile.MetaBlockDoesNotExist;
+import org.apache.accumulo.core.summary.Gatherer.RowRange;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.io.WritableUtils;
+
+public class SummaryReader {
+
+  private static interface BlockReader {
+    DataInputStream getMetaBlock(String name) throws IOException;
+  }
+
+  private static class CompositeCache implements BlockCache {
+
+    private BlockCache summaryCache;
+    private BlockCache indexCache;
+
+    CompositeCache(BlockCache summaryCache, BlockCache indexCache) {
+      this.summaryCache = summaryCache;
+      this.indexCache = indexCache;
+    }
+
+    @Override
+    public CacheEntry cacheBlock(String blockName, byte[] buf) {
+      return summaryCache.cacheBlock(blockName, buf);
+    }
+
+    @Override
+    public CacheEntry cacheBlock(String blockName, byte[] buf, boolean inMemory) {
+      return summaryCache.cacheBlock(blockName, buf, inMemory);
+    }
+
+    @Override
+    public CacheEntry getBlock(String blockName) {
+      CacheEntry ce = summaryCache.getBlock(blockName);
+      if (ce == null) {
+        // Its possible the index cache may have this info, so check there. This is an opportunistic check.
+        ce = indexCache.getBlock(blockName);
+      }
+      return ce;
+    }
+
+    @Override
+    public long getMaxSize() {
+      return summaryCache.getMaxSize();
+    }
+
+    @Override
+    public Stats getStats() {
+      return summaryCache.getStats();
+    }
+  }
+
+  private static List<SummarySerializer> load(BlockReader bcReader, Predicate<SummarizerConfiguration> summarySelector) throws IOException {
+
+    try (DataInputStream in = bcReader.getMetaBlock(SummaryWriter.METASTORE_INDEX)) {
+      List<SummarySerializer> stores = new ArrayList<>();
+
+      readHeader(in);
+      int numSummaries = WritableUtils.readVInt(in);
+      for (int i = 0; i < numSummaries; i++) {
+        SummarizerConfiguration conf = readConfig(in);
+        boolean inline = in.readBoolean();
+        if (inline) {
+          if (summarySelector.test(conf)) {
+            stores.add(SummarySerializer.load(conf, in));
+          } else {
+            SummarySerializer.skip(in);
+          }
+        } else {
+          int block = WritableUtils.readVInt(in);
+          int offset = WritableUtils.readVInt(in);
+          if (summarySelector.test(conf)) {
+            try (DataInputStream summaryIn = bcReader.getMetaBlock(SummaryWriter.METASTORE_PREFIX + "." + block)) {
+              long skipped = in.skip(offset);
+              while (skipped < offset) {
+                skipped += in.skip(offset - skipped);
+              }
+              stores.add(SummarySerializer.load(conf, summaryIn));
+            } catch (MetaBlockDoesNotExist e) {
+              // this is unexpected
+              throw new IOException(e);
+            }
+          }
+        }
+      }
+
+      return stores;
+    } catch (MetaBlockDoesNotExist e) {
+      return Collections.emptyList();
+    }
+  }
+
+  private static SummaryReader load(CachableBlockFile.Reader bcReader, Predicate<SummarizerConfiguration> summarySelector, SummarizerFactory factory)
+      throws IOException {
+    SummaryReader fileSummaries = new SummaryReader();
+    fileSummaries.summaryStores = load(name -> bcReader.getMetaBlock(name), summarySelector);
+    fileSummaries.factory = factory;
+    return fileSummaries;
+  }
+
+  public static SummaryReader load(Configuration conf, AccumuloConfiguration aConf, InputStream inputStream, long length,
+      Predicate<SummarizerConfiguration> summarySelector, SummarizerFactory factory) throws IOException {
+    org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.Reader bcReader = new CachableBlockFile.Reader((InputStream & Seekable) inputStream, length,
+        conf, aConf);
+    return load(bcReader, summarySelector, factory);
+  }
+
+  public static SummaryReader load(FileSystem fs, Configuration conf, AccumuloConfiguration aConf, SummarizerFactory factory, Path file,
+      Predicate<SummarizerConfiguration> summarySelector, BlockCache summaryCache, BlockCache indexCache) {
+    CachableBlockFile.Reader bcReader = null;
+
+    try {
+      // the reason BCFile is used instead of RFile is to avoid reading in the RFile meta block when only summary data is wanted.
+      CompositeCache compositeCache = new CompositeCache(summaryCache, indexCache);
+      bcReader = new CachableBlockFile.Reader(fs, file, conf, null, compositeCache, aConf);
+      return load(bcReader, summarySelector, factory);
+    } catch (FileNotFoundException fne) {
+      SummaryReader sr = new SummaryReader();
+      sr.factory = factory;
+      sr.summaryStores = Collections.emptyList();
+      sr.deleted = true;
+      return sr;
+    } catch (IOException e) {
+      try {
+        if (!fs.exists(file)) {
+          SummaryReader sr = new SummaryReader();
+          sr.factory = factory;
+          sr.summaryStores = Collections.emptyList();
+          sr.deleted = true;
+          return sr;
+        }
+      } catch (IOException e1) {}
+      throw new UncheckedIOException(e);
+    } finally {
+      if (bcReader != null) {
+        try {
+          bcReader.close();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      }
+    }
+
+  }
+
+  private static void print(FileSKVIterator fsi, String indent, PrintStream out) throws IOException {
+
+    out.printf("Summary data : \n");
+
+    List<SummarySerializer> stores = load(name -> fsi.getMetaStore(name), conf -> true);
+    int i = 1;
+    for (SummarySerializer summaryStore : stores) {
+      out.printf("%sSummary %d of %d generated by : %s\n", indent, i, stores.size(), summaryStore.getSummarizerConfiguration());
+      i++;
+      summaryStore.print(indent, indent, out);
+    }
+  }
+
+  public static void print(Reader iter, PrintStream out) throws IOException {
+    print(iter, "   ", out);
+  }
+
+  private static SummarizerConfiguration readConfig(DataInputStream in) throws IOException {
+    // read summarizer configuration
+    String summarizerClazz = in.readUTF();
+    String configId = in.readUTF();
+    org.apache.accumulo.core.client.summary.SummarizerConfiguration.Builder scb = SummarizerConfiguration.builder(summarizerClazz).setPropertyId(configId);
+    int numOpts = WritableUtils.readVInt(in);
+    for (int i = 0; i < numOpts; i++) {
+      String k = in.readUTF();
+      String v = in.readUTF();
+      scb.addOption(k, v);
+    }
+
+    return scb.build();
+  }
+
+  private static byte readHeader(DataInputStream in) throws IOException {
+    long magic = in.readLong();
+    if (magic != SummaryWriter.MAGIC) {
+      throw new IOException("Bad magic : " + String.format("%x", magic));
+    }
+
+    byte ver = in.readByte();
+    if (ver != SummaryWriter.VER) {
+      throw new IOException("Unknown version : " + ver);
+    }
+
+    return ver;
+  }
+
+  private List<SummarySerializer> summaryStores;
+
+  private SummarizerFactory factory;
+
+  private boolean deleted;
+
+  public SummaryCollection getSummaries(List<RowRange> ranges) {
+
+    List<SummaryCollection.FileSummary> initial = new ArrayList<>();
+    if (deleted) {
+      return new SummaryCollection(initial, true);
+    }
+    for (SummarySerializer summaryStore : summaryStores) {
+      if (summaryStore.exceededMaxSize()) {
+        initial.add(new SummaryCollection.FileSummary(summaryStore.getSummarizerConfiguration()));
+      } else {
+        Map<String,Long> summary = summaryStore.getSummary(ranges, factory);
+        boolean exceeded = summaryStore.exceedsRange(ranges);
+        initial.add(new SummaryCollection.FileSummary(summaryStore.getSummarizerConfiguration(), summary, exceeded));
+      }
+    }
+
+    return new SummaryCollection(initial);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java b/core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java
new file mode 100644
index 0000000..d76bd1a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java
@@ -0,0 +1,542 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.summary;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.Summarizer.Collector;
+import org.apache.accumulo.core.client.summary.Summarizer.Combiner;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.summary.Gatherer.RowRange;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * This class supports serializing summaries and periodically storing summaries. The implementations attempts to generate around 10 summaries that are evenly
+ * spaced. This allows asking for summaries for sub-ranges of data in a rfile.
+ *
+ * <p>
+ * At first summaries are created for every 1000 keys values. After 10 summaries are added, the 10 summaries are merged to 5 and summaries are then created for
+ * every 2000 key values. The code keeps merging summaries and doubling the amount of key values per summary. This results in each summary covering about the
+ * same number of key values.
+ *
+ */
+
+class SummarySerializer {
+
+  private SummarizerConfiguration sconf;
+  private LgSummaries[] allSummaries;
+
+  private SummarySerializer(SummarizerConfiguration sconf, LgSummaries[] allSummaries) {
+    this.sconf = sconf;
+    this.allSummaries = allSummaries;
+  }
+
+  private SummarySerializer(SummarizerConfiguration sconf) {
+    this.sconf = sconf;
+    // this indicates max size was exceeded
+    this.allSummaries = null;
+  }
+
+  public SummarizerConfiguration getSummarizerConfiguration() {
+    return sconf;
+  }
+
+  public void print(String prefix, String indent, PrintStream out) {
+
+    if (allSummaries == null) {
+      out.printf("%sSummary not stored because it was too large\n", prefix + indent);
+    } else {
+      for (LgSummaries lgs : allSummaries) {
+        lgs.print(prefix, indent, out);
+      }
+    }
+  }
+
+  public Map<String,Long> getSummary(List<RowRange> ranges, SummarizerFactory sf) {
+
+    Summarizer kvs = sf.getSummarizer(sconf);
+
+    Map<String,Long> summary = new HashMap<>();
+    for (LgSummaries lgs : allSummaries) {
+      lgs.getSummary(ranges, kvs.combiner(sconf), summary);
+    }
+    return summary;
+  }
+
+  public boolean exceedsRange(List<RowRange> ranges) {
+    boolean er = false;
+    for (LgSummaries lgs : allSummaries) {
+      for (RowRange ke : ranges) {
+        er |= lgs.exceedsRange(ke.getStartRow(), ke.getEndRow());
+        if (er) {
+          return er;
+        }
+      }
+    }
+
+    return er;
+  }
+
+  public boolean exceededMaxSize() {
+    return allSummaries == null;
+  }
+
+  private static class SummaryStoreImpl implements org.apache.accumulo.core.client.summary.Summarizer.StatisticConsumer {
+
+    HashMap<String,Long> summaries;
+
+    @Override
+    public void accept(String summary, long value) {
+      summaries.put(summary, value);
+    }
+  }
+
+  private static class LgBuilder {
+    private Summarizer summarizer;
+    private SummarizerConfiguration conf;
+    private Collector collector;
+
+    private int maxSummaries = 10;
+
+    private int cutoff = 1000;
+    private int count = 0;
+
+    private List<SummaryInfo> summaries = new ArrayList<>();
+
+    private Key lastKey;
+
+    private SummaryStoreImpl sci = new SummaryStoreImpl();
+
+    private String name;
+
+    private boolean sawFirst = false;
+    private Text firstRow;
+
+    private boolean finished = false;
+
+    public LgBuilder(SummarizerConfiguration conf, Summarizer kvs) {
+      this.conf = conf;
+      this.summarizer = kvs;
+      this.name = "<DEFAULT>";
+      this.collector = kvs.collector(conf);
+    }
+
+    public LgBuilder(SummarizerConfiguration conf, Summarizer kvs, String name) {
+      this.conf = conf;
+      this.summarizer = kvs;
+      this.name = name;
+      this.collector = kvs.collector(conf);
+    }
+
+    public void put(Key k, Value v) {
+      collector.accept(k, v);
+      count++;
+
+      if (!sawFirst) {
+        firstRow = k.getRow();
+        sawFirst = true;
+
+      }
+
+      if (count >= cutoff) {
+        sci.summaries = new HashMap<>();
+        collector.summarize(sci);
+        collector = summarizer.collector(conf);
+        addSummary(k.getRow(), sci.summaries, count);
+        count = 0;
+      }
+
+      lastKey = k;
+    }
+
+    private List<SummaryInfo> merge(int end) {
+      List<SummaryInfo> mergedSummaries = new ArrayList<>();
+      for (int i = 0; i < end; i += 2) {
+        int mergedCount = summaries.get(i).count + summaries.get(i + 1).count;
+        summarizer.combiner(conf).merge(summaries.get(i).summary, summaries.get(i + 1).summary);
+        mergedSummaries.add(new SummaryInfo(summaries.get(i + 1).getLastRow(), summaries.get(i).summary, mergedCount));
+      }
+      return mergedSummaries;
+    }
+
+    private void addSummary(Text row, Map<String,Long> summary, int count) {
+      Preconditions.checkState(!finished);
+      summaries.add(new SummaryInfo(row, summary, count));
+
+      if (summaries.size() % 2 == 0 && summaries.size() > maxSummaries) {
+        summaries = merge(summaries.size());
+        cutoff *= 2;
+      }
+    }
+
+    boolean collapse() {
+      Preconditions.checkState(finished);
+      if (summaries.size() <= 1) {
+        return false;
+      }
+
+      int end = summaries.size();
+      if (end % 2 == 1) {
+        end--;
+      }
+
+      List<SummaryInfo> mergedSummaries = merge(end);
+
+      if (summaries.size() % 2 == 1) {
+        mergedSummaries.add(summaries.get(summaries.size() - 1));
+      }
+
+      summaries = mergedSummaries;
+
+      return true;
+    }
+
+    void finish() {
+      Preconditions.checkState(!finished);
+      // summarize last data
+      if (count > 0) {
+        sci.summaries = new HashMap<>();
+        collector.summarize(sci);
+        collector = null;
+        addSummary(lastKey.getRow(), sci.summaries, count);
+        count = 0;
+        finished = true;
+      }
+    }
+
+    public void save(DataOutputStream dos, HashMap<String,Integer> symbolTable) throws IOException {
+      Preconditions.checkState(count == 0);
+
+      dos.writeUTF(name);
+
+      if (firstRow == null) {
+        WritableUtils.writeVInt(dos, 0);
+      } else {
+        firstRow.write(dos);
+      }
+
+      // write summaries
+      WritableUtils.writeVInt(dos, summaries.size());
+      for (SummaryInfo summaryInfo : summaries) {
+        summaryInfo.getLastRow().write(dos);
+        WritableUtils.writeVInt(dos, summaryInfo.count);
+        saveSummary(dos, symbolTable, summaryInfo.summary);
+      }
+    }
+
+    private void saveSummary(DataOutputStream dos, HashMap<String,Integer> symbolTable, Map<String,Long> summary) throws IOException {
+      WritableUtils.writeVInt(dos, summary.size());
+      for (Entry<String,Long> e : summary.entrySet()) {
+        WritableUtils.writeVInt(dos, symbolTable.get(e.getKey()));
+        WritableUtils.writeVLong(dos, e.getValue());
+      }
+    }
+  }
+
+  public static class Builder {
+    private Summarizer kvs;
+
+    private SummarizerConfiguration conf;
+
+    private List<LgBuilder> locGroups;
+    private LgBuilder lgb;
+
+    private long maxSize;
+
+    public Builder(SummarizerConfiguration conf, Summarizer kvs, long maxSize) {
+      this.conf = conf;
+      this.kvs = kvs;
+      this.locGroups = new ArrayList<>();
+      this.maxSize = maxSize;
+    }
+
+    public void put(Key k, Value v) {
+      lgb.put(k, v);
+    }
+
+    public SummarizerConfiguration getSummarizerConfiguration() {
+      return conf;
+    }
+
+    public void save(DataOutputStream dos) throws IOException {
+
+      if (lgb != null) {
+        lgb.finish();
+        locGroups.add(lgb);
+      }
+
+      byte[] data = _save();
+
+      while (data.length > maxSize) {
+        boolean collapsedSome = false;
+        for (LgBuilder lgBuilder : locGroups) {
+          collapsedSome |= lgBuilder.collapse();
+        }
+
+        if (collapsedSome) {
+          data = _save();
+        } else {
+          break;
+        }
+      }
+
+      if (data.length > maxSize) {
+        dos.writeBoolean(true);
+      } else {
+        dos.writeBoolean(false);
+        // write this out to support efficient skipping
+        WritableUtils.writeVInt(dos, data.length);
+        dos.write(data);
+      }
+    }
+
+    private byte[] _save() throws IOException {
+
+      try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream dos = new DataOutputStream(baos)) {
+        // create a symbol table
+        HashMap<String,Integer> symbolTable = new HashMap<>();
+        ArrayList<String> symbols = new ArrayList<>();
+        for (LgBuilder lg : locGroups) {
+          for (SummaryInfo si : lg.summaries) {
+            for (String symbol : si.summary.keySet()) {
+              if (!symbolTable.containsKey(symbol)) {
+                symbolTable.put(symbol, symbols.size());
+                symbols.add(symbol);
+              }
+            }
+          }
+        }
+
+        // write symbol table
+        WritableUtils.writeVInt(dos, symbols.size());
+        for (String symbol : symbols) {
+          dos.writeUTF(symbol);
+        }
+
+        WritableUtils.writeVInt(dos, locGroups.size());
+        for (LgBuilder lg : locGroups) {
+          lg.save(dos, symbolTable);
+        }
+
+        dos.close();
+        return baos.toByteArray();
+      }
+    }
+
+    public void startNewLocalityGroup(String name, Set<ByteSequence> columnFamilies) {
+      if (lgb != null) {
+        lgb.finish();
+        locGroups.add(lgb);
+      }
+
+      lgb = new LgBuilder(conf, kvs, name);
+    }
+
+    public void startDefaultLocalityGroup() {
+      if (lgb != null) {
+        lgb.finish();
+        locGroups.add(lgb);
+      }
+
+      lgb = new LgBuilder(conf, kvs);
+    }
+  }
+
+  public static Builder builder(SummarizerConfiguration conf, SummarizerFactory factory, long maxSize) {
+    return new Builder(conf, factory.getSummarizer(conf), maxSize);
+  }
+
+  static void skip(DataInputStream in) throws IOException {
+    boolean exceededMaxSize = in.readBoolean();
+    if (!exceededMaxSize) {
+      long len = WritableUtils.readVInt(in);
+      long skipped = in.skip(len);
+      while (skipped < len) {
+        skipped += in.skip(len - skipped);
+      }
+    }
+  }
+
+  static SummarySerializer load(SummarizerConfiguration sconf, DataInputStream in) throws IOException {
+    boolean exceededMaxSize = in.readBoolean();
+    if (!exceededMaxSize) {
+      WritableUtils.readVInt(in);
+      // load symbol table
+      int numSymbols = WritableUtils.readVInt(in);
+      String[] symbols = new String[numSymbols];
+      for (int i = 0; i < numSymbols; i++) {
+        symbols[i] = in.readUTF();
+      }
+
+      int numLGroups = WritableUtils.readVInt(in);
+      LgSummaries[] allSummaries = new LgSummaries[numLGroups];
+      for (int i = 0; i < numLGroups; i++) {
+        allSummaries[i] = readLGroup(in, symbols);
+      }
+
+      return new SummarySerializer(sconf, allSummaries);
+    } else {
+      return new SummarySerializer(sconf);
+    }
+  }
+
+  private static class LgSummaries {
+
+    private Text firstRow;
+    private SummaryInfo[] summaries;
+    private String lgroupName;
+
+    LgSummaries(Text firstRow, SummaryInfo[] summaries, String lgroupName) {
+      this.firstRow = firstRow;
+      this.summaries = summaries;
+      this.lgroupName = lgroupName;
+    }
+
+    boolean exceedsRange(Text startRow, Text endRow) {
+
+      Text lastRow = summaries[summaries.length - 1].lastRow;
+      if (startRow != null && firstRow.compareTo(startRow) <= 0 && startRow.compareTo(lastRow) < 0) {
+        return true;
+      }
+
+      if (endRow != null && endRow.compareTo(firstRow) >= 0 && lastRow.compareTo(endRow) > 0) {
+        return true;
+      }
+
+      return false;
+    }
+
+    void print(String prefix, String indent, PrintStream out) {
+      String p = prefix + indent;
+      out.printf("%sLocality group : %s\n", p, lgroupName);
+      p += indent;
+      for (SummaryInfo si : summaries) {
+        out.printf("%sSummary of %d key values (row of last key '%s') : \n", p, si.count, si.lastRow);
+        Set<Entry<String,Long>> es = si.summary.entrySet();
+        String p2 = p + indent;
+        for (Entry<String,Long> entry : es) {
+          out.printf("%s%s = %s\n", p2, entry.getKey(), entry.getValue());
+        }
+      }
+    }
+
+    void getSummary(List<RowRange> ranges, Combiner combiner, Map<String,Long> summary) {
+      boolean[] summariesThatOverlap = new boolean[summaries.length];
+
+      for (RowRange keyExtent : ranges) {
+        Text startRow = keyExtent.getStartRow();
+        Text endRow = keyExtent.getEndRow();
+
+        if (endRow != null && endRow.compareTo(firstRow) < 0) {
+          continue;
+        }
+
+        int start = -1;
+        int end = summaries.length - 1;
+
+        if (startRow == null) {
+          start = 0;
+        } else {
+          for (int i = 0; i < summaries.length; i++) {
+            if (startRow.compareTo(summaries[i].getLastRow()) < 0) {
+              start = i;
+              break;
+            }
+          }
+        }
+
+        if (start == -1) {
+          continue;
+        }
+
+        if (endRow == null) {
+          end = summaries.length - 1;
+        } else {
+          for (int i = start; i < summaries.length; i++) {
+            if (endRow.compareTo(summaries[i].getLastRow()) < 0) {
+              end = i;
+              break;
+            }
+          }
+        }
+
+        for (int i = start; i <= end; i++) {
+          summariesThatOverlap[i] = true;
+        }
+      }
+
+      for (int i = 0; i < summaries.length; i++) {
+        if (summariesThatOverlap[i]) {
+          combiner.merge(summary, summaries[i].summary);
+        }
+      }
+    }
+  }
+
+  private static LgSummaries readLGroup(DataInputStream in, String[] symbols) throws IOException {
+    String lgroupName = in.readUTF();
+
+    // read first row
+    Text firstRow = new Text();
+    firstRow.readFields(in);
+
+    // read summaries
+    int numSummaries = WritableUtils.readVInt(in);
+    SummaryInfo[] summaries = new SummaryInfo[numSummaries];
+    for (int i = 0; i < numSummaries; i++) {
+      int rowLen = WritableUtils.readVInt(in);
+      byte[] row = new byte[rowLen];
+      in.readFully(row);
+      int count = WritableUtils.readVInt(in);
+      Map<String,Long> summary = readSummary(in, symbols);
+      summaries[i] = new SummaryInfo(row, summary, count);
+    }
+
+    return new LgSummaries(firstRow, summaries, lgroupName);
+  }
+
+  private static Map<String,Long> readSummary(DataInputStream in, String[] symbols) throws IOException {
+    com.google.common.collect.ImmutableMap.Builder<String,Long> imb = ImmutableMap.builder();
+    int numEntries = WritableUtils.readVInt(in);
+
+    for (int i = 0; i < numEntries; i++) {
+      String symbol = symbols[WritableUtils.readVInt(in)];
+      imb.put(symbol, WritableUtils.readVLong(in));
+    }
+
+    return imb.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/summary/SummaryWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/SummaryWriter.java b/core/src/main/java/org/apache/accumulo/core/summary/SummaryWriter.java
new file mode 100644
index 0000000..1ebeeae
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummaryWriter.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.summary;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.file.FileSKVWriter;
+import org.apache.hadoop.io.WritableUtils;
+
+public class SummaryWriter implements FileSKVWriter {
+
+  static final String METASTORE_PREFIX = "accumulo.summaries";
+  static final String METASTORE_INDEX = "accumulo.summaries.index";
+
+  // echo "accumulo summarize" | sha1sum | head -c 8
+  static long MAGIC = 0x15ea283ec03e4c49L;
+  static byte VER = 1;
+
+  private FileSKVWriter writer;
+  private SummarySerializer.Builder[] summaryStores;
+
+  private SummaryWriter(FileSKVWriter writer, SummarizerFactory factory, List<SummarizerConfiguration> configs, long maxSize) {
+    this.writer = writer;
+    int i = 0;
+    summaryStores = new SummarySerializer.Builder[configs.size()];
+    for (SummarizerConfiguration sconf : configs) {
+      summaryStores[i++] = SummarySerializer.builder(sconf, factory, maxSize);
+    }
+  }
+
+  @Override
+  public boolean supportsLocalityGroups() {
+    return true;
+  }
+
+  @Override
+  public void startNewLocalityGroup(String name, Set<ByteSequence> columnFamilies) throws IOException {
+    for (SummarySerializer.Builder ssb : summaryStores) {
+      ssb.startNewLocalityGroup(name, columnFamilies);
+    }
+
+    writer.startNewLocalityGroup(name, columnFamilies);
+  }
+
+  @Override
+  public void startDefaultLocalityGroup() throws IOException {
+    for (SummarySerializer.Builder ssb : summaryStores) {
+      ssb.startDefaultLocalityGroup();
+    }
+    writer.startDefaultLocalityGroup();
+  }
+
+  @Override
+  public void append(Key key, Value value) throws IOException {
+    writer.append(key, value);
+    for (SummarySerializer.Builder ssb : summaryStores) {
+      ssb.put(key, value);
+    }
+  }
+
+  @Override
+  public DataOutputStream createMetaStore(String name) throws IOException {
+    return writer.createMetaStore(name);
+  }
+
+  public void writeConfig(SummarizerConfiguration conf, DataOutputStream dos) throws IOException {
+    // save class (and its config) used to generate summaries
+    dos.writeUTF(conf.getClassName());
+    dos.writeUTF(conf.getPropertyId());
+    WritableUtils.writeVInt(dos, conf.getOptions().size());
+    for (Entry<String,String> entry : conf.getOptions().entrySet()) {
+      dos.writeUTF(entry.getKey());
+      dos.writeUTF(entry.getValue());
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+
+    DataOutputStream out = writer.createMetaStore(METASTORE_INDEX);
+    out.writeLong(MAGIC);
+    out.write(VER);
+    WritableUtils.writeVInt(out, summaryStores.length);
+
+    // Could possibly inline small summaries in the future. Breaking summaries into multiple block is better for caching a subset of summaries. Also, keeping
+    // the index small is good for the case where summaries that do not exist are requested. However multiple blocks cause more random I/O in the case when its
+    // not yet in the cache.
+
+    for (int i = 0; i < summaryStores.length; i++) {
+      writeConfig(summaryStores[i].getSummarizerConfiguration(), out);
+      // write if summary is inlined in index... support for possible future optimizations.
+      out.writeBoolean(false);
+      // write pointer to block that will contain summary data
+      WritableUtils.writeVInt(out, i);
+      // write offset of summary data within block. This is not currently used, but it supports storing multiple summaries in an external block in the
+      // future without changing the code.
+      WritableUtils.writeVInt(out, 0);
+    }
+    out.close();
+
+    for (int i = 0; i < summaryStores.length; i++) {
+      DataOutputStream summaryOut = writer.createMetaStore(METASTORE_PREFIX + "." + i);
+      summaryStores[i].save(summaryOut);
+      summaryOut.close();
+    }
+
+    writer.close();
+  }
+
+  @Override
+  public long getLength() throws IOException {
+    return writer.getLength();
+  }
+
+  public static FileSKVWriter wrap(FileSKVWriter writer, AccumuloConfiguration tableConfig, boolean useAccumuloStart) {
+    List<SummarizerConfiguration> configs = SummarizerConfigurationUtil.getSummarizerConfigs(tableConfig);
+
+    if (configs.size() == 0) {
+      return writer;
+    }
+
+    SummarizerFactory factory;
+    if (useAccumuloStart) {
+      factory = new SummarizerFactory(tableConfig);
+    } else {
+      factory = new SummarizerFactory();
+    }
+
+    long maxSize = tableConfig.getMemoryInBytes(Property.TABLE_FILE_SUMMARY_MAX_SIZE);
+    return new SummaryWriter(writer, factory, configs, maxSize);
+  }
+}


[4/9] accumulo git commit: ACCUMULO-4501 ACCUMULO-96 Added Summarization

Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
index 641556c..c45d91f 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
@@ -90,6 +90,14 @@ public class TabletClientService {
 
     public java.util.List<java.lang.String> getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.thrift.TException;
 
+    public org.apache.accumulo.core.data.thrift.TSummaries startGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
+
+    public org.apache.accumulo.core.data.thrift.TSummaries startGetSummariesForPartition(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, int modulus, int remainder) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+    public org.apache.accumulo.core.data.thrift.TSummaries startGetSummariesFromFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> files) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+    public org.apache.accumulo.core.data.thrift.TSummaries contiuneGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId) throws NoSuchScanIDException, org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface extends org.apache.accumulo.core.client.impl.thrift.ClientService .AsyncIface {
@@ -156,6 +164,14 @@ public class TabletClientService {
 
     public void getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException;
 
+    public void startGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
+
+    public void startGetSummariesForPartition(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, int modulus, int remainder, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
+
+    public void startGetSummariesFromFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> files, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
+
+    public void contiuneGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.accumulo.core.client.impl.thrift.ClientService.Client implements Iface {
@@ -934,6 +950,123 @@ public class TabletClientService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getActiveLogs failed: unknown result");
     }
 
+    public org.apache.accumulo.core.data.thrift.TSummaries startGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    {
+      send_startGetSummaries(tinfo, credentials, request);
+      return recv_startGetSummaries();
+    }
+
+    public void send_startGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request) throws org.apache.thrift.TException
+    {
+      startGetSummaries_args args = new startGetSummaries_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setRequest(request);
+      sendBase("startGetSummaries", args);
+    }
+
+    public org.apache.accumulo.core.data.thrift.TSummaries recv_startGetSummaries() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    {
+      startGetSummaries_result result = new startGetSummaries_result();
+      receiveBase(result, "startGetSummaries");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      if (result.tope != null) {
+        throw result.tope;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "startGetSummaries failed: unknown result");
+    }
+
+    public org.apache.accumulo.core.data.thrift.TSummaries startGetSummariesForPartition(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, int modulus, int remainder) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_startGetSummariesForPartition(tinfo, credentials, request, modulus, remainder);
+      return recv_startGetSummariesForPartition();
+    }
+
+    public void send_startGetSummariesForPartition(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, int modulus, int remainder) throws org.apache.thrift.TException
+    {
+      startGetSummariesForPartition_args args = new startGetSummariesForPartition_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setRequest(request);
+      args.setModulus(modulus);
+      args.setRemainder(remainder);
+      sendBase("startGetSummariesForPartition", args);
+    }
+
+    public org.apache.accumulo.core.data.thrift.TSummaries recv_startGetSummariesForPartition() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      startGetSummariesForPartition_result result = new startGetSummariesForPartition_result();
+      receiveBase(result, "startGetSummariesForPartition");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "startGetSummariesForPartition failed: unknown result");
+    }
+
+    public org.apache.accumulo.core.data.thrift.TSummaries startGetSummariesFromFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> files) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_startGetSummariesFromFiles(tinfo, credentials, request, files);
+      return recv_startGetSummariesFromFiles();
+    }
+
+    public void send_startGetSummariesFromFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> files) throws org.apache.thrift.TException
+    {
+      startGetSummariesFromFiles_args args = new startGetSummariesFromFiles_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setRequest(request);
+      args.setFiles(files);
+      sendBase("startGetSummariesFromFiles", args);
+    }
+
+    public org.apache.accumulo.core.data.thrift.TSummaries recv_startGetSummariesFromFiles() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_result();
+      receiveBase(result, "startGetSummariesFromFiles");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "startGetSummariesFromFiles failed: unknown result");
+    }
+
+    public org.apache.accumulo.core.data.thrift.TSummaries contiuneGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId) throws NoSuchScanIDException, org.apache.thrift.TException
+    {
+      send_contiuneGetSummaries(tinfo, sessionId);
+      return recv_contiuneGetSummaries();
+    }
+
+    public void send_contiuneGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId) throws org.apache.thrift.TException
+    {
+      contiuneGetSummaries_args args = new contiuneGetSummaries_args();
+      args.setTinfo(tinfo);
+      args.setSessionId(sessionId);
+      sendBase("contiuneGetSummaries", args);
+    }
+
+    public org.apache.accumulo.core.data.thrift.TSummaries recv_contiuneGetSummaries() throws NoSuchScanIDException, org.apache.thrift.TException
+    {
+      contiuneGetSummaries_result result = new contiuneGetSummaries_result();
+      receiveBase(result, "contiuneGetSummaries");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.nssi != null) {
+        throw result.nssi;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "contiuneGetSummaries failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends org.apache.accumulo.core.client.impl.thrift.ClientService.AsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -2220,6 +2353,164 @@ public class TabletClientService {
       }
     }
 
+    public void startGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      startGetSummaries_call method_call = new startGetSummaries_call(tinfo, credentials, request, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class startGetSummaries_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.data.thrift.TSummaries> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      private org.apache.accumulo.core.data.thrift.TSummaryRequest request;
+      public startGetSummaries_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.request = request;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startGetSummaries", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        startGetSummaries_args args = new startGetSummaries_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setRequest(request);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public org.apache.accumulo.core.data.thrift.TSummaries getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_startGetSummaries();
+      }
+    }
+
+    public void startGetSummariesForPartition(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, int modulus, int remainder, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      startGetSummariesForPartition_call method_call = new startGetSummariesForPartition_call(tinfo, credentials, request, modulus, remainder, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class startGetSummariesForPartition_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.data.thrift.TSummaries> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      private org.apache.accumulo.core.data.thrift.TSummaryRequest request;
+      private int modulus;
+      private int remainder;
+      public startGetSummariesForPartition_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, int modulus, int remainder, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.request = request;
+        this.modulus = modulus;
+        this.remainder = remainder;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startGetSummariesForPartition", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        startGetSummariesForPartition_args args = new startGetSummariesForPartition_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setRequest(request);
+        args.setModulus(modulus);
+        args.setRemainder(remainder);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public org.apache.accumulo.core.data.thrift.TSummaries getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_startGetSummariesForPartition();
+      }
+    }
+
+    public void startGetSummariesFromFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> files, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      startGetSummariesFromFiles_call method_call = new startGetSummariesFromFiles_call(tinfo, credentials, request, files, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class startGetSummariesFromFiles_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.data.thrift.TSummaries> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      private org.apache.accumulo.core.data.thrift.TSummaryRequest request;
+      private java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> files;
+      public startGetSummariesFromFiles_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> files, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.request = request;
+        this.files = files;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startGetSummariesFromFiles", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        startGetSummariesFromFiles_args args = new startGetSummariesFromFiles_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setRequest(request);
+        args.setFiles(files);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public org.apache.accumulo.core.data.thrift.TSummaries getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_startGetSummariesFromFiles();
+      }
+    }
+
+    public void contiuneGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      contiuneGetSummaries_call method_call = new contiuneGetSummaries_call(tinfo, sessionId, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class contiuneGetSummaries_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.data.thrift.TSummaries> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private long sessionId;
+      public contiuneGetSummaries_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.sessionId = sessionId;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("contiuneGetSummaries", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        contiuneGetSummaries_args args = new contiuneGetSummaries_args();
+        args.setTinfo(tinfo);
+        args.setSessionId(sessionId);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public org.apache.accumulo.core.data.thrift.TSummaries getResult() throws NoSuchScanIDException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_contiuneGetSummaries();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends org.apache.accumulo.core.client.impl.thrift.ClientService.Processor<I> implements org.apache.thrift.TProcessor {
@@ -2264,6 +2555,10 @@ public class TabletClientService {
       processMap.put("getActiveCompactions", new getActiveCompactions());
       processMap.put("removeLogs", new removeLogs());
       processMap.put("getActiveLogs", new getActiveLogs());
+      processMap.put("startGetSummaries", new startGetSummaries());
+      processMap.put("startGetSummariesForPartition", new startGetSummariesForPartition());
+      processMap.put("startGetSummariesFromFiles", new startGetSummariesFromFiles());
+      processMap.put("contiuneGetSummaries", new contiuneGetSummaries());
       return processMap;
     }
 
@@ -2971,6 +3266,104 @@ public class TabletClientService {
       }
     }
 
+    public static class startGetSummaries<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startGetSummaries_args> {
+      public startGetSummaries() {
+        super("startGetSummaries");
+      }
+
+      public startGetSummaries_args getEmptyArgsInstance() {
+        return new startGetSummaries_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public startGetSummaries_result getResult(I iface, startGetSummaries_args args) throws org.apache.thrift.TException {
+        startGetSummaries_result result = new startGetSummaries_result();
+        try {
+          result.success = iface.startGetSummaries(args.tinfo, args.credentials, args.request);
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
+          result.tope = tope;
+        }
+        return result;
+      }
+    }
+
+    public static class startGetSummariesForPartition<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startGetSummariesForPartition_args> {
+      public startGetSummariesForPartition() {
+        super("startGetSummariesForPartition");
+      }
+
+      public startGetSummariesForPartition_args getEmptyArgsInstance() {
+        return new startGetSummariesForPartition_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public startGetSummariesForPartition_result getResult(I iface, startGetSummariesForPartition_args args) throws org.apache.thrift.TException {
+        startGetSummariesForPartition_result result = new startGetSummariesForPartition_result();
+        try {
+          result.success = iface.startGetSummariesForPartition(args.tinfo, args.credentials, args.request, args.modulus, args.remainder);
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class startGetSummariesFromFiles<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startGetSummariesFromFiles_args> {
+      public startGetSummariesFromFiles() {
+        super("startGetSummariesFromFiles");
+      }
+
+      public startGetSummariesFromFiles_args getEmptyArgsInstance() {
+        return new startGetSummariesFromFiles_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public startGetSummariesFromFiles_result getResult(I iface, startGetSummariesFromFiles_args args) throws org.apache.thrift.TException {
+        startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_result();
+        try {
+          result.success = iface.startGetSummariesFromFiles(args.tinfo, args.credentials, args.request, args.files);
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class contiuneGetSummaries<I extends Iface> extends org.apache.thrift.ProcessFunction<I, contiuneGetSummaries_args> {
+      public contiuneGetSummaries() {
+        super("contiuneGetSummaries");
+      }
+
+      public contiuneGetSummaries_args getEmptyArgsInstance() {
+        return new contiuneGetSummaries_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public contiuneGetSummaries_result getResult(I iface, contiuneGetSummaries_args args) throws org.apache.thrift.TException {
+        contiuneGetSummaries_result result = new contiuneGetSummaries_result();
+        try {
+          result.success = iface.contiuneGetSummaries(args.tinfo, args.sessionId);
+        } catch (NoSuchScanIDException nssi) {
+          result.nssi = nssi;
+        }
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends org.apache.accumulo.core.client.impl.thrift.ClientService.AsyncProcessor<I> {
@@ -3015,6 +3408,10 @@ public class TabletClientService {
       processMap.put("getActiveCompactions", new getActiveCompactions());
       processMap.put("removeLogs", new removeLogs());
       processMap.put("getActiveLogs", new getActiveLogs());
+      processMap.put("startGetSummaries", new startGetSummaries());
+      processMap.put("startGetSummariesForPartition", new startGetSummariesForPartition());
+      processMap.put("startGetSummariesFromFiles", new startGetSummariesFromFiles());
+      processMap.put("contiuneGetSummaries", new contiuneGetSummaries());
       return processMap;
     }
 
@@ -4724,6 +5121,270 @@ public class TabletClientService {
       }
     }
 
+    public static class startGetSummaries<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummaries_args, org.apache.accumulo.core.data.thrift.TSummaries> {
+      public startGetSummaries() {
+        super("startGetSummaries");
+      }
+
+      public startGetSummaries_args getEmptyArgsInstance() {
+        return new startGetSummaries_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries>() { 
+          public void onComplete(org.apache.accumulo.core.data.thrift.TSummaries o) {
+            startGetSummaries_result result = new startGetSummaries_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            startGetSummaries_result result = new startGetSummaries_result();
+            if (e instanceof org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException) {
+              result.tope = (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException) e;
+              result.setTopeIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, startGetSummaries_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+        iface.startGetSummaries(args.tinfo, args.credentials, args.request,resultHandler);
+      }
+    }
+
+    public static class startGetSummariesForPartition<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummariesForPartition_args, org.apache.accumulo.core.data.thrift.TSummaries> {
+      public startGetSummariesForPartition() {
+        super("startGetSummariesForPartition");
+      }
+
+      public startGetSummariesForPartition_args getEmptyArgsInstance() {
+        return new startGetSummariesForPartition_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries>() { 
+          public void onComplete(org.apache.accumulo.core.data.thrift.TSummaries o) {
+            startGetSummariesForPartition_result result = new startGetSummariesForPartition_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            startGetSummariesForPartition_result result = new startGetSummariesForPartition_result();
+            if (e instanceof org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, startGetSummariesForPartition_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+        iface.startGetSummariesForPartition(args.tinfo, args.credentials, args.request, args.modulus, args.remainder,resultHandler);
+      }
+    }
+
+    public static class startGetSummariesFromFiles<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummariesFromFiles_args, org.apache.accumulo.core.data.thrift.TSummaries> {
+      public startGetSummariesFromFiles() {
+        super("startGetSummariesFromFiles");
+      }
+
+      public startGetSummariesFromFiles_args getEmptyArgsInstance() {
+        return new startGetSummariesFromFiles_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries>() { 
+          public void onComplete(org.apache.accumulo.core.data.thrift.TSummaries o) {
+            startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_result();
+            if (e instanceof org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, startGetSummariesFromFiles_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+        iface.startGetSummariesFromFiles(args.tinfo, args.credentials, args.request, args.files,resultHandler);
+      }
+    }
+
+    public static class contiuneGetSummaries<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, contiuneGetSummaries_args, org.apache.accumulo.core.data.thrift.TSummaries> {
+      public contiuneGetSummaries() {
+        super("contiuneGetSummaries");
+      }
+
+      public contiuneGetSummaries_args getEmptyArgsInstance() {
+        return new contiuneGetSummaries_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries>() { 
+          public void onComplete(org.apache.accumulo.core.data.thrift.TSummaries o) {
+            contiuneGetSummaries_result result = new contiuneGetSummaries_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            contiuneGetSummaries_result result = new contiuneGetSummaries_result();
+            if (e instanceof NoSuchScanIDException) {
+              result.nssi = (NoSuchScanIDException) e;
+              result.setNssiIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, contiuneGetSummaries_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+        iface.contiuneGetSummaries(args.tinfo, args.sessionId,resultHandler);
+      }
+    }
+
   }
 
   public static class startScan_args implements org.apache.thrift.TBase<startScan_args, startScan_args._Fields>, java.io.Serializable, Cloneable, Comparable<startScan_args>   {
@@ -36245,51 +36906,4622 @@ public class TabletClientService {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getActiveLogs_args.class, metaDataMap);
+    }
+
+    public getActiveLogs_args() {
+    }
+
+    public getActiveLogs_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.security.thrift.TCredentials credentials)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getActiveLogs_args(getActiveLogs_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
+      }
+    }
+
+    public getActiveLogs_args deepCopy() {
+      return new getActiveLogs_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+    }
+
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getActiveLogs_args setTinfo(org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public getActiveLogs_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getActiveLogs_args)
+        return this.equals((getActiveLogs_args)that);
+      return false;
+    }
+
+    public boolean equals(getActiveLogs_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getActiveLogs_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetTinfo()).compareTo(other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetCredentials()).compareTo(other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getActiveLogs_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getActiveLogs_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getActiveLogs_argsStandardScheme getScheme() {
+        return new getActiveLogs_argsStandardScheme();
+      }
+    }
+
+    private static class getActiveLogs_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getActiveLogs_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveLogs_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveLogs_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getActiveLogs_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getActiveLogs_argsTupleScheme getScheme() {
+        return new getActiveLogs_argsTupleScheme();
+      }
+    }
+
+    private static class getActiveLogs_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getActiveLogs_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class getActiveLogs_result implements org.apache.thrift.TBase<getActiveLogs_result, getActiveLogs_result._Fields>, java.io.Serializable, Cloneable, Comparable<getActiveLogs_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getActiveLogs_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveLogs_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveLogs_resultTupleSchemeFactory();
+
+    public java.util.List<java.lang.String> success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getActiveLogs_result.class, metaDataMap);
+    }
+
+    public getActiveLogs_result() {
+    }
+
+    public getActiveLogs_result(
+      java.util.List<java.lang.String> success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getActiveLogs_result(getActiveLogs_result other) {
+      if (other.isSetSuccess()) {
+        java.util.List<java.lang.String> __this__success = new java.util.ArrayList<java.lang.String>(other.success);
+        this.success = __this__success;
+      }
+    }
+
+    public getActiveLogs_result deepCopy() {
+      return new getActiveLogs_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    public java.util.Iterator<java.lang.String> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(java.lang.String elem) {
+      if (this.success == null) {
+        this.success = new java.util.ArrayList<java.lang.String>();
+      }
+      this.success.add(elem);
+    }
+
+    public java.util.List<java.lang.String> getSuccess() {
+      return this.success;
+    }
+
+    public getActiveLogs_result setSuccess(java.util.List<java.lang.String> success) {
+      this.success = success;
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.util.List<java.lang.String>)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getActiveLogs_result)
+        return this.equals((getActiveLogs_result)that);
+      return false;
+    }
+
+    public boolean equals(getActiveLogs_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getActiveLogs_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getActiveLogs_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getActiveLogs_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getActiveLogs_resultStandardScheme getScheme() {
+        return new getActiveLogs_resultStandardScheme();
+      }
+    }
+
+    private static class getActiveLogs_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getActiveLogs_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveLogs_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list332 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list332.size);
+                  java.lang.String _elem333;
+                  for (int _i334 = 0; _i334 < _list332.size; ++_i334)
+                  {
+                    _elem333 = iprot.readString();
+                    struct.success.add(_elem333);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveLogs_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
+            for (java.lang.String _iter335 : struct.success)
+            {
+              oprot.writeString(_iter335);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getActiveLogs_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getActiveLogs_resultTupleScheme getScheme() {
+        return new getActiveLogs_resultTupleScheme();
+      }
+    }
+
+    private static class getActiveLogs_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getActiveLogs_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (java.lang.String _iter336 : struct.success)
+            {
+              oprot.writeString(_iter336);
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          {
+            org.apache.thrift.protocol.TList _list337 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new java.util.ArrayList<java.lang.String>(_list337.size);
+            java.lang.String _elem338;
+            for (int _i339 = 0; _i339 < _list337.size; ++_i339)
+            {
+              _elem338 = iprot.readString();
+              struct.success.add(_elem338);
+            }
+          }
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class startGetSummaries_args implements org.apache.thrift.TBase<startGetSummaries_args, startGetSummaries_args._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummaries_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummaries_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("request", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startGetSummaries_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummaries_argsTupleSchemeFactory();
+
+    public org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
+    public org.apache.accumulo.core.data.thrift.TSummaryRequest request; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)2, "credentials"),
+      REQUEST((short)3, "request");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          case 3: // REQUEST
+            return REQUEST;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.REQUEST, new org.apache.thrift.meta_data.FieldMetaData("request", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.data.thrift.TSummaryRequest.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startGetSummaries_args.class, metaDataMap);
+    }
+
+    public startGetSummaries_args() {
+    }
+
+    public startGetSummaries_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.security.thrift.TCredentials credentials,
+      org.apache.accumulo.core.data.thrift.TSummaryRequest request)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.request = request;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public startGetSummaries_args(startGetSummaries_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetRequest()) {
+        this.request = new org.apache.accumulo.core.data.thrift.TSummaryRequest(other.request);
+      }
+    }
+
+    public startGetSummaries_args deepCopy() {
+      return new startGetSummaries_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.request = null;
+    }
+
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public startGetSummaries_args setTinfo(org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public startGetSummaries_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public org.apache.accumulo.core.data.thrift.TSummaryRequest getRequest() {
+      return this.request;
+    }
+
+    public startGetSummaries_args setRequest(org.apache.accumulo.core.data.thrift.TSummaryRequest request) {
+      this.request = request;
+      return this;
+    }
+
+    public void unsetRequest() {
+      this.request = null;
+    }
+
+    /** Returns true if field request is set (has been assigned a value) and false otherwise */
+    public boolean isSetRequest() {
+      return this.request != null;
+    }
+
+    public void setRequestIsSet(boolean value) {
+      if (!value) {
+        this.request = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
+        }
+        break;
+
+      case REQUEST:
+        if (value == null) {
+          unsetRequest();
+        } else {
+          setRequest((org.apache.accumulo.core.data.thrift.TSummaryRequest)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case REQUEST:
+        return getRequest();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case REQUEST:
+        return isSetRequest();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof startGetSummaries_args)
+        return this.equals((startGetSummaries_args)that);
+      return false;
+    }
+
+    public boolean equals(startGetSummaries_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_request = true && this.isSetRequest();
+      boolean that_present_request = true && that.isSetRequest();
+      if (this_present_request || that_present_request) {
+        if (!(this_present_request && that_present_request))
+          return false;
+        if (!this.request.equals(that.request))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetRequest()) ? 131071 : 524287);
+      if (isSetRequest())
+        hashCode = hashCode * 8191 + request.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(startGetSummaries_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetTinfo()).compareTo(other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetCredentials()).compareTo(other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetRequest()).compareTo(other.isSetRequest());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRequest()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request, other.request);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("startGetSummaries_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("request:");
+      if (this.request == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.request);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+      if (request != null) {
+        request.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class startGetSummaries_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public startGetSummaries_argsStandardSc

<TRUNCATED>

[2/9] accumulo git commit: ACCUMULO-4501 ACCUMULO-96 Added Summarization

Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
index 08bff26..e29d30c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
@@ -17,9 +17,20 @@
 package org.apache.accumulo.tserver.compaction;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
+import java.util.function.Predicate;
 
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.mapred.AccumuloFileOutputFormat;
+import org.apache.accumulo.core.client.rfile.RFile.WriterOptions;
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.Summarizer.Combiner;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.Summary;
+import org.apache.accumulo.core.client.summary.Summary.FileStatistics;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.TabletId;
@@ -27,11 +38,21 @@ import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.data.impl.TabletIdImpl;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.file.blockfile.cache.BlockCache;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.summary.SummaryReader;
+import org.apache.accumulo.core.summary.Gatherer;
+import org.apache.accumulo.core.summary.SummarizerFactory;
+import org.apache.accumulo.core.summary.SummaryCollection;
+import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.tserver.compaction.strategies.TooManyDeletesCompactionStrategy;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Information that can be used to determine how a tablet is to be major compacted, if needed.
  */
@@ -40,18 +61,27 @@ public class MajorCompactionRequest implements Cloneable {
   final private MajorCompactionReason reason;
   final private VolumeManager volumeManager;
   final private AccumuloConfiguration tableConfig;
+  final private BlockCache indexCache;
+  final private BlockCache summaryCache;
   private Map<FileRef,DataFileValue> files;
 
-  public MajorCompactionRequest(KeyExtent extent, MajorCompactionReason reason, VolumeManager manager, AccumuloConfiguration tabletConfig) {
+  public MajorCompactionRequest(KeyExtent extent, MajorCompactionReason reason, VolumeManager manager, AccumuloConfiguration tabletConfig,
+      BlockCache summaryCache, BlockCache indexCache) {
     this.extent = extent;
     this.reason = reason;
     this.volumeManager = manager;
     this.tableConfig = tabletConfig;
     this.files = Collections.emptyMap();
+    this.summaryCache = summaryCache;
+    this.indexCache = indexCache;
+  }
+
+  public MajorCompactionRequest(KeyExtent extent, MajorCompactionReason reason, AccumuloConfiguration tabletConfig) {
+    this(extent, reason, null, tabletConfig, null, null);
   }
 
   public MajorCompactionRequest(MajorCompactionRequest mcr) {
-    this(mcr.extent, mcr.reason, mcr.volumeManager, mcr.tableConfig);
+    this(mcr.extent, mcr.reason, mcr.volumeManager, mcr.tableConfig, mcr.summaryCache, mcr.indexCache);
     // know this is already unmodifiable, no need to wrap again
     this.files = mcr.files;
   }
@@ -68,11 +98,61 @@ public class MajorCompactionRequest implements Cloneable {
     return files;
   }
 
+  /**
+   * Returns all summaries present in each file.
+   *
+   * <p>
+   * This method can only be called from {@link CompactionStrategy#gatherInformation(MajorCompactionRequest)}. Unfortunately, {@code gatherInformation()} is not
+   * called before {@link CompactionStrategy#shouldCompact(MajorCompactionRequest)}. Therefore {@code shouldCompact()) should just return true when a compactions strategy
+   * wants to use summary information.
+   *
+   * <p>
+   * When using summaries to make compaction decisions, its important to ensure that all summary data fits in the tablet server summary cache. The size of this
+   * cache is configured by code tserver.cache.summary.size}. Also its important to use the summarySelector predicate to only retrieve the needed summary data.
+   * Otherwise uneeded summary data could be brought into the cache.
+   *
+   * <p>
+   * Some files may contain data outside of a tablets range. When {@link Summarizer}'s generate small amounts of summary data, multiple summaries may be stored
+   * within a file for different row ranges. This will allow more accurate summaries to be returned for the case where a file has data outside a tablets range.
+   * However, some summary data outside of the tablets range may still be included. When this happens {@link FileStatistics#getExtra()} will be non zero. Also,
+   * its good to be aware of the other potential causes of inaccuracies {@link FileStatistics#getInaccurate()}
+   *
+   * <p>
+   * When this method is called with multiple files, it will automatically merge summary data using {@link Combiner#merge(Map, Map)}. If summary information is
+   * needed for each file, then just call this method for each file.
+   *
+   * <p>
+   * Writing a compaction strategy that uses summary information is a bit tricky. See the source code for {@link TooManyDeletesCompactionStrategy} as an example
+   * of a compaction strategy.
+   *
+   * @see Summarizer
+   * @see TableOperations#addSummarizers(String, SummarizerConfiguration...)
+   * @see AccumuloFileOutputFormat#setSummarizers(org.apache.hadoop.mapred.JobConf, SummarizerConfiguration...)
+   * @see WriterOptions#withSummarizers(SummarizerConfiguration...)
+   */
+  public List<Summary> getSummaries(Collection<FileRef> files, Predicate<SummarizerConfiguration> summarySelector) throws IOException {
+    Preconditions.checkState(volumeManager != null,
+        "Getting summaries is not supported at this time.  Its only supported when CompactionStrategy.gatherInformation() is called.");
+    SummaryCollection sc = new SummaryCollection();
+    SummarizerFactory factory = new SummarizerFactory(tableConfig);
+    for (FileRef file : files) {
+      FileSystem fs = volumeManager.getVolumeByPath(file.path()).getFileSystem();
+      Configuration conf = CachedConfiguration.getInstance();
+      SummaryCollection fsc = SummaryReader.load(fs, conf, tableConfig, factory, file.path(), summarySelector, summaryCache, indexCache).getSummaries(
+          Collections.singletonList(new Gatherer.RowRange(extent)));
+      sc.merge(fsc, factory);
+    }
+
+    return sc.getSummaries();
+  }
+
   public void setFiles(Map<FileRef,DataFileValue> update) {
     this.files = Collections.unmodifiableMap(update);
   }
 
   public FileSKVIterator openReader(FileRef ref) throws IOException {
+    Preconditions.checkState(volumeManager != null,
+        "Opening files is not supported at this time.  Its only supported when CompactionStrategy.gatherInformation() is called.");
     // @TODO verify the file isn't some random file in HDFS
     // @TODO ensure these files are always closed?
     FileOperations fileFactory = FileOperations.getInstance();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java
index 5ec175b..9bd5dd3 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java
@@ -18,8 +18,11 @@
 package org.apache.accumulo.tserver.compaction.strategies;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -27,6 +30,8 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.regex.Pattern;
 
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.Summary;
 import org.apache.accumulo.core.compaction.CompactionSettings;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.file.FileSKVIterator;
@@ -48,6 +53,84 @@ public class ConfigurableCompactionStrategy extends CompactionStrategy {
     abstract boolean shouldCompact(Entry<FileRef,DataFileValue> file, MajorCompactionRequest request);
   }
 
+  private static class SummaryTest extends Test {
+
+    private boolean selectExtraSummary;
+    private boolean selectNoSummary;
+
+    private boolean summaryConfigured = true;
+    private boolean gatherCalled = false;
+
+    // files that do not need compaction
+    private Set<FileRef> okFiles = Collections.emptySet();
+
+    public SummaryTest(boolean selectExtraSummary, boolean selectNoSummary) {
+      this.selectExtraSummary = selectExtraSummary;
+      this.selectNoSummary = selectNoSummary;
+    }
+
+    @Override
+    void gatherInformation(MajorCompactionRequest request) {
+      gatherCalled = true;
+      Collection<SummarizerConfiguration> configs = SummarizerConfiguration.fromTableProperties(request.getTableProperties());
+      if (configs.size() == 0) {
+        summaryConfigured = false;
+      } else {
+        Set<SummarizerConfiguration> configsSet = configs instanceof Set ? (Set<SummarizerConfiguration>) configs : new HashSet<>(configs);
+        okFiles = new HashSet<>();
+
+        for (FileRef fref : request.getFiles().keySet()) {
+          Map<SummarizerConfiguration,Summary> sMap = new HashMap<>();
+          Collection<Summary> summaries;
+          try {
+            summaries = request.getSummaries(Collections.singletonList(fref), conf -> configsSet.contains(conf));
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+          for (Summary summary : summaries) {
+            sMap.put(summary.getSummarizerConfiguration(), summary);
+          }
+
+          boolean needsCompaction = false;
+          for (SummarizerConfiguration sc : configs) {
+            Summary summary = sMap.get(sc);
+
+            if (summary == null && selectNoSummary) {
+              needsCompaction = true;
+              break;
+            }
+
+            if (summary != null && summary.getFileStatistics().getExtra() > 0 && selectExtraSummary) {
+              needsCompaction = true;
+              break;
+            }
+          }
+
+          if (!needsCompaction) {
+            okFiles.add(fref);
+          }
+        }
+      }
+
+    }
+
+    @Override
+    public boolean shouldCompact(Entry<FileRef,DataFileValue> file, MajorCompactionRequest request) {
+
+      if (!gatherCalled) {
+        Collection<SummarizerConfiguration> configs = SummarizerConfiguration.fromTableProperties(request.getTableProperties());
+        return configs.size() > 0;
+      }
+
+      if (!summaryConfigured) {
+        return false;
+      }
+
+      // Its possible the set of files could change between gather and now. So this will default to compacting any files that are unknown.
+      return !okFiles.contains(file.getKey());
+    }
+  }
+
   private static class NoSampleTest extends Test {
 
     private Set<FileRef> filesWithSample = Collections.emptySet();
@@ -69,7 +152,7 @@ public class ConfigurableCompactionStrategy extends CompactionStrategy {
               filesWithSample.add(fref);
             }
           } catch (IOException e) {
-            throw new RuntimeException(e);
+            throw new UncheckedIOException(e);
           }
         }
       }
@@ -130,10 +213,19 @@ public class ConfigurableCompactionStrategy extends CompactionStrategy {
   @Override
   public void init(Map<String,String> options) {
 
+    boolean selectNoSummary = false;
+    boolean selectExtraSummary = false;
+
     Set<Entry<String,String>> es = options.entrySet();
     for (Entry<String,String> entry : es) {
 
       switch (CompactionSettings.valueOf(entry.getKey())) {
+        case SF_EXTRA_SUMMARY:
+          selectExtraSummary = true;
+          break;
+        case SF_NO_SUMMARY:
+          selectNoSummary = true;
+          break;
         case SF_NO_SAMPLE:
           tests.add(new NoSampleTest());
           break;
@@ -191,6 +283,11 @@ public class ConfigurableCompactionStrategy extends CompactionStrategy {
           throw new IllegalArgumentException("Unknown option " + entry.getKey());
       }
     }
+
+    if (selectExtraSummary || selectNoSummary) {
+      tests.add(new SummaryTest(selectExtraSummary, selectNoSummary));
+    }
+
   }
 
   private List<FileRef> getFilesToCompact(MajorCompactionRequest request) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/TooManyDeletesCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/TooManyDeletesCompactionStrategy.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/TooManyDeletesCompactionStrategy.java
new file mode 100644
index 0000000..6399623
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/TooManyDeletesCompactionStrategy.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.tserver.compaction.strategies;
+
+import static org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer.DELETES_STAT;
+import static org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer.TOTAL_STAT;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.function.Predicate;
+
+import org.apache.accumulo.core.client.mapred.AccumuloFileOutputFormat;
+import org.apache.accumulo.core.client.rfile.RFile.WriterOptions;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.client.summary.Summary;
+import org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.tserver.compaction.CompactionPlan;
+import org.apache.accumulo.tserver.compaction.DefaultCompactionStrategy;
+import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This compaction strategy works in concert with the {@link DeletesSummarizer}. Using the statistics from DeleteSummarizer this strategy will compact all files
+ * in a table when the number of deletes/non-deletes exceeds a threshold.
+ *
+ * <p>
+ * This strategy has two options. First the {@value #THRESHOLD_OPT} option allows setting the point at which a compaction will be triggered. This options
+ * defaults to {@value #THRESHOLD_OPT_DEFAULT} and must be in the range (0.0, 1.0]. The second option is {@value #PROCEED_ZERO_NO_SUMMARY_OPT} which determines
+ * if the strategy should proceed when a bulk imported file has no summary information.
+ *
+ * <p>
+ * If the delete summarizer was configured on a table that already had files, then those files will have not summary information. This strategy can still
+ * proceed in this situation. It will fall back to using Accumulo's estimated entires per file in this case. For the files without summary information the
+ * estimated number of deletes will be zero. This fall back method will underestimate deletes which will not lead to false positives, except for the case of
+ * bulk imported files. Accumulo estimates that bulk imported files have zero entires. The second option {@value #PROCEED_ZERO_NO_SUMMARY_OPT} determines if
+ * this strategy should proceed when it sees bulk imported files that do not have summary data. This option defaults to
+ * {@value #PROCEED_ZERO_NO_SUMMARY_OPT_DEFAULT}.
+ *
+ * <p>
+ * Bulk files can be generated with summary information by calling
+ * {@link AccumuloFileOutputFormat#setSummarizers(org.apache.hadoop.mapred.JobConf, SummarizerConfiguration...)} or
+ * {@link WriterOptions#withSummarizers(SummarizerConfiguration...)}
+ *
+ * <p>
+ * When this strategy does not decide to compact based on the number of deletes, then it will defer the decision to the {@link DefaultCompactionStrategy}.
+ *
+ * <p>
+ * Configuring this compaction strategy for a table will cause it to always queue compactions, even though it may not decide to compact. These queued
+ * compactions may show up on the Accumulo monitor page. This is because summary data can not be read until after compaction is queued and dequeued. When the
+ * compaction is dequeued it can then decide not to compact. See <a href=https://issues.apache.org/jira/browse/ACCUMULO-4573>ACCUMULO-4573</a>
+ *
+ * @since 2.0.0
+ */
+public class TooManyDeletesCompactionStrategy extends DefaultCompactionStrategy {
+
+  private boolean shouldCompact = false;
+
+  Logger log = LoggerFactory.getLogger(TooManyDeletesCompactionStrategy.class);
+
+  private double threshold;
+
+  private boolean proceed_bns;
+
+  /**
+   * This option should be a floating point number between 1 and 0.
+   */
+  public static final String THRESHOLD_OPT = "threshold";
+
+  /**
+   * The default threshold.
+   */
+  public static final String THRESHOLD_OPT_DEFAULT = ".25";
+
+  public static final String PROCEED_ZERO_NO_SUMMARY_OPT = "proceed_zero_no_summary";
+
+  public static final String PROCEED_ZERO_NO_SUMMARY_OPT_DEFAULT = "false";
+
+  public void init(Map<String,String> options) {
+    this.threshold = Double.parseDouble(options.getOrDefault(THRESHOLD_OPT, THRESHOLD_OPT_DEFAULT));
+    if (threshold <= 0.0 || threshold > 1.0) {
+      throw new IllegalArgumentException("Threshold must be in range (0.0, 1.0], saw : " + threshold);
+    }
+
+    this.proceed_bns = Boolean.parseBoolean(options.getOrDefault(PROCEED_ZERO_NO_SUMMARY_OPT, PROCEED_ZERO_NO_SUMMARY_OPT_DEFAULT));
+  }
+
+  @Override
+  public boolean shouldCompact(MajorCompactionRequest request) {
+    Collection<SummarizerConfiguration> configuredSummarizers = SummarizerConfiguration.fromTableProperties(request.getTableProperties());
+
+    // check if delete summarizer is configured for table
+    if (configuredSummarizers.stream().map(sc -> sc.getClassName()).anyMatch(cn -> cn.equals(DeletesSummarizer.class.getName()))) {
+      // This is called before gatherInformation, so need to always queue for compaction until info can be gathered. Also its not safe to request summary
+      // information here as its a blocking operation. Blocking operations are not allowed in shouldCompact.
+      return true;
+    } else {
+      return super.shouldCompact(request);
+    }
+  }
+
+  @Override
+  public void gatherInformation(MajorCompactionRequest request) throws IOException {
+    super.gatherInformation(request);
+
+    Predicate<SummarizerConfiguration> summarizerPredicate = conf -> conf.getClassName().equals(DeletesSummarizer.class.getName())
+        && conf.getOptions().isEmpty();
+
+    long total = 0;
+    long deletes = 0;
+
+    for (Entry<FileRef,DataFileValue> entry : request.getFiles().entrySet()) {
+      Collection<Summary> summaries = request.getSummaries(Collections.singleton(entry.getKey()), summarizerPredicate);
+      if (summaries.size() == 1) {
+        Summary summary = summaries.iterator().next();
+        total += summary.getStatistics().get(TOTAL_STAT);
+        deletes += summary.getStatistics().get(DELETES_STAT);
+      } else {
+        long numEntries = entry.getValue().getNumEntries();
+        if (numEntries == 0 && !proceed_bns) {
+          shouldCompact = false;
+          return;
+        } else {
+          // no summary data so use Accumulo's estimate of total entries in file
+          total += entry.getValue().getNumEntries();
+        }
+      }
+    }
+
+    long nonDeletes = total - deletes;
+
+    if (nonDeletes >= 0) {
+      // check nonDeletes >= 0 because if this is not true then its clear evidence that the estimates are off
+
+      double ratio = deletes / (double) nonDeletes;
+      shouldCompact = ratio >= threshold;
+    } else {
+      shouldCompact = false;
+    }
+  }
+
+  @Override
+  public CompactionPlan getCompactionPlan(MajorCompactionRequest request) {
+    if (shouldCompact) {
+      CompactionPlan cp = new CompactionPlan();
+      cp.inputFiles.addAll(request.getFiles().keySet());
+      return cp;
+    }
+
+    // fall back to default
+    return super.getCompactionPlan(request);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SummarySession.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SummarySession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SummarySession.java
new file mode 100644
index 0000000..7d9145e
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SummarySession.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.tserver.session;
+
+import java.util.concurrent.Future;
+
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.summary.SummaryCollection;
+
+public class SummarySession extends Session {
+
+  private Future<SummaryCollection> future;
+
+  public SummarySession(TCredentials credentials, Future<SummaryCollection> future) {
+    super(credentials);
+    this.future = future;
+  }
+
+  public Future<SummaryCollection> getFuture() {
+    return future;
+  }
+
+  @Override
+  public boolean cleanup() {
+    return future.cancel(true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index f0c0695..e4d32a6 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -73,6 +73,7 @@ import org.apache.accumulo.core.data.thrift.IterInfo;
 import org.apache.accumulo.core.data.thrift.MapFileInfo;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.file.blockfile.cache.BlockCache;
 import org.apache.accumulo.core.iterators.IterationInterruptedException;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -1667,7 +1668,9 @@ public class Tablet implements TabletCommitter {
     }
 
     if (strategy != null) {
-      MajorCompactionRequest request = new MajorCompactionRequest(extent, reason, getTabletServer().getFileSystem(), tableConfiguration);
+      BlockCache sc = tabletResources.getTabletServerResourceManager().getSummaryCache();
+      BlockCache ic = tabletResources.getTabletServerResourceManager().getIndexCache();
+      MajorCompactionRequest request = new MajorCompactionRequest(extent, reason, getTabletServer().getFileSystem(), tableConfiguration, sc, ic);
       request.setFiles(getDatafileManager().getDatafileSizes());
       strategy.gatherInformation(request);
     }
@@ -1711,7 +1714,7 @@ public class Tablet implements TabletCommitter {
         // enforce rules: files with keys outside our range need to be compacted
         inputFiles.addAll(findChopFiles(extent, firstAndLastKeys, allFiles.keySet()));
       } else {
-        MajorCompactionRequest request = new MajorCompactionRequest(extent, reason, fs, tableConfiguration);
+        MajorCompactionRequest request = new MajorCompactionRequest(extent, reason, tableConfiguration);
         request.setFiles(allFiles);
         plan = strategy.getCompactionPlan(request);
         if (plan != null) {
@@ -2463,7 +2466,7 @@ public class Tablet implements TabletCommitter {
       CompactionStrategyConfig strategyConfig = compactionConfig.getCompactionStrategy();
       CompactionStrategy strategy = createCompactionStrategy(strategyConfig);
 
-      MajorCompactionRequest request = new MajorCompactionRequest(extent, MajorCompactionReason.USER, getTabletServer().getFileSystem(), tableConfiguration);
+      MajorCompactionRequest request = new MajorCompactionRequest(extent, MajorCompactionReason.USER, tableConfiguration);
       request.setFiles(getDatafileManager().getDatafileSizes());
 
       try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java
index e54e1c8..0cb2ab2 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java
@@ -151,7 +151,7 @@ public class DefaultCompactionStrategyTest {
     }
 
     TestCompactionRequest(KeyExtent extent, MajorCompactionReason reason, Map<FileRef,DataFileValue> files) {
-      super(extent, reason, null, dfault);
+      super(extent, reason, dfault);
       setFiles(files);
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.java
index 648f451..91e9860 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.java
@@ -51,7 +51,7 @@ public class SizeLimitCompactionStrategyTest {
     slcs.init(opts);
 
     KeyExtent ke = new KeyExtent("0", null, null);
-    MajorCompactionRequest mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, null, AccumuloConfiguration.getDefaultConfiguration());
+    MajorCompactionRequest mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, AccumuloConfiguration.getDefaultConfiguration());
 
     mcr.setFiles(nfl("f1", "2G", "f2", "2G", "f3", "2G", "f4", "2G"));
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/TwoTierCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/TwoTierCompactionStrategyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/TwoTierCompactionStrategyTest.java
index 6fb37da..be73cb2 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/TwoTierCompactionStrategyTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/TwoTierCompactionStrategyTest.java
@@ -62,7 +62,7 @@ public class TwoTierCompactionStrategyTest {
     ttcs.init(opts);
     conf = AccumuloConfiguration.getDefaultConfiguration();
     KeyExtent ke = new KeyExtent("0", null, null);
-    mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, null, conf);
+    mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, conf);
     Map<FileRef,DataFileValue> fileMap = createFileMap("f1", "10M", "f2", "10M", "f3", "10M", "f4", "10M", "f5", "100M", "f6", "100M", "f7", "100M", "f8",
         "100M");
     mcr.setFiles(fileMap);
@@ -81,7 +81,7 @@ public class TwoTierCompactionStrategyTest {
     ttcs.init(opts);
     conf = AccumuloConfiguration.getDefaultConfiguration();
     KeyExtent ke = new KeyExtent("0", null, null);
-    mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, null, conf);
+    mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, conf);
     Map<FileRef,DataFileValue> fileMap = createFileMap("f1", "2G", "f2", "2G", "f3", "2G", "f4", "2G");
     mcr.setFiles(fileMap);
 
@@ -110,7 +110,7 @@ public class TwoTierCompactionStrategyTest {
     ttcs.init(opts);
     conf = AccumuloConfiguration.getDefaultConfiguration();
     KeyExtent ke = new KeyExtent("0", null, null);
-    mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, null, conf);
+    mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, conf);
     Map<FileRef,DataFileValue> fileMap = createFileMap("f1", "1G", "f2", "10M", "f3", "10M", "f4", "10M", "f5", "10M", "f6", "10M", "f7", "10M");
     Map<FileRef,DataFileValue> filesToCompactMap = createFileMap("f2", "10M", "f3", "10M", "f4", "10M", "f5", "10M", "f6", "10M", "f7", "10M");
     mcr.setFiles(fileMap);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java
index d2a1fe4..ce43665 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java
@@ -37,7 +37,7 @@ public class ConfigurableCompactionStrategyTest {
 
   @Test
   public void testOutputOptions() throws Exception {
-    MajorCompactionRequest mcr = new MajorCompactionRequest(new KeyExtent("1", null, null), MajorCompactionReason.USER, null, null);
+    MajorCompactionRequest mcr = new MajorCompactionRequest(new KeyExtent("1", null, null), MajorCompactionReason.USER, null);
 
     Map<FileRef,DataFileValue> files = new HashMap<>();
     files.put(new FileRef("hdfs://nn1/accumulo/tables/1/t-009/F00001.rf"), new DataFileValue(50000, 400));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/shell/src/main/java/org/apache/accumulo/shell/Shell.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/accumulo/shell/Shell.java b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
index 73f4a42..4649a7e 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/Shell.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
@@ -152,6 +152,7 @@ import org.apache.accumulo.shell.commands.SetIterCommand;
 import org.apache.accumulo.shell.commands.SetScanIterCommand;
 import org.apache.accumulo.shell.commands.SetShellIterCommand;
 import org.apache.accumulo.shell.commands.SleepCommand;
+import org.apache.accumulo.shell.commands.SummariesCommand;
 import org.apache.accumulo.shell.commands.SystemPermissionsCommand;
 import org.apache.accumulo.shell.commands.TableCommand;
 import org.apache.accumulo.shell.commands.TablePermissionsCommand;
@@ -415,7 +416,8 @@ public class Shell extends ShellOptions implements KeywordExecutable {
         new TableCommand(), new UserCommand(), new WhoAmICommand()};
     Command[] tableCommands = {new CloneTableCommand(), new ConfigCommand(), new CreateTableCommand(), new DeleteTableCommand(), new DropTableCommand(),
         new DUCommand(), new ExportTableCommand(), new ImportTableCommand(), new OfflineCommand(), new OnlineCommand(), new RenameTableCommand(),
-        new TablesCommand(), new NamespacesCommand(), new CreateNamespaceCommand(), new DeleteNamespaceCommand(), new RenameNamespaceCommand()};
+        new TablesCommand(), new NamespacesCommand(), new CreateNamespaceCommand(), new DeleteNamespaceCommand(), new RenameNamespaceCommand(),
+        new SummariesCommand()};
     Command[] tableControlCommands = {new AddSplitsCommand(), new CompactCommand(), new ConstraintCommand(), new FlushCommand(), new GetGroupsCommand(),
         new GetSplitsCommand(), new MergeCommand(), new SetGroupsCommand()};
     Command[] userCommands = {new AddAuthsCommand(), new CreateUserCommand(), new DeleteUserCommand(), new DropUserCommand(), new GetAuthsCommand(),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java
index c8b0e11..31003b0 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java
@@ -38,7 +38,7 @@ public class CompactCommand extends TableOperation {
 
   // file selection and file output options
   private Option enameOption, epathOption, sizeLtOption, sizeGtOption, minFilesOption, outBlockSizeOpt, outHdfsBlockSizeOpt, outIndexBlockSizeOpt,
-      outCompressionOpt, outReplication, enoSampleOption;
+      outCompressionOpt, outReplication, enoSampleOption, extraSummaryOption, enoSummaryOption;
 
   private CompactionConfig compactionConfig = null;
 
@@ -89,6 +89,8 @@ public class CompactCommand extends TableOperation {
   private Map<String,String> getConfigurableCompactionStrategyOpts(CommandLine cl) {
     Map<String,String> opts = new HashMap<>();
 
+    put(cl, opts, extraSummaryOption, CompactionSettings.SF_EXTRA_SUMMARY);
+    put(cl, opts, enoSummaryOption, CompactionSettings.SF_NO_SUMMARY);
     put(cl, opts, enoSampleOption, CompactionSettings.SF_NO_SAMPLE);
     put(cl, opts, enameOption, CompactionSettings.SF_NAME_RE_OPT);
     put(cl, opts, epathOption, CompactionSettings.SF_PATH_RE_OPT);
@@ -191,6 +193,10 @@ public class CompactCommand extends TableOperation {
     cancelOpt = new Option(null, "cancel", false, "cancel user initiated compactions");
     opts.addOption(cancelOpt);
 
+    enoSummaryOption = new Option(null, "sf-no-summary", false, "Select files that do not have the summaries specified in the table configuration.");
+    opts.addOption(enoSummaryOption);
+    extraSummaryOption = new Option(null, "sf-extra-summary", false, "Select files that have summary information which exceeds the tablets boundries.");
+    opts.addOption(extraSummaryOption);
     enoSampleOption = new Option(null, "sf-no-sample", false,
         "Select files that have no sample data or sample data that differes from the table configuration.");
     opts.addOption(enoSampleOption);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/shell/src/main/java/org/apache/accumulo/shell/commands/SummariesCommand.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/SummariesCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/SummariesCommand.java
new file mode 100644
index 0000000..e0e6d2a
--- /dev/null
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/SummariesCommand.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.shell.commands;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.SummaryRetriever;
+import org.apache.accumulo.core.client.summary.Summary;
+import org.apache.accumulo.shell.Shell;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.io.Text;
+
+public class SummariesCommand extends TableOperation {
+
+  private Text startRow;
+  private Text endRow;
+  private boolean paginate;
+  private String selectionRegex = ".*";
+
+  private Option disablePaginationOpt;
+  private Option summarySelectionOpt;
+
+  @Override
+  public String description() {
+    return "retrieves summary statistics";
+  }
+
+  @Override
+  protected void doTableOp(final Shell shellState, final String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException,
+      IOException {
+    Connector conn = shellState.getConnector();
+    SummaryRetriever retriever = conn.tableOperations().summaries(tableName).withMatchingConfiguration(selectionRegex);
+    if (startRow != null) {
+      retriever.startRow(startRow);
+    }
+
+    if (endRow != null) {
+      retriever.endRow(endRow);
+    }
+
+    Collection<Summary> summaries = retriever.retrieve();
+
+    ArrayList<String> lines = new ArrayList<>();
+
+    boolean addEmpty = false;
+    for (Summary summary : summaries) {
+      if (addEmpty)
+        lines.add("");
+      addEmpty = true;
+      lines.add(String.format(" Summarizer         : %s", summary.getSummarizerConfiguration()));
+      lines.add(String.format(" File Statistics    : %s", summary.getFileStatistics()));
+      lines.add(String.format(" Summary Statistics : "));
+
+      Map<String,Long> stats = summary.getStatistics();
+      ArrayList<String> keys = new ArrayList<>(stats.keySet());
+      Collections.sort(keys);
+      for (String key : keys) {
+        lines.add(String.format("    %-60s = %,d", key, stats.get(key)));
+      }
+    }
+
+    shellState.printLines(lines.iterator(), paginate);
+
+  }
+
+  @Override
+  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
+    startRow = OptUtil.getStartRow(cl);
+    endRow = OptUtil.getEndRow(cl);
+    paginate = !cl.hasOption(disablePaginationOpt.getOpt());
+    if (cl.hasOption(summarySelectionOpt.getOpt())) {
+      selectionRegex = cl.getOptionValue(summarySelectionOpt.getOpt());
+    } else {
+      selectionRegex = ".*";
+    }
+    return super.execute(fullCommand, cl, shellState);
+  }
+
+  @Override
+  public Options getOptions() {
+    final Options opts = super.getOptions();
+    disablePaginationOpt = new Option("np", "no-pagination", false, "disable pagination of output");
+    summarySelectionOpt = new Option("sr", "select-regex", true,
+        "regex to select summaries. Matches against class name and options used to generate summaries.");
+    opts.addOption(disablePaginationOpt);
+    opts.addOption(summarySelectionOpt);
+    opts.addOption(OptUtil.startRowOpt());
+    opts.addOption(OptUtil.endRowOpt());
+    return opts;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
index eae5ca9..d6d83cc 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
@@ -54,6 +54,8 @@ import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer;
+import org.apache.accumulo.core.client.summary.summarizers.FamilySummarizer;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -1487,7 +1489,7 @@ public class ShellServerIT extends SharedMiniClusterBase {
     ts.exec("systempermissions");
     assertEquals(12, ts.output.get().split("\n").length - 1);
     ts.exec("tablepermissions", true);
-    assertEquals(6, ts.output.get().split("\n").length - 1);
+    assertEquals(7, ts.output.get().split("\n").length - 1);
   }
 
   @Test
@@ -1894,4 +1896,129 @@ public class ShellServerIT extends SharedMiniClusterBase {
     return null;
   }
 
+  private static void assertMatches(String output, String pattern) {
+    Assert.assertTrue("Pattern " + pattern + " did not match output : " + output, output.matches(pattern));
+  }
+
+  private static void assertNotContains(String output, String subsequence) {
+    Assert.assertFalse("Expected '" + subsequence + "' would not occur in output : " + output, output.contains(subsequence));
+  }
+
+  @Test
+  public void testSummaries() throws Exception {
+    ts.exec("createtable summary");
+    ts.exec("config -t summary -s table.summarizer.del=" + DeletesSummarizer.class.getName());
+    ts.exec("config -t summary -s table.summarizer.fam=" + FamilySummarizer.class.getName());
+
+    ts.exec("addsplits -t summary r1 r2");
+    ts.exec("insert r1 f1 q1 v1");
+    ts.exec("insert r2 f2 q1 v3");
+    ts.exec("insert r2 f2 q2 v4");
+    ts.exec("insert r3 f3 q1 v5");
+    ts.exec("insert r3 f3 q2 v6");
+    ts.exec("insert r3 f3 q3 v7");
+    ts.exec("flush -t summary -w");
+
+    String output = ts.exec("summaries");
+    assertMatches(output, "(?sm).*^.*deletes\\s+=\\s+0.*$.*");
+    assertMatches(output, "(?sm).*^.*total\\s+=\\s+6.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f1\\s+=\\s+1.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f2\\s+=\\s+2.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f3\\s+=\\s+3.*$.*");
+
+    ts.exec("delete r1 f1 q2");
+    ts.exec("delete r2 f2 q1");
+    ts.exec("flush -t summary -w");
+
+    output = ts.exec("summaries");
+    assertMatches(output, "(?sm).*^.*deletes\\s+=\\s+2.*$.*");
+    assertMatches(output, "(?sm).*^.*total\\s+=\\s+8.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f1\\s+=\\s+1.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f2\\s+=\\s+2.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f3\\s+=\\s+3.*$.*");
+
+    output = ts.exec("summaries -e r2");
+    assertMatches(output, "(?sm).*^.*deletes\\s+=\\s+2.*$.*");
+    assertMatches(output, "(?sm).*^.*total\\s+=\\s+5.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f1\\s+=\\s+1.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f2\\s+=\\s+2.*$.*");
+    assertNotContains(output, "c:f3");
+
+    output = ts.exec("summaries -b r2");
+    assertMatches(output, "(?sm).*^.*deletes\\s+=\\s+0.*$.*");
+    assertMatches(output, "(?sm).*^.*total\\s+=\\s+3.*$.*");
+    assertNotContains(output, "c:f1");
+    assertNotContains(output, "c:f2");
+    assertMatches(output, "(?sm).*^.*c:f3\\s+=\\s+3.*$.*");
+
+    output = ts.exec("summaries -b r1 -e r2");
+    assertMatches(output, "(?sm).*^.*deletes\\s+=\\s+1.*$.*");
+    assertMatches(output, "(?sm).*^.*total\\s+=\\s+3.*$.*");
+    assertNotContains(output, "c:f1");
+    assertMatches(output, "(?sm).*^.*c:f2\\s+=\\s+2.*$.*");
+    assertNotContains(output, "c:f3");
+
+    output = ts.exec("summaries -sr .*Family.*");
+    assertNotContains(output, "deletes ");
+    assertMatches(output, "(?sm).*^.*c:f1\\s+=\\s+1.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f2\\s+=\\s+2.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f3\\s+=\\s+3.*$.*");
+
+    output = ts.exec("summaries -b r1 -e r2 -sr .*Family.*");
+    assertNotContains(output, "deletes ");
+    assertNotContains(output, "c:f1");
+    assertMatches(output, "(?sm).*^.*c:f2\\s+=\\s+2.*$.*");
+    assertNotContains(output, "c:f3");
+  }
+
+  @Test
+  public void testSummarySelection() throws Exception {
+    ts.exec("createtable summary2");
+    // will create a few files and do not want them compacted
+    ts.exec("config -t summary2 -s " + Property.TABLE_MAJC_RATIO + "=10");
+
+    ts.exec("insert r1 f1 q1 v1");
+    ts.exec("insert r2 f2 q1 v2");
+    ts.exec("flush -t summary2 -w");
+
+    ts.exec("config -t summary2 -s table.summarizer.fam=" + FamilySummarizer.class.getName());
+
+    ts.exec("insert r1 f2 q1 v3");
+    ts.exec("insert r3 f3 q1 v4");
+    ts.exec("flush -t summary2 -w");
+
+    String output = ts.exec("summaries");
+    assertNotContains(output, "c:f1");
+    assertMatches(output, "(?sm).*^.*c:f2\\s+=\\s+1.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f3\\s+=\\s+1.*$.*");
+    // check that there are two files, with one missing summary info
+    assertMatches(output, "(?sm).*^.*total[:]2[,]\\s+missing[:]1[,]\\s+extra[:]0.*$.*");
+
+    // compact only the file missing summary info
+    ts.exec("compact -t summary2 --sf-no-summary -w");
+    output = ts.exec("summaries");
+    assertMatches(output, "(?sm).*^.*c:f1\\s+=\\s+1.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f2\\s+=\\s+2.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f3\\s+=\\s+1.*$.*");
+    // check that there are two files, with none missing summary info
+    assertMatches(output, "(?sm).*^.*total[:]2[,]\\s+missing[:]0[,]\\s+extra[:]0.*$.*");
+
+    // create a situation where files has summary data outside of tablet
+    ts.exec("addsplits -t summary2 r2");
+    output = ts.exec("summaries -e r2");
+    assertMatches(output, "(?sm).*^.*c:f1\\s+=\\s+1.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f2\\s+=\\s+2.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f3\\s+=\\s+1.*$.*");
+    // check that there are two files, with one having extra summary info
+    assertMatches(output, "(?sm).*^.*total[:]2[,]\\s+missing[:]0[,]\\s+extra[:]1.*$.*");
+
+    // compact only the files with extra summary info
+    ts.exec("compact -t summary2 --sf-extra-summary -w");
+    output = ts.exec("summaries -e r2");
+    assertMatches(output, "(?sm).*^.*c:f1\\s+=\\s+1.*$.*");
+    assertMatches(output, "(?sm).*^.*c:f2\\s+=\\s+2.*$.*");
+    assertNotContains(output, "c:f3");
+    // check that there are two files, with none having extra summary info
+    assertMatches(output, "(?sm).*^.*total[:]2[,]\\s+missing[:]0[,]\\s+extra[:]0.*$.*");
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/94cdcc4d/test/src/main/java/org/apache/accumulo/test/functional/BasicSummarizer.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BasicSummarizer.java b/test/src/main/java/org/apache/accumulo/test/functional/BasicSummarizer.java
new file mode 100644
index 0000000..b109c7e
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BasicSummarizer.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.test.functional;
+
+import org.apache.accumulo.core.client.summary.Summarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+
+/**
+ * This summarizer collects some very basic statistics about Keys.
+ */
+public class BasicSummarizer implements Summarizer {
+
+  public static final String DELETES_STAT = "deletes";
+  public static final String MIN_TIMESTAMP_STAT = "minTimestamp";
+  public static final String MAX_TIMESTAMP_STAT = "maxTimestamp";
+  public static final String TOTAL_STAT = "total";
+
+  @Override
+  public Collector collector(SummarizerConfiguration sc) {
+    return new Collector() {
+
+      private long minStamp = Long.MAX_VALUE;
+      private long maxStamp = Long.MIN_VALUE;
+      private long deletes = 0;
+      private long total = 0;
+
+      @Override
+      public void accept(Key k, Value v) {
+        if (k.getTimestamp() < minStamp) {
+          minStamp = k.getTimestamp();
+        }
+
+        if (k.getTimestamp() > maxStamp) {
+          maxStamp = k.getTimestamp();
+        }
+
+        if (k.isDeleted()) {
+          deletes++;
+        }
+
+        total++;
+      }
+
+      @Override
+      public void summarize(StatisticConsumer sc) {
+        sc.accept(MIN_TIMESTAMP_STAT, minStamp);
+        sc.accept(MAX_TIMESTAMP_STAT, maxStamp);
+        sc.accept(DELETES_STAT, deletes);
+        sc.accept(TOTAL_STAT, total);
+      }
+    };
+  }
+
+  @Override
+  public Combiner combiner(SummarizerConfiguration sc) {
+    return (stats1, stats2) -> {
+      stats1.merge(DELETES_STAT, stats2.get(DELETES_STAT), Long::sum);
+      stats1.merge(TOTAL_STAT, stats2.get(TOTAL_STAT), Long::sum);
+      stats1.merge(MIN_TIMESTAMP_STAT, stats2.get(MIN_TIMESTAMP_STAT), Long::min);
+      stats1.merge(MAX_TIMESTAMP_STAT, stats2.get(MAX_TIMESTAMP_STAT), Long::max);
+    };
+  }
+}