You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@accumulo.apache.org by keith-turner <gi...@git.apache.org> on 2017/03/01 03:02:31 UTC

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

GitHub user keith-turner opened a pull request:

    https://github.com/apache/accumulo/pull/224

    ACCUMULO-4500 ACCUMULO-96 Added summarization

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/keith-turner/accumulo ACCUMULO-4501-5

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/accumulo/pull/224.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #224
    
----
commit e2c56510ed502fadaf583aaf6c08476ed6721f70
Author: Keith Turner <kt...@apache.org>
Date:   2017-03-01T02:56:19Z

    ACCUMULO-4500 ACCUMULO-96 Added summarization

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104530943
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java ---
    @@ -0,0 +1,543 @@
    +/*
    + * 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 {
    +
    +      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();
    --- End diff --
    
    try/finally (even though we know it doesn't really matter)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104690619
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java ---
    @@ -0,0 +1,543 @@
    +/*
    + * 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 diff --
    
    There lower bound is determined by the method that calls it.  That method keeps calling collapse as long as its greater than a certain size.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103731270
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/summary/SummarizerConfiguration.java ---
    @@ -0,0 +1,238 @@
    +/*
    + * 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();
    +    }
    +
    +    /**
    +     * Setting this is optional. If not set, an id is generated using hashing that will likely be unique.
    +     *
    +     * @param propId
    +     *          This is id is used when converting a {@link SummarizerConfiguration} to table properties. Since tables can have multiple summarizers, make sure
    +     *          its unique.
    +     */
    +    public Builder setPropertyId(String propId) {
    +      Preconditions.checkArgument(propId.matches("\\w+"), "Config Id %s is not alphanum", propId);
    +      this.configId = propId;
    +      return this;
    +    }
    +
    +    public Builder addOption(String key, String value) {
    --- End diff --
    
    These methods need javadoc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105247239
  
    --- Diff: 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 {
    --- End diff --
    
    I can't adequately express how cool I think this is.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104506924
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/summary/Summary.java ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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 {
    +
    +  public static class FileStatistics {
    +    private final long total;
    +    private final long missing;
    +    private final long extra;
    +    private final long large;
    +
    +    private FileStatistics(long total, long missing, long extra, long large) {
    +      this.total = total;
    +      this.missing = missing;
    +      this.extra = extra;
    +      this.large = large;
    +    }
    +
    +    /**
    +     * @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.
    +     */
    +    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.
    +     */
    +    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 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()}, and {{@link #getLarge()}.
    +     */
    +    public long getInaccurate() {
    +      return getMissing() + getExtra() + getLarge();
    +    }
    +
    +    @Override
    +    public String toString() {
    +      return String.format("[total:%,d, missing:%,d, extra:%,d, large:%,d]", total, missing, extra, large);
    +    }
    +  }
    +
    +  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) {
    +    this.statistics = ImmutableMap.copyOf(summary);
    +    this.config = config;
    +    this.fileStats = new FileStatistics(totalFiles, filesMissingSummary, filesWithExtra, filesWithLarge);
    +  }
    +
    +  /**
    +   * @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 The statistics that were generated and merged by the specfied {@link Summarizer}.
    --- End diff --
    
    Include in the javadoc that the map itself is immutable as a warning.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on the issue:

    https://github.com/apache/accumulo/pull/224
  
    >  These issues will cause problems when there are problems on the cluster (and that never happens :)
    
    It's so convenient that we always have perfect computers, networks, and administrators :)
    
    >  I am going to work on fixing them. Thinking of introducing configurable thread pools as needed. Also going to make RPC work like scanner with sessions and sessions timeouts.
    
    LMK if you have ideas you want to bounce around. I was a bit brutal in the first review -- we kind of live in a world right now where our RPC code can just let stuff run for a very long time (you inherited a bit of the problem). I don't want to completely block this feature, but I also want to make sure we don't make the problem worse.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103731310
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/summary/SummarizerConfiguration.java ---
    @@ -0,0 +1,238 @@
    +/*
    + * 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();
    +    }
    +
    +    /**
    +     * Setting this is optional. If not set, an id is generated using hashing that will likely be unique.
    --- End diff --
    
    3rd-person declarative, please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105760463
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---
    @@ -1661,4 +1676,138 @@ public Locations locate(String tableName, Collection<Range> ranges) throws Accum
     
         return new LoctionsImpl(binnedRanges);
       }
    +
    +  @Override
    +  public SummaryRetriever getSummaries(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);
    +        }
    +
    +        ClientContext cct = new ClientContext(context.getInstance(), context.getCredentials(), context.getConfiguration()) {
    +          @Override
    +          public long getClientTimeoutInMillis() {
    +            return Math.max(super.getClientTimeoutInMillis(), 60 * 60 * 1000);
    --- End diff --
    
    > Correct me if I'm wrong, but don't we usually do that for client-facing RPCs?
    
    That is a good point.  I think so.
    
    > My worry is that if we do this inside the TabletServer, the risk is larger since that is a Java process which we expect to run for days/months/indefinitely.
    
    One thing I am unsure about is the relationship between the timeout and TCP keepalive.  From the research I have done this morning and previously, I think the timeout works mechanism above the socket and just closes it if the operation takes longer than expected (even if the remote machine is fine and working).  In this case I would rather wait indefinitely and rely on something like TCP keep alive mechanism to close the socket if the remote machine is unresponsive.  I am going to do an experiment where I kill a tserver that I am waiting on with no timeout and see what happens (if something is enabling tcp keepalive, the socket should still be closed eventually causing a retry).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103721572
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java ---
    @@ -121,8 +128,22 @@ public NewTableConfiguration setProperties(Map<String,String> prop) {
        */
       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, summarizerProps);
    +    this.samplerProps = tmp;
    --- End diff --
    
    Seems odd to be checking disjoint with `summarizerProps` but then updating `samplerProps` instead. Maybe javadoc on those member variables would help describe what they each hold.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103730327
  
    --- Diff: 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
    +  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() {
    --- End diff --
    
    Make this a private static class instead of inline to make testing `merge` easier


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105246048
  
    --- Diff: test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java ---
    @@ -0,0 +1,759 @@
    +/*
    + * 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.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.Set;
    +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.Summarizer;
    +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.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(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().getSummaries(table).flush(false).retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    LongSummaryStatistics stats = getTimestampStats(table, c);
    +
    +    summaries = c.tableOperations().getSummaries(table).flush(true).retrieve();
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 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().getSummaries(table).retrieve();
    +
    +    checkSummaries(summaries, sc1, 2, 0, 0, "total", 100_002l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 1l);
    +
    +    bw.close();
    +
    +    c.tableOperations().compact(table, new CompactionConfig().setWait(true));
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // split tablet into two
    +    String sp1 = String.format("r%09x", 50_000);
    +    addSplits(table, c, sp1);
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // compact 2nd tablet
    +    c.tableOperations().compact(table, new CompactionConfig().setStartRow(new Text(sp1)).setWait(true));
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    checkSummaries(summaries, sc1, 2, 0, 1, "total", 113_999l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // get summaries for first tablet
    +    stats = getTimestampStats(table, c, sp1, null);
    +    summaries = c.tableOperations().getSummaries(table).startRow(sp1).retrieve();
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 49_999l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // compact all tablets and regenerate all summaries
    +    c.tableOperations().compact(table, new CompactionConfig());
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    stats = getTimestampStats(table, c);
    +    checkSummaries(summaries, sc1, 2, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    summaries = c.tableOperations().getSummaries(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");
    +    Assert.assertTrue(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().getSummaries(table).retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    c.tableOperations().addSummarizers(table, sc1);
    +    c.tableOperations().compact(table, new CompactionConfig().setWait(true));
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    checkSummaries(summaries, sc1, 2, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 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().getSummaries(table).withConfiguration(sc2).retrieve();
    +    Assert.assertEquals(1, summaries.size());
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    summaries = c.tableOperations().getSummaries(table).withConfiguration(sc1).retrieve();
    +    Assert.assertEquals(1, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // retrieve a non-existant summary
    +    SummarizerConfiguration sc3 = SummarizerConfiguration.builder(KeySizeSummarizer.class.getName()).addOption("maxLen", "256").build();
    +    summaries = c.tableOperations().getSummaries(table).withConfiguration(sc3).retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    summaries = c.tableOperations().getSummaries(table).withConfiguration(sc1, sc2).retrieve();
    +    Assert.assertEquals(2, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    Assert.assertEquals(2, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*BasicSummarizer \\{\\}.*").retrieve();
    +    Assert.assertEquals(1, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*KeySizeSummarizer \\{maxLen=512\\}.*").retrieve();
    +    Assert.assertEquals(1, summaries.size());
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*KeySizeSummarizer \\{maxLen=256\\}.*").retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*BasicSummarizer \\{\\}.*").withConfiguration(sc2).retrieve();
    +    Assert.assertEquals(2, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    // Ensure a bad regex fails fast.
    +    try {
    +      summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*KeySizeSummarizer {maxLen=256}.*").retrieve();
    +      Assert.fail("Bad regex should have caused exception");
    +    } catch (PatternSyntaxException e) {}
    +  }
    +
    +  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));
    +    }
    +  }
    +
    +  public static class FooFilter extends Filter {
    --- End diff --
    
    Ditto on name/javadoc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103720966
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java ---
    @@ -42,7 +46,13 @@
       private boolean limitVersion = true;
     
       private Map<String,String> properties = new HashMap<>();
    -  private SamplerConfiguration samplerConfiguration;
    +  private Map<String,String> samplerProps = Collections.emptyMap();
    +  private Map<String,String> summarizerProps = Collections.emptyMap();
    +
    +  private void checkDisjoint(Map<String,String> props, Map<String,String> sampleProps, Map<String,String> summrizerProps) {
    --- End diff --
    
    s/summrizerProps/summarizerProps/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/accumulo/pull/224


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by mikewalch <gi...@git.apache.org>.
Github user mikewalch commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103748199
  
    --- Diff: docs/src/main/asciidoc/chapters/summaries.txt ---
    @@ -0,0 +1,209 @@
    +// 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 define 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+.
    +
    +=== Configuring
    +
    --- End diff --
    
    I think this section should better describe how to configure table summaries and not just list the properties.  The prefix property needs more documentation on how to set.  Also it looks like most are optional but prefix property is required.  This would be nice to mention.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by mikewalch <gi...@git.apache.org>.
Github user mikewalch commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103737501
  
    --- Diff: docs/src/main/asciidoc/chapters/summaries.txt ---
    @@ -0,0 +1,209 @@
    +// 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 define functions.  Currently these statistics are only generated for
    --- End diff --
    
    should be `user defined`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103816644
  
    --- Diff: 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
    +  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++;
    --- End diff --
    
    Currently it would bork the compaction, so the counter issue would be moot.
    
    I did think about making behavior for failures configurable.  If a summarizer fails, we could record that fact that it ran and failed in the rfile OR let it bork the compaction ... depending on config.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104701438
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/SummaryCollection.java ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummarizerConfiguration;
    +import org.apache.accumulo.core.client.impl.thrift.TSummary;
    +import org.apache.accumulo.core.client.summary.Summarizer;
    +import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
    +import org.apache.accumulo.core.client.summary.Summary;
    +
    +/**
    + * 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;
    +
    +  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();
    +  }
    +
    +  SummaryCollection(Collection<FileSummary> initialEntries) {
    +    mergedSummaries = new HashMap<>();
    +    for (FileSummary entry : initialEntries) {
    +      mergedSummaries.put(entry.conf, new MergedSummary(entry));
    +    }
    +    totalFiles = 1;
    +  }
    +
    +  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) {
    --- End diff --
    
    I don't think so.  Alot of code is assumed to be single threaded.  Also, its internal as you mentioned.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r106811796
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/summary/Summary.java ---
    @@ -78,27 +80,38 @@ public long getLarge() {
         }
     
         /**
    +     * @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;
    --- End diff --
    
    Is this onerous to compute? If this is rare and there is a workaround, I could see keeping this from public API. If it's not bad to compute/track, it's not an issue.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103727728
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java ---
    @@ -92,10 +102,11 @@ public RFileWriter build() throws IOException {
           } 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);
    --- End diff --
    
    Why `false`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103819636
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java ---
    @@ -209,4 +210,12 @@ public static void setSampler(Class<?> implementingClass, Configuration conf, Sa
         }
       }
     
    +  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());
    --- End diff --
    
    I don't think this code is called frequently enough to justify optimizations that would make the code longer/harder to read.  Also the java compiler will use StringBuilder when concatenating  multiple strings.  However, I don't think it will reuse it across the loop.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r106811717
  
    --- Diff: test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java ---
    @@ -770,4 +774,44 @@ public void testExceptions() throws Exception {
           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 = 100000; split < 1000000; split += 100000) {
    --- End diff --
    
    nit: make these `100_000` and `1_000_000` for readability


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104515744
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/SummaryCollection.java ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummarizerConfiguration;
    +import org.apache.accumulo.core.client.impl.thrift.TSummary;
    +import org.apache.accumulo.core.client.summary.Summarizer;
    +import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
    +import org.apache.accumulo.core.client.summary.Summary;
    +
    +/**
    + * 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;
    +
    +  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();
    +  }
    +
    +  SummaryCollection(Collection<FileSummary> initialEntries) {
    +    mergedSummaries = new HashMap<>();
    +    for (FileSummary entry : initialEntries) {
    +      mergedSummaries.put(entry.conf, new MergedSummary(entry));
    +    }
    +    totalFiles = 1;
    +  }
    +
    +  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) {
    --- End diff --
    
    Do we need to doc that this is not-thread-safe? I guess it's internal.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r106811743
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java ---
    @@ -835,7 +835,7 @@ void setSamplerConfiguration(String tableName, SamplerConfiguration samplerConfi
        * @since 2.0.0
        * @see Summarizer
        */
    -  SummaryRetriever getSummaries(String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
    +  SummaryRetriever summarize(String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
    --- End diff --
    
    I think `summarize` is a good improvement!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r106811464
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/conf/Property.java ---
    @@ -362,8 +362,14 @@
           "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 retreied from files.  For a large number of files, the files are broken into partitions of 100K files.  This setting determines "
    --- End diff --
    
    s/retreied/retrieved/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r106811693
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java ---
    @@ -232,7 +234,9 @@ public TabletServerResourceManager(TabletServer tserver, VolumeManager fs) {
         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 retriever", 60, TimeUnit.SECONDS);
    +    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);
    --- End diff --
    
    s/summaryParitionPool/summaryPartitionPool/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105246078
  
    --- Diff: test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java ---
    @@ -0,0 +1,759 @@
    +/*
    + * 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.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.Set;
    +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.Summarizer;
    +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.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(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().getSummaries(table).flush(false).retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    LongSummaryStatistics stats = getTimestampStats(table, c);
    +
    +    summaries = c.tableOperations().getSummaries(table).flush(true).retrieve();
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 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().getSummaries(table).retrieve();
    +
    +    checkSummaries(summaries, sc1, 2, 0, 0, "total", 100_002l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 1l);
    +
    +    bw.close();
    +
    +    c.tableOperations().compact(table, new CompactionConfig().setWait(true));
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // split tablet into two
    +    String sp1 = String.format("r%09x", 50_000);
    +    addSplits(table, c, sp1);
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // compact 2nd tablet
    +    c.tableOperations().compact(table, new CompactionConfig().setStartRow(new Text(sp1)).setWait(true));
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    checkSummaries(summaries, sc1, 2, 0, 1, "total", 113_999l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // get summaries for first tablet
    +    stats = getTimestampStats(table, c, sp1, null);
    +    summaries = c.tableOperations().getSummaries(table).startRow(sp1).retrieve();
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 49_999l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // compact all tablets and regenerate all summaries
    +    c.tableOperations().compact(table, new CompactionConfig());
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    stats = getTimestampStats(table, c);
    +    checkSummaries(summaries, sc1, 2, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    summaries = c.tableOperations().getSummaries(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");
    +    Assert.assertTrue(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().getSummaries(table).retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    c.tableOperations().addSummarizers(table, sc1);
    +    c.tableOperations().compact(table, new CompactionConfig().setWait(true));
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    checkSummaries(summaries, sc1, 2, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 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().getSummaries(table).withConfiguration(sc2).retrieve();
    +    Assert.assertEquals(1, summaries.size());
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    summaries = c.tableOperations().getSummaries(table).withConfiguration(sc1).retrieve();
    +    Assert.assertEquals(1, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // retrieve a non-existant summary
    +    SummarizerConfiguration sc3 = SummarizerConfiguration.builder(KeySizeSummarizer.class.getName()).addOption("maxLen", "256").build();
    +    summaries = c.tableOperations().getSummaries(table).withConfiguration(sc3).retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    summaries = c.tableOperations().getSummaries(table).withConfiguration(sc1, sc2).retrieve();
    +    Assert.assertEquals(2, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    Assert.assertEquals(2, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*BasicSummarizer \\{\\}.*").retrieve();
    +    Assert.assertEquals(1, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*KeySizeSummarizer \\{maxLen=512\\}.*").retrieve();
    +    Assert.assertEquals(1, summaries.size());
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*KeySizeSummarizer \\{maxLen=256\\}.*").retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*BasicSummarizer \\{\\}.*").withConfiguration(sc2).retrieve();
    +    Assert.assertEquals(2, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    // Ensure a bad regex fails fast.
    +    try {
    +      summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*KeySizeSummarizer {maxLen=256}.*").retrieve();
    +      Assert.fail("Bad regex should have caused exception");
    +    } catch (PatternSyntaxException e) {}
    +  }
    +
    +  public static class FooCounter implements Summarizer {
    --- End diff --
    
    Ditto name/javadoc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105252088
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java ---
    @@ -0,0 +1,513 @@
    +/*
    + * 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 java.io.UncheckedIOException;
    +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.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.function.Predicate;
    +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.impl.thrift.TRowRange;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaryRequest;
    +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.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.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.TextUtil;
    +import org.apache.accumulo.fate.util.UtilWaitThread;
    +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.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()}.
    + * <li> {@link #gather()} will call make RPC calls to multiple tservers to remotely execute {@link #processPartition(int, int)}
    + * <li> {@link #processPartition(int, int)} will make RPC calls to multiple tserver to remotely execute
    + * {@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<>();
    +    }
    +  }
    +
    +  private class FilesProcessor implements Callable<ProcessedFiles> {
    +
    +    HostAndPort location;
    +    Map<String,List<TRowRange>> allFiles;
    +    private TInfo tinfo;
    +
    +    public FilesProcessor(TInfo tinfo, HostAndPort location, Map<String,List<TRowRange>> allFiles) {
    +      this.location = location;
    +      this.allFiles = allFiles;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public ProcessedFiles call() throws Exception {
    +      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;
    +          }
    +
    +          TSummaries tSums = null;
    +          try {
    +            tSums = client.getSummariesFromFiles(tinfo, ctx.rpcCreds(), getRequest(), files);
    +          } catch (TApplicationException tae) {
    +            throw tae;
    +          } catch (TTransportException e) {
    +            pfiles.failedFiles.addAll(files.keySet());
    +            continue;
    +          }
    +
    +          pfiles.summaries.merge(new SummaryCollection(tSums), factory);
    +        }
    +      } finally {
    +        ThriftUtil.returnClient(client);
    +      }
    +
    +      return pfiles;
    +    }
    +  }
    +
    +  /**
    +   * 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 SummaryCollection processPartition(int modulus, int remainder) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
    +
    +    Predicate<String> hashingFileSelector = file -> Math.abs(Hashing.murmur3_32().hashString(file).asInt()) % modulus == remainder;
    +
    +    Map<String,Map<String,List<TRowRange>>> filesGBL = getFilesGroupedByLocation(hashingFileSelector);
    +
    +    SummaryCollection partitionSummaries = new SummaryCollection();
    +
    +    ExecutorService execSrv = Executors.newCachedThreadPool();
    +    try {
    +      TInfo tinfo = Tracer.traceInfo();
    +      while (true) {
    +        Set<String> failedFiles = new HashSet<>();
    +
    +        List<Future<ProcessedFiles>> futures = new ArrayList<>();
    +        for (Entry<String,Map<String,List<TRowRange>>> entry : filesGBL.entrySet()) {
    +          HostAndPort location = HostAndPort.fromString(entry.getKey());
    +          Map<String,List<TRowRange>> allFiles = entry.getValue();
    +
    +          Future<ProcessedFiles> future = execSrv.submit(new FilesProcessor(tinfo, location, allFiles));
    +          futures.add(future);
    +        }
    +
    +        for (Future<ProcessedFiles> future : futures) {
    +          try {
    +            ProcessedFiles pfiles = future.get();
    +            if (pfiles.summaries.getTotalFiles() > 0) {
    +              partitionSummaries.merge(pfiles.summaries, factory);
    +            }
    +
    +            failedFiles.addAll(pfiles.failedFiles);
    +          } catch (InterruptedException | ExecutionException e) {
    +            throw new AccumuloException(e);
    +          }
    +        }
    +
    +        if (failedFiles.size() > 0) {
    +          // get new locations just for failed files
    +          Predicate<String> fileSelector = hashingFileSelector.and(file -> failedFiles.contains(file));
    +          filesGBL = getFilesGroupedByLocation(fileSelector);
    +          UtilWaitThread.sleep(250);
    +        } else {
    +          break;
    +        }
    +      }
    +    } finally {
    +      execSrv.shutdownNow();
    +    }
    +
    +    return partitionSummaries;
    +  }
    +
    +  public static interface FileSystemResolver {
    +    FileSystem get(Path file);
    +  }
    +
    +  /**
    +   * This method will read summaries from a set of files.
    +   */
    +  public SummaryCollection processFiles(FileSystemResolver volMgr, Map<String,List<TRowRange>> files, BlockCache summaryCache, BlockCache indexCache,
    +      ExecutorService srp) {
    +    SummaryCollection fileSummaries = new SummaryCollection();
    +
    +    List<Future<SummaryCollection>> futures = new ArrayList<>();
    +    for (Entry<String,List<TRowRange>> entry : files.entrySet()) {
    +      Future<SummaryCollection> future = srp.submit(() -> {
    +        List<RowRange> rrl = Lists.transform(entry.getValue(), RowRange::new);
    +        return getSummaries(volMgr, entry.getKey(), rrl, summaryCache, indexCache);
    +      });
    +      futures.add(future);
    +    }
    +
    +    for (Future<SummaryCollection> future : futures) {
    +      try {
    +        fileSummaries.merge(future.get(), factory);
    +      } catch (ExecutionException | InterruptedException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +
    +    return fileSummaries;
    +  }
    +
    +  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 Callable<SummaryCollection> {
    +
    +    private int remainder;
    +    private int modulus;
    +    private TInfo tinfo;
    +
    +    GatherRequest(TInfo tinfo, int remainder, int modulus) {
    +      this.remainder = remainder;
    +      this.modulus = modulus;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public SummaryCollection call() throws Exception {
    +      TSummaryRequest req = getRequest();
    +
    +      ClientContext cct = new ClientContext(ctx.getInstance(), ctx.getCredentials(), ctx.getConfiguration()) {
    +        @Override
    +        public long getClientTimeoutInMillis() {
    +          return Math.max(super.getClientTimeoutInMillis(), 10 * 60 * 1000);
    +        }
    +      };
    +
    +      TSummaries tSums = ServerClient.execute(cct, c -> c.getSummariesForPartition(tinfo, ctx.rpcCreds(), req, modulus, remainder));
    +      return new SummaryCollection(tSums);
    +    }
    +  }
    +
    +  public SummaryCollection gather() throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
    +
    +    ExecutorService es = Executors.newFixedThreadPool(ctx.getConnector().instanceOperations().getTabletServers().size());
    --- End diff --
    
    More surprise threads??


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105750148
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java ---
    @@ -0,0 +1,513 @@
    +/*
    + * 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 java.io.UncheckedIOException;
    +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.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.function.Predicate;
    +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.impl.thrift.TRowRange;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaryRequest;
    +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.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.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.TextUtil;
    +import org.apache.accumulo.fate.util.UtilWaitThread;
    +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.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()}.
    + * <li> {@link #gather()} will call make RPC calls to multiple tservers to remotely execute {@link #processPartition(int, int)}
    + * <li> {@link #processPartition(int, int)} will make RPC calls to multiple tserver to remotely execute
    + * {@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<>();
    +    }
    +  }
    +
    +  private class FilesProcessor implements Callable<ProcessedFiles> {
    +
    +    HostAndPort location;
    +    Map<String,List<TRowRange>> allFiles;
    +    private TInfo tinfo;
    +
    +    public FilesProcessor(TInfo tinfo, HostAndPort location, Map<String,List<TRowRange>> allFiles) {
    +      this.location = location;
    +      this.allFiles = allFiles;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public ProcessedFiles call() throws Exception {
    +      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;
    +          }
    +
    +          TSummaries tSums = null;
    +          try {
    +            tSums = client.getSummariesFromFiles(tinfo, ctx.rpcCreds(), getRequest(), files);
    +          } catch (TApplicationException tae) {
    +            throw tae;
    +          } catch (TTransportException e) {
    +            pfiles.failedFiles.addAll(files.keySet());
    +            continue;
    +          }
    +
    +          pfiles.summaries.merge(new SummaryCollection(tSums), factory);
    +        }
    +      } finally {
    +        ThriftUtil.returnClient(client);
    +      }
    +
    +      return pfiles;
    +    }
    +  }
    +
    +  /**
    +   * 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 SummaryCollection processPartition(int modulus, int remainder) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
    +
    +    Predicate<String> hashingFileSelector = file -> Math.abs(Hashing.murmur3_32().hashString(file).asInt()) % modulus == remainder;
    +
    +    Map<String,Map<String,List<TRowRange>>> filesGBL = getFilesGroupedByLocation(hashingFileSelector);
    +
    +    SummaryCollection partitionSummaries = new SummaryCollection();
    +
    +    ExecutorService execSrv = Executors.newCachedThreadPool();
    +    try {
    +      TInfo tinfo = Tracer.traceInfo();
    +      while (true) {
    +        Set<String> failedFiles = new HashSet<>();
    +
    +        List<Future<ProcessedFiles>> futures = new ArrayList<>();
    +        for (Entry<String,Map<String,List<TRowRange>>> entry : filesGBL.entrySet()) {
    +          HostAndPort location = HostAndPort.fromString(entry.getKey());
    +          Map<String,List<TRowRange>> allFiles = entry.getValue();
    +
    +          Future<ProcessedFiles> future = execSrv.submit(new FilesProcessor(tinfo, location, allFiles));
    +          futures.add(future);
    +        }
    +
    +        for (Future<ProcessedFiles> future : futures) {
    +          try {
    +            ProcessedFiles pfiles = future.get();
    +            if (pfiles.summaries.getTotalFiles() > 0) {
    +              partitionSummaries.merge(pfiles.summaries, factory);
    +            }
    +
    +            failedFiles.addAll(pfiles.failedFiles);
    +          } catch (InterruptedException | ExecutionException e) {
    +            throw new AccumuloException(e);
    +          }
    +        }
    +
    +        if (failedFiles.size() > 0) {
    +          // get new locations just for failed files
    +          Predicate<String> fileSelector = hashingFileSelector.and(file -> failedFiles.contains(file));
    +          filesGBL = getFilesGroupedByLocation(fileSelector);
    +          UtilWaitThread.sleep(250);
    +        } else {
    +          break;
    +        }
    +      }
    +    } finally {
    +      execSrv.shutdownNow();
    +    }
    +
    +    return partitionSummaries;
    +  }
    +
    +  public static interface FileSystemResolver {
    +    FileSystem get(Path file);
    +  }
    +
    +  /**
    +   * This method will read summaries from a set of files.
    +   */
    +  public SummaryCollection processFiles(FileSystemResolver volMgr, Map<String,List<TRowRange>> files, BlockCache summaryCache, BlockCache indexCache,
    +      ExecutorService srp) {
    +    SummaryCollection fileSummaries = new SummaryCollection();
    +
    +    List<Future<SummaryCollection>> futures = new ArrayList<>();
    +    for (Entry<String,List<TRowRange>> entry : files.entrySet()) {
    +      Future<SummaryCollection> future = srp.submit(() -> {
    +        List<RowRange> rrl = Lists.transform(entry.getValue(), RowRange::new);
    +        return getSummaries(volMgr, entry.getKey(), rrl, summaryCache, indexCache);
    +      });
    +      futures.add(future);
    +    }
    +
    +    for (Future<SummaryCollection> future : futures) {
    +      try {
    +        fileSummaries.merge(future.get(), factory);
    +      } catch (ExecutionException | InterruptedException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +
    +    return fileSummaries;
    +  }
    +
    +  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 Callable<SummaryCollection> {
    +
    +    private int remainder;
    +    private int modulus;
    +    private TInfo tinfo;
    +
    +    GatherRequest(TInfo tinfo, int remainder, int modulus) {
    +      this.remainder = remainder;
    +      this.modulus = modulus;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public SummaryCollection call() throws Exception {
    +      TSummaryRequest req = getRequest();
    +
    +      ClientContext cct = new ClientContext(ctx.getInstance(), ctx.getCredentials(), ctx.getConfiguration()) {
    +        @Override
    +        public long getClientTimeoutInMillis() {
    +          return Math.max(super.getClientTimeoutInMillis(), 10 * 60 * 1000);
    +        }
    +      };
    +
    +      TSummaries tSums = ServerClient.execute(cct, c -> c.getSummariesForPartition(tinfo, ctx.rpcCreds(), req, modulus, remainder));
    +      return new SummaryCollection(tSums);
    +    }
    +  }
    +
    +  public SummaryCollection gather() throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
    +
    +    ExecutorService es = Executors.newFixedThreadPool(ctx.getConnector().instanceOperations().getTabletServers().size());
    --- End diff --
    
    Yeah this not ideal.  These threads will not consume CPU, they are just waiting on remote operations.  The threads will consume memory resources though.  One nice solution for this would be to make all of the request asynchronously AND not use threads.  I looked into doing this, it would be a lot of new code that would need to be tested for fault tolerance.  I will open a follow up issue for this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103727371
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java ---
    @@ -59,15 +62,22 @@ OutputStream getOutputStream() {
       }
     
       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> sampleProps, Map<String,String> summrizerProps) {
    --- End diff --
    
    Same method you have in NewTableConfiguration. Consolidate?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103816184
  
    --- Diff: 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
    --- End diff --
    
    I should add some documentation about considerations for persistence to the super type.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103795950
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---
    @@ -1661,4 +1676,138 @@ public Locations locate(String tableName, Collection<Range> ranges) throws Accum
     
         return new LoctionsImpl(binnedRanges);
       }
    +
    +  @Override
    +  public SummaryRetriever getSummaries(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);
    +        }
    +
    +        ClientContext cct = new ClientContext(context.getInstance(), context.getCredentials(), context.getConfiguration()) {
    +          @Override
    +          public long getClientTimeoutInMillis() {
    +            return Math.max(super.getClientTimeoutInMillis(), 60 * 60 * 1000);
    +          }
    +        };
    +        TSummaries ret = ServerClient.execute(cct, c -> c.getSummaries(Tracer.traceInfo(), context.rpcCreds(), request));
    +        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)));
    --- End diff --
    
    The function returns a collection, which could have multiple.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r106568670
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java ---
    @@ -808,4 +812,64 @@ void setSamplerConfiguration(String tableName, SamplerConfiguration samplerConfi
        * @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. The
    +   * size of this cache is determined by the property {code tserver.cache.summary.size}. When summary data for a file is not present, it will be retrieved using
    +   * threads on the tserver. The property {@code tserver.summary.retrieval.threads} determines the max number of threads the tserver will use for this.
    +   *
    +   * <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 getSummaries(String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
    --- End diff --
    
    I renamed it to `summarize()`  still not sure about that name, but I like it better than `getSummaries()`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r106906812
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/summary/Summary.java ---
    @@ -78,27 +80,38 @@ public long getLarge() {
         }
     
         /**
    +     * @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;
    --- End diff --
    
    It was not onerous to compute.  Special action needs to be taken on file not found anyway (do not want to retry).  In addition to ignoring the file, just need to report the info back.  Reporting back was not too bad as there were already other file stats.
    
    I could not figure out to write a test for it.  I tested it manually.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103731088
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/VisibilitySummarizer.java ---
    @@ -0,0 +1,37 @@
    +/*
    + * 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.summary.CountingSummarizer;
    +import org.apache.accumulo.core.data.ArrayByteSequence;
    +import org.apache.accumulo.core.data.ByteSequence;
    +
    +public class VisibilitySummarizer extends CountingSummarizer<ByteSequence> {
    --- End diff --
    
    needs javadoc & since tag


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104512326
  
    --- Diff: 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.impl.thrift.TSummarizerConfiguration;
    +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;
    +
    +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));
    --- End diff --
    
    Blech.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103729447
  
    --- Diff: 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
    --- End diff --
    
    Can you expand on this comment?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104566494
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java ---
    @@ -0,0 +1,543 @@
    +/*
    + * 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 {
    +
    +      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();
    --- End diff --
    
    try-with-resources even better.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103819806
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java ---
    @@ -121,8 +128,22 @@ public NewTableConfiguration setProperties(Map<String,String> prop) {
        */
       public NewTableConfiguration enableSampling(SamplerConfiguration samplerConfiguration) {
         requireNonNull(samplerConfiguration);
    -    SamplerConfigurationImpl.checkDisjoint(properties, samplerConfiguration);
    -    this.samplerConfiguration = samplerConfiguration;
    +    Map<String,String> tmp = new SamplerConfigurationImpl(samplerConfiguration).toTablePropertiesMap();
    --- End diff --
    
    I don't think this would be called frequently enough for it to be worth it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103820479
  
    --- Diff: docs/src/main/asciidoc/chapters/summaries.txt ---
    @@ -0,0 +1,209 @@
    +// 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 define 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+.
    +
    +=== Configuring
    +
    --- End diff --
    
    In the rendered ascii doc this will link the prefix.  The prefix property has documentation on how to configure a summarize which will show up in the ascii doc. Did you see that documentation?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103723706
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java ---
    @@ -808,4 +812,64 @@ void setSamplerConfiguration(String tableName, SamplerConfiguration samplerConfi
        * @since 1.8.0
        */
       SamplerConfiguration getSamplerConfiguration(String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
    +
    +  /**
    +   * This is a entry point for retrieving summaries with optional restrictions.
    +   *
    +   * <p>
    +   * Inorder 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. The
    +   * size of this cache is determined by the property {code tserver.cache.summary.size}. When summary data for a file is not present, it will be retrieved using
    +   * threads on the tserver. The property {@code tserver.summary.retrieval.threads} determines the max number of threads the tserver will use for this.
    +   *
    +   * <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 its important that the required summarizers are on the clients
    --- End diff --
    
    s/its/it's/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105250934
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java ---
    @@ -0,0 +1,513 @@
    +/*
    + * 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 java.io.UncheckedIOException;
    +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.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.function.Predicate;
    +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.impl.thrift.TRowRange;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaryRequest;
    +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.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.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.TextUtil;
    +import org.apache.accumulo.fate.util.UtilWaitThread;
    +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.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()}.
    + * <li> {@link #gather()} will call make RPC calls to multiple tservers to remotely execute {@link #processPartition(int, int)}
    + * <li> {@link #processPartition(int, int)} will make RPC calls to multiple tserver to remotely execute
    + * {@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<>();
    +    }
    +  }
    +
    +  private class FilesProcessor implements Callable<ProcessedFiles> {
    +
    +    HostAndPort location;
    +    Map<String,List<TRowRange>> allFiles;
    +    private TInfo tinfo;
    +
    +    public FilesProcessor(TInfo tinfo, HostAndPort location, Map<String,List<TRowRange>> allFiles) {
    +      this.location = location;
    +      this.allFiles = allFiles;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public ProcessedFiles call() throws Exception {
    +      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;
    +          }
    +
    +          TSummaries tSums = null;
    +          try {
    +            tSums = client.getSummariesFromFiles(tinfo, ctx.rpcCreds(), getRequest(), files);
    +          } catch (TApplicationException tae) {
    +            throw tae;
    +          } catch (TTransportException e) {
    +            pfiles.failedFiles.addAll(files.keySet());
    +            continue;
    +          }
    +
    +          pfiles.summaries.merge(new SummaryCollection(tSums), factory);
    +        }
    +      } finally {
    +        ThriftUtil.returnClient(client);
    +      }
    +
    +      return pfiles;
    +    }
    +  }
    +
    +  /**
    +   * 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 SummaryCollection processPartition(int modulus, int remainder) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
    +
    +    Predicate<String> hashingFileSelector = file -> Math.abs(Hashing.murmur3_32().hashString(file).asInt()) % modulus == remainder;
    +
    +    Map<String,Map<String,List<TRowRange>>> filesGBL = getFilesGroupedByLocation(hashingFileSelector);
    +
    +    SummaryCollection partitionSummaries = new SummaryCollection();
    +
    +    ExecutorService execSrv = Executors.newCachedThreadPool();
    +    try {
    +      TInfo tinfo = Tracer.traceInfo();
    +      while (true) {
    +        Set<String> failedFiles = new HashSet<>();
    +
    +        List<Future<ProcessedFiles>> futures = new ArrayList<>();
    +        for (Entry<String,Map<String,List<TRowRange>>> entry : filesGBL.entrySet()) {
    +          HostAndPort location = HostAndPort.fromString(entry.getKey());
    +          Map<String,List<TRowRange>> allFiles = entry.getValue();
    +
    +          Future<ProcessedFiles> future = execSrv.submit(new FilesProcessor(tinfo, location, allFiles));
    --- End diff --
    
    From `Executors.newCachedThreadPool()` javadoc:
    
    > These pools will typically improve the performance of programs that execute many short-lived asynchronous tasks
    
    But, in `FilesProcessor` aren't we running an RPC without timeout? Seem to be competing goals (and potentially keep us to hold on to a lot of threads if RPCs are delayed).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105246492
  
    --- Diff: test/src/main/java/org/apache/accumulo/test/ShellServerIT.java ---
    @@ -1894,4 +1896,71 @@ private String getTableId(String tableName) throws Exception {
         return null;
       }
     
    +  @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("config -t summary -s table.summarizer.fam.opt."+CountingSummarizer.INGNORE_DELETES_OPT+"=false");
    --- End diff --
    
    nit: delete


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103724796
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---
    @@ -1661,4 +1676,138 @@ public Locations locate(String tableName, Collection<Range> ranges) throws Accum
     
         return new LoctionsImpl(binnedRanges);
       }
    +
    +  @Override
    +  public SummaryRetriever getSummaries(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);
    +        }
    +
    +        ClientContext cct = new ClientContext(context.getInstance(), context.getCredentials(), context.getConfiguration()) {
    +          @Override
    +          public long getClientTimeoutInMillis() {
    +            return Math.max(super.getClientTimeoutInMillis(), 60 * 60 * 1000);
    --- End diff --
    
    This is forcing a minimum client timeout of 60mins which could be irritating for someone who set a 10minute timeout. Is this what you intended?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by mikewalch <gi...@git.apache.org>.
Github user mikewalch commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103750219
  
    --- Diff: docs/src/main/asciidoc/chapters/summaries.txt ---
    @@ -0,0 +1,209 @@
    +// 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 define 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
    --- End diff --
    
    Should mention that there are summarizers available for use (and list them) in addition to creating your own custom serializer


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by mjwall <gi...@git.apache.org>.
Github user mjwall commented on the issue:

    https://github.com/apache/accumulo/pull/224
  
    @keith-turner we talked about this yesterday, but I wanted to post it here.  What would happen if a file is deleted, like maybe compacted and gc'd, after the file list is grabbed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r106811664
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java ---
    @@ -1812,7 +1884,26 @@ public TSummaries getSummariesFromFiles(TInfo tinfo, TCredentials credentials, T
           BlockCache summaryCache = resourceManager.getSummaryCache();
           BlockCache indexCache = resourceManager.getIndexCache();
           FileSystemResolver volMgr = p -> fs.getVolumeByPath(p).getFileSystem();
    -      return new Gatherer(TabletServer.this, request, tableCfg).processFiles(volMgr, files, summaryCache, indexCache, srp).toThrift();
    +      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();
    --- End diff --
    
    Is it OK to be re-using the same exception as we use in the Scanner path?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105704572
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java ---
    @@ -48,6 +53,83 @@ void gatherInformation(MajorCompactionRequest request) {}
         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);
    +
    +        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);
    --- End diff --
    
    I Added an IT for these shell options in  	7a9fa75


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105251541
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java ---
    @@ -0,0 +1,513 @@
    +/*
    + * 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 java.io.UncheckedIOException;
    +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.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.function.Predicate;
    +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.impl.thrift.TRowRange;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaryRequest;
    +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.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.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.TextUtil;
    +import org.apache.accumulo.fate.util.UtilWaitThread;
    +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.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()}.
    + * <li> {@link #gather()} will call make RPC calls to multiple tservers to remotely execute {@link #processPartition(int, int)}
    + * <li> {@link #processPartition(int, int)} will make RPC calls to multiple tserver to remotely execute
    + * {@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<>();
    +    }
    +  }
    +
    +  private class FilesProcessor implements Callable<ProcessedFiles> {
    +
    +    HostAndPort location;
    +    Map<String,List<TRowRange>> allFiles;
    +    private TInfo tinfo;
    +
    +    public FilesProcessor(TInfo tinfo, HostAndPort location, Map<String,List<TRowRange>> allFiles) {
    +      this.location = location;
    +      this.allFiles = allFiles;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public ProcessedFiles call() throws Exception {
    +      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;
    +          }
    +
    +          TSummaries tSums = null;
    +          try {
    +            tSums = client.getSummariesFromFiles(tinfo, ctx.rpcCreds(), getRequest(), files);
    +          } catch (TApplicationException tae) {
    +            throw tae;
    +          } catch (TTransportException e) {
    +            pfiles.failedFiles.addAll(files.keySet());
    +            continue;
    +          }
    +
    +          pfiles.summaries.merge(new SummaryCollection(tSums), factory);
    +        }
    +      } finally {
    +        ThriftUtil.returnClient(client);
    +      }
    +
    +      return pfiles;
    +    }
    +  }
    +
    +  /**
    +   * 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 SummaryCollection processPartition(int modulus, int remainder) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
    +
    +    Predicate<String> hashingFileSelector = file -> Math.abs(Hashing.murmur3_32().hashString(file).asInt()) % modulus == remainder;
    +
    +    Map<String,Map<String,List<TRowRange>>> filesGBL = getFilesGroupedByLocation(hashingFileSelector);
    +
    +    SummaryCollection partitionSummaries = new SummaryCollection();
    +
    +    ExecutorService execSrv = Executors.newCachedThreadPool();
    +    try {
    +      TInfo tinfo = Tracer.traceInfo();
    +      while (true) {
    +        Set<String> failedFiles = new HashSet<>();
    +
    +        List<Future<ProcessedFiles>> futures = new ArrayList<>();
    +        for (Entry<String,Map<String,List<TRowRange>>> entry : filesGBL.entrySet()) {
    +          HostAndPort location = HostAndPort.fromString(entry.getKey());
    +          Map<String,List<TRowRange>> allFiles = entry.getValue();
    +
    +          Future<ProcessedFiles> future = execSrv.submit(new FilesProcessor(tinfo, location, allFiles));
    +          futures.add(future);
    +        }
    +
    +        for (Future<ProcessedFiles> future : futures) {
    +          try {
    +            ProcessedFiles pfiles = future.get();
    +            if (pfiles.summaries.getTotalFiles() > 0) {
    +              partitionSummaries.merge(pfiles.summaries, factory);
    +            }
    +
    +            failedFiles.addAll(pfiles.failedFiles);
    +          } catch (InterruptedException | ExecutionException e) {
    +            throw new AccumuloException(e);
    +          }
    +        }
    +
    +        if (failedFiles.size() > 0) {
    +          // get new locations just for failed files
    +          Predicate<String> fileSelector = hashingFileSelector.and(file -> failedFiles.contains(file));
    +          filesGBL = getFilesGroupedByLocation(fileSelector);
    +          UtilWaitThread.sleep(250);
    +        } else {
    +          break;
    --- End diff --
    
    Just make this `return partitionSummaries;`? Not sure if I'm missing an execution branch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103817193
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/FileOperations.java ---
    @@ -301,6 +302,16 @@ public RateLimiter getRateLimiter() {
           return this;
         }
     
    +    public boolean isAccumuloStartEnabled() {
    +      return enableAccumuloStart;
    +    }
    +
    +    @Override
    +    public OpenWriterOperation setAccumuloStartEnabled(boolean enableAccumuloStart) {
    --- End diff --
    
    The one thing I miss from Review Board is its per review issues.  Those were nice to help track what needed to be done on a review with lots of comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105249413
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java ---
    @@ -0,0 +1,513 @@
    +/*
    + * 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 java.io.UncheckedIOException;
    +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.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.function.Predicate;
    +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.impl.thrift.TRowRange;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaryRequest;
    +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.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.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.TextUtil;
    +import org.apache.accumulo.fate.util.UtilWaitThread;
    +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.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()}.
    + * <li> {@link #gather()} will call make RPC calls to multiple tservers to remotely execute {@link #processPartition(int, int)}
    + * <li> {@link #processPartition(int, int)} will make RPC calls to multiple tserver to remotely execute
    + * {@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<>();
    +    }
    +  }
    +
    +  private class FilesProcessor implements Callable<ProcessedFiles> {
    +
    +    HostAndPort location;
    +    Map<String,List<TRowRange>> allFiles;
    +    private TInfo tinfo;
    +
    +    public FilesProcessor(TInfo tinfo, HostAndPort location, Map<String,List<TRowRange>> allFiles) {
    +      this.location = location;
    +      this.allFiles = allFiles;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public ProcessedFiles call() throws Exception {
    +      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;
    +          }
    +
    +          TSummaries tSums = null;
    +          try {
    +            tSums = client.getSummariesFromFiles(tinfo, ctx.rpcCreds(), getRequest(), files);
    --- End diff --
    
    Should we have some kind of inspection as to this cross-sectional RPC? What kind of metrics/logging can we add to understand this process better?
    
    If I want to get the summaries for a table as a client, and that table is hosted by every tserver, so that mean we might get a cartesian product of RPCs from tserver->tserver?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on the issue:

    https://github.com/apache/accumulo/pull/224
  
    > I was a bit brutal in the first review -- we kind of live in a world right now where our RPC code can just let stuff run for a very long time (you inherited a bit of the problem).
    
    There were so many things to consider for this feature.  I am really glad you made me focus on this aspect.  
    
    >  we kind of live in a world right now where our RPC code can just let stuff run for a very long time
    
    The session concept that some of the code currently uses (like scans) defends against the following : 
    
     * Doing work for a client that is no longer there.
     * Clients starting a second instance of a task, while the first is still processing.
    
    The current code is susceptible to both of the problems above and I would like to avoid that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105247760
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java ---
    @@ -48,6 +53,83 @@ void gatherInformation(MajorCompactionRequest request) {}
         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);
    +
    +        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);
    --- End diff --
    
    It looks like `okFiles` is only set once (to `Collections.emptyMap()`). Isn't the `Map` returned by that method immutable?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104516639
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.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.summary;
    +
    +import java.io.DataInputStream;
    +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) throws IllegalArgumentException, IOException {
    +    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 (IOException e) {
    +      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) throws IOException {
    --- End diff --
    
    If the `PrintStream` is configurable, there should be a `public` entry-point for it too, no?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105249686
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java ---
    @@ -0,0 +1,513 @@
    +/*
    + * 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 java.io.UncheckedIOException;
    +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.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.function.Predicate;
    +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.impl.thrift.TRowRange;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaryRequest;
    +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.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.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.TextUtil;
    +import org.apache.accumulo.fate.util.UtilWaitThread;
    +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.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()}.
    + * <li> {@link #gather()} will call make RPC calls to multiple tservers to remotely execute {@link #processPartition(int, int)}
    + * <li> {@link #processPartition(int, int)} will make RPC calls to multiple tserver to remotely execute
    + * {@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<>();
    +    }
    +  }
    +
    +  private class FilesProcessor implements Callable<ProcessedFiles> {
    +
    +    HostAndPort location;
    +    Map<String,List<TRowRange>> allFiles;
    +    private TInfo tinfo;
    +
    +    public FilesProcessor(TInfo tinfo, HostAndPort location, Map<String,List<TRowRange>> allFiles) {
    +      this.location = location;
    +      this.allFiles = allFiles;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public ProcessedFiles call() throws Exception {
    +      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;
    +          }
    +
    +          TSummaries tSums = null;
    +          try {
    +            tSums = client.getSummariesFromFiles(tinfo, ctx.rpcCreds(), getRequest(), files);
    +          } catch (TApplicationException tae) {
    +            throw tae;
    --- End diff --
    
    What about the case of a failing tserver? Wouldn't this propagate up and fail the entire getSummaries call? Do we want to try a couple of times to get the summaries from the tserver? Maybe try a diffferent tserver?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103822610
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java ---
    @@ -59,15 +62,22 @@ OutputStream getOutputStream() {
       }
     
       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> sampleProps, Map<String,String> summrizerProps) {
    --- End diff --
    
    That would be the right thing to do.  Its a bit annoying because its private in classes that in the public API space.  So I would need to place in some third class that not in a public API package, and I don't know where to put it.  I will added a comment to both methods mentioning its doppelganger .


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104515984
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/SummaryInfo.java ---
    @@ -0,0 +1,45 @@
    +/*
    + * 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 {
    +
    +  Map<String,Long> summary;
    +  Text lastRow;
    +  int count = 0;
    +
    +  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;
    --- End diff --
    
    Only one getter? Would prefer to see the members private and the package-protected getters for all members.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104566230
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java ---
    @@ -117,6 +121,7 @@ String getFormatDescription() {
        * @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????");
    --- End diff --
    
    Enums are inherently java-serializable. This cannot be prevented. What this is trying to work around (in conjunction with the predicate being transient) is that the predicates (especially the lambdas?) are not themselves serializable and cannot necessarily be included with the enum. If, for whatever reason, these enums ever get serialized, and then deserialized, we won't necessarily be able to call the `isValidFormat` function on them.
    
    I'm not sure if this can even ever happen. I would expect enums to have one, and only one, instance in the JVM ever.... the one created by the constructor... not the one produced by deserializing. So, this might not ever be able to happen... but it'd be interesting to see if it did.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103726746
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java ---
    @@ -209,4 +210,12 @@ public static void setSampler(Class<?> implementingClass, Configuration conf, Sa
         }
       }
     
    +  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());
    --- End diff --
    
    Use a `StringBuilder` to cut down on transient `String`s you're generating.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on the issue:

    https://github.com/apache/accumulo/pull/224
  
    > @keith-turner we talked about this yesterday, but I wanted to post it here. What would happen if a file is deleted, like maybe compacted and gc'd, after the file list is grabbed?
    
    @mjwall I had not thought of this case and currently have no handling for it.  Yet another win for code reviews.
    
    I think the best solution to this problem is to introduce a new inaccuracy counter called `deleted`.  There are already a few inaccuracy counters reported when gather summary information.  I will add another comment that shows where these can be found.
    
    At first I thought I could circle back and use the file that replaced a missing file.  However this approach has a problem.  Multiple deleted files could have been compacted into the replacement file, and for some of those deleted files we may have already gathered and merged summary information.  Trying to avoid this problem would make gathering summaries more expensive.  In order to keep gathering summaries fast, I think it would be best to just report the problem.  If someone really wants to avoid this problem, they can clone the table and make the request against the clone.  I can put this avoidance strategy in the javadoc for `deleted`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104701186
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/SummaryInfo.java ---
    @@ -0,0 +1,45 @@
    +/*
    + * 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 {
    +
    +  Map<String,Long> summary;
    +  Text lastRow;
    +  int count = 0;
    +
    +  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;
    --- End diff --
    
    fixed this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103730007
  
    --- Diff: 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
    +  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++;
    --- End diff --
    
    Do we care about over-counting `emitted`? e.g. what happens when `encoded.apply(counter)` would throw an exception?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104508413
  
    --- Diff: 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));
    --- End diff --
    
    This seems like a common case for Summarizers. Would it make sense to lift this into an `abstract class LongSummarizer`?
    
    ```java
    public abstract class LongSummarizer implements Summarizer {
      @Override
      public Combiner combiner(SummarizerConfiguration sc) {
        return (m1, m2) -> m2.forEach((k, v) -> m1.merge(k, v, Long::sum));
      }
    }
    ```
    
    Then, make this class..
    
    ```java
    public class DeletesSummarizer extends LongSummarizer {
    ...
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103796867
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---
    @@ -1661,4 +1676,138 @@ public Locations locate(String tableName, Collection<Range> ranges) throws Accum
     
         return new LoctionsImpl(binnedRanges);
       }
    +
    +  @Override
    +  public SummaryRetriever getSummaries(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);
    +        }
    +
    +        ClientContext cct = new ClientContext(context.getInstance(), context.getCredentials(), context.getConfiguration()) {
    +          @Override
    +          public long getClientTimeoutInMillis() {
    +            return Math.max(super.getClientTimeoutInMillis(), 60 * 60 * 1000);
    +          }
    +        };
    +        TSummaries ret = ServerClient.execute(cct, c -> c.getSummaries(Tracer.traceInfo(), context.rpcCreds(), request));
    +        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)));
    --- End diff --
    
    Ahh, I missed that. I just expected this method to return a single SummarizeConfiguration. Makes sense.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104691063
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java ---
    @@ -0,0 +1,543 @@
    +/*
    + * 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()));
    --- End diff --
    
    It should always have a mapping.  It it does not the code will fail fast when trying to convert `null` to `int`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104463134
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/summary/Summary.java ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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 {
    +
    +  public static class FileStatistics {
    --- End diff --
    
    @mjwall I was thinking of adding the `deleted` statistic to this class.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104686891
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java ---
    @@ -117,6 +121,7 @@ String getFormatDescription() {
        * @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????");
    --- End diff --
    
    For some reason the changes I made resulted in findbugs complaining about code I didn't change.  These changes were a result of a findbugs complaint.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105683369
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java ---
    @@ -0,0 +1,513 @@
    +/*
    + * 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 java.io.UncheckedIOException;
    +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.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.function.Predicate;
    +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.impl.thrift.TRowRange;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaryRequest;
    +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.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.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.TextUtil;
    +import org.apache.accumulo.fate.util.UtilWaitThread;
    +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.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()}.
    + * <li> {@link #gather()} will call make RPC calls to multiple tservers to remotely execute {@link #processPartition(int, int)}
    + * <li> {@link #processPartition(int, int)} will make RPC calls to multiple tserver to remotely execute
    + * {@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<>();
    +    }
    +  }
    +
    +  private class FilesProcessor implements Callable<ProcessedFiles> {
    +
    +    HostAndPort location;
    +    Map<String,List<TRowRange>> allFiles;
    +    private TInfo tinfo;
    +
    +    public FilesProcessor(TInfo tinfo, HostAndPort location, Map<String,List<TRowRange>> allFiles) {
    +      this.location = location;
    +      this.allFiles = allFiles;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public ProcessedFiles call() throws Exception {
    +      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;
    +          }
    +
    +          TSummaries tSums = null;
    +          try {
    +            tSums = client.getSummariesFromFiles(tinfo, ctx.rpcCreds(), getRequest(), files);
    +          } catch (TApplicationException tae) {
    +            throw tae;
    --- End diff --
    
    Yes, I think it would fail the entire call and that is the intent.  The client should get an `AccumuloServerException`
    
    All other accumulo code fails when an unexpected server side exception is seen. I would not want to do something different here. If anything different was done, would want to consider doing it for all code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on the issue:

    https://github.com/apache/accumulo/pull/224
  
    I think I have addressed all of the code review issues, except for metrics/tracing.  I think I will open a follow on issue to instrument this feature.  The thread pool and rpc timeout issues are no more. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105244887
  
    --- Diff: test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java ---
    @@ -0,0 +1,759 @@
    +/*
    + * 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.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.Set;
    +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.Summarizer;
    +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.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(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().getSummaries(table).flush(false).retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    LongSummaryStatistics stats = getTimestampStats(table, c);
    +
    +    summaries = c.tableOperations().getSummaries(table).flush(true).retrieve();
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 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().getSummaries(table).retrieve();
    +
    +    checkSummaries(summaries, sc1, 2, 0, 0, "total", 100_002l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 1l);
    +
    +    bw.close();
    +
    +    c.tableOperations().compact(table, new CompactionConfig().setWait(true));
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // split tablet into two
    +    String sp1 = String.format("r%09x", 50_000);
    +    addSplits(table, c, sp1);
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // compact 2nd tablet
    +    c.tableOperations().compact(table, new CompactionConfig().setStartRow(new Text(sp1)).setWait(true));
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    checkSummaries(summaries, sc1, 2, 0, 1, "total", 113_999l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // get summaries for first tablet
    +    stats = getTimestampStats(table, c, sp1, null);
    +    summaries = c.tableOperations().getSummaries(table).startRow(sp1).retrieve();
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 49_999l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // compact all tablets and regenerate all summaries
    +    c.tableOperations().compact(table, new CompactionConfig());
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    stats = getTimestampStats(table, c);
    +    checkSummaries(summaries, sc1, 2, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    summaries = c.tableOperations().getSummaries(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");
    +    Assert.assertTrue(total > 0 && total <= 10_000);
    --- End diff --
    
    Nit: list what the total was in the message argument for `assertTrue(String, boolean)`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105268539
  
    --- Diff: 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));
    --- End diff --
    
    My guess is that summations would be the most common summaries that we'd want to do in Accumulo so, even if the boilerplate reduction is small for each, there might be some long-term gain for all of the uses of summation-summaries.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105251834
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java ---
    @@ -0,0 +1,513 @@
    +/*
    + * 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 java.io.UncheckedIOException;
    +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.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.function.Predicate;
    +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.impl.thrift.TRowRange;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaryRequest;
    +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.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.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.TextUtil;
    +import org.apache.accumulo.fate.util.UtilWaitThread;
    +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.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()}.
    + * <li> {@link #gather()} will call make RPC calls to multiple tservers to remotely execute {@link #processPartition(int, int)}
    + * <li> {@link #processPartition(int, int)} will make RPC calls to multiple tserver to remotely execute
    + * {@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<>();
    +    }
    +  }
    +
    +  private class FilesProcessor implements Callable<ProcessedFiles> {
    +
    +    HostAndPort location;
    +    Map<String,List<TRowRange>> allFiles;
    +    private TInfo tinfo;
    +
    +    public FilesProcessor(TInfo tinfo, HostAndPort location, Map<String,List<TRowRange>> allFiles) {
    +      this.location = location;
    +      this.allFiles = allFiles;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public ProcessedFiles call() throws Exception {
    +      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;
    +          }
    +
    +          TSummaries tSums = null;
    +          try {
    +            tSums = client.getSummariesFromFiles(tinfo, ctx.rpcCreds(), getRequest(), files);
    +          } catch (TApplicationException tae) {
    +            throw tae;
    +          } catch (TTransportException e) {
    +            pfiles.failedFiles.addAll(files.keySet());
    +            continue;
    +          }
    +
    +          pfiles.summaries.merge(new SummaryCollection(tSums), factory);
    +        }
    +      } finally {
    +        ThriftUtil.returnClient(client);
    +      }
    +
    +      return pfiles;
    +    }
    +  }
    +
    +  /**
    +   * 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 SummaryCollection processPartition(int modulus, int remainder) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
    +
    +    Predicate<String> hashingFileSelector = file -> Math.abs(Hashing.murmur3_32().hashString(file).asInt()) % modulus == remainder;
    +
    +    Map<String,Map<String,List<TRowRange>>> filesGBL = getFilesGroupedByLocation(hashingFileSelector);
    +
    +    SummaryCollection partitionSummaries = new SummaryCollection();
    +
    +    ExecutorService execSrv = Executors.newCachedThreadPool();
    +    try {
    +      TInfo tinfo = Tracer.traceInfo();
    +      while (true) {
    +        Set<String> failedFiles = new HashSet<>();
    +
    +        List<Future<ProcessedFiles>> futures = new ArrayList<>();
    +        for (Entry<String,Map<String,List<TRowRange>>> entry : filesGBL.entrySet()) {
    +          HostAndPort location = HostAndPort.fromString(entry.getKey());
    +          Map<String,List<TRowRange>> allFiles = entry.getValue();
    +
    +          Future<ProcessedFiles> future = execSrv.submit(new FilesProcessor(tinfo, location, allFiles));
    +          futures.add(future);
    +        }
    +
    +        for (Future<ProcessedFiles> future : futures) {
    +          try {
    +            ProcessedFiles pfiles = future.get();
    +            if (pfiles.summaries.getTotalFiles() > 0) {
    +              partitionSummaries.merge(pfiles.summaries, factory);
    +            }
    +
    +            failedFiles.addAll(pfiles.failedFiles);
    +          } catch (InterruptedException | ExecutionException e) {
    +            throw new AccumuloException(e);
    +          }
    +        }
    +
    +        if (failedFiles.size() > 0) {
    +          // get new locations just for failed files
    +          Predicate<String> fileSelector = hashingFileSelector.and(file -> failedFiles.contains(file));
    +          filesGBL = getFilesGroupedByLocation(fileSelector);
    +          UtilWaitThread.sleep(250);
    +        } else {
    +          break;
    +        }
    +      }
    +    } finally {
    +      execSrv.shutdownNow();
    +    }
    +
    +    return partitionSummaries;
    +  }
    +
    +  public static interface FileSystemResolver {
    +    FileSystem get(Path file);
    +  }
    +
    +  /**
    +   * This method will read summaries from a set of files.
    +   */
    +  public SummaryCollection processFiles(FileSystemResolver volMgr, Map<String,List<TRowRange>> files, BlockCache summaryCache, BlockCache indexCache,
    +      ExecutorService srp) {
    +    SummaryCollection fileSummaries = new SummaryCollection();
    +
    +    List<Future<SummaryCollection>> futures = new ArrayList<>();
    +    for (Entry<String,List<TRowRange>> entry : files.entrySet()) {
    +      Future<SummaryCollection> future = srp.submit(() -> {
    +        List<RowRange> rrl = Lists.transform(entry.getValue(), RowRange::new);
    +        return getSummaries(volMgr, entry.getKey(), rrl, summaryCache, indexCache);
    +      });
    +      futures.add(future);
    +    }
    +
    +    for (Future<SummaryCollection> future : futures) {
    +      try {
    +        fileSummaries.merge(future.get(), factory);
    +      } catch (ExecutionException | InterruptedException e) {
    +        throw new RuntimeException(e);
    --- End diff --
    
    Should be a good citizen and re-set the interrupted state when catching `InterruptedExeption`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103815654
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/FileOperations.java ---
    @@ -301,6 +302,16 @@ public RateLimiter getRateLimiter() {
           return this;
         }
     
    +    public boolean isAccumuloStartEnabled() {
    +      return enableAccumuloStart;
    +    }
    +
    +    @Override
    +    public OpenWriterOperation setAccumuloStartEnabled(boolean enableAccumuloStart) {
    --- End diff --
    
    I added documentation to the method declaration in the super type. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103721888
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java ---
    @@ -121,8 +128,22 @@ public NewTableConfiguration setProperties(Map<String,String> prop) {
        */
       public NewTableConfiguration enableSampling(SamplerConfiguration samplerConfiguration) {
         requireNonNull(samplerConfiguration);
    -    SamplerConfigurationImpl.checkDisjoint(properties, samplerConfiguration);
    -    this.samplerConfiguration = samplerConfiguration;
    +    Map<String,String> tmp = new SamplerConfigurationImpl(samplerConfiguration).toTablePropertiesMap();
    --- End diff --
    
    Maybe a static utility to see if we can do an (unsafe) cast to `SamplerConfigurationImpl` to avoid an unnecessary object creation?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105753115
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java ---
    @@ -0,0 +1,513 @@
    +/*
    + * 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 java.io.UncheckedIOException;
    +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.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.function.Predicate;
    +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.impl.thrift.TRowRange;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaryRequest;
    +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.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.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.TextUtil;
    +import org.apache.accumulo.fate.util.UtilWaitThread;
    +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.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()}.
    + * <li> {@link #gather()} will call make RPC calls to multiple tservers to remotely execute {@link #processPartition(int, int)}
    + * <li> {@link #processPartition(int, int)} will make RPC calls to multiple tserver to remotely execute
    + * {@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<>();
    +    }
    +  }
    +
    +  private class FilesProcessor implements Callable<ProcessedFiles> {
    +
    +    HostAndPort location;
    +    Map<String,List<TRowRange>> allFiles;
    +    private TInfo tinfo;
    +
    +    public FilesProcessor(TInfo tinfo, HostAndPort location, Map<String,List<TRowRange>> allFiles) {
    +      this.location = location;
    +      this.allFiles = allFiles;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public ProcessedFiles call() throws Exception {
    +      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;
    +          }
    +
    +          TSummaries tSums = null;
    +          try {
    +            tSums = client.getSummariesFromFiles(tinfo, ctx.rpcCreds(), getRequest(), files);
    +          } catch (TApplicationException tae) {
    +            throw tae;
    +          } catch (TTransportException e) {
    +            pfiles.failedFiles.addAll(files.keySet());
    +            continue;
    +          }
    +
    +          pfiles.summaries.merge(new SummaryCollection(tSums), factory);
    +        }
    +      } finally {
    +        ThriftUtil.returnClient(client);
    +      }
    +
    +      return pfiles;
    +    }
    +  }
    +
    +  /**
    +   * 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 SummaryCollection processPartition(int modulus, int remainder) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
    +
    +    Predicate<String> hashingFileSelector = file -> Math.abs(Hashing.murmur3_32().hashString(file).asInt()) % modulus == remainder;
    +
    +    Map<String,Map<String,List<TRowRange>>> filesGBL = getFilesGroupedByLocation(hashingFileSelector);
    +
    +    SummaryCollection partitionSummaries = new SummaryCollection();
    +
    +    ExecutorService execSrv = Executors.newCachedThreadPool();
    +    try {
    +      TInfo tinfo = Tracer.traceInfo();
    +      while (true) {
    +        Set<String> failedFiles = new HashSet<>();
    +
    +        List<Future<ProcessedFiles>> futures = new ArrayList<>();
    +        for (Entry<String,Map<String,List<TRowRange>>> entry : filesGBL.entrySet()) {
    +          HostAndPort location = HostAndPort.fromString(entry.getKey());
    +          Map<String,List<TRowRange>> allFiles = entry.getValue();
    +
    +          Future<ProcessedFiles> future = execSrv.submit(new FilesProcessor(tinfo, location, allFiles));
    +          futures.add(future);
    +        }
    +
    +        for (Future<ProcessedFiles> future : futures) {
    +          try {
    +            ProcessedFiles pfiles = future.get();
    +            if (pfiles.summaries.getTotalFiles() > 0) {
    +              partitionSummaries.merge(pfiles.summaries, factory);
    +            }
    +
    +            failedFiles.addAll(pfiles.failedFiles);
    +          } catch (InterruptedException | ExecutionException e) {
    +            throw new AccumuloException(e);
    +          }
    +        }
    +
    +        if (failedFiles.size() > 0) {
    +          // get new locations just for failed files
    +          Predicate<String> fileSelector = hashingFileSelector.and(file -> failedFiles.contains(file));
    +          filesGBL = getFilesGroupedByLocation(fileSelector);
    +          UtilWaitThread.sleep(250);
    +        } else {
    +          break;
    +        }
    +      }
    +    } finally {
    +      execSrv.shutdownNow();
    +    }
    +
    +    return partitionSummaries;
    +  }
    +
    +  public static interface FileSystemResolver {
    +    FileSystem get(Path file);
    +  }
    +
    +  /**
    +   * This method will read summaries from a set of files.
    +   */
    +  public SummaryCollection processFiles(FileSystemResolver volMgr, Map<String,List<TRowRange>> files, BlockCache summaryCache, BlockCache indexCache,
    +      ExecutorService srp) {
    +    SummaryCollection fileSummaries = new SummaryCollection();
    +
    +    List<Future<SummaryCollection>> futures = new ArrayList<>();
    +    for (Entry<String,List<TRowRange>> entry : files.entrySet()) {
    +      Future<SummaryCollection> future = srp.submit(() -> {
    +        List<RowRange> rrl = Lists.transform(entry.getValue(), RowRange::new);
    +        return getSummaries(volMgr, entry.getKey(), rrl, summaryCache, indexCache);
    +      });
    +      futures.add(future);
    +    }
    +
    +    for (Future<SummaryCollection> future : futures) {
    +      try {
    +        fileSummaries.merge(future.get(), factory);
    +      } catch (ExecutionException | InterruptedException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +
    +    return fileSummaries;
    +  }
    +
    +  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 Callable<SummaryCollection> {
    +
    +    private int remainder;
    +    private int modulus;
    +    private TInfo tinfo;
    +
    +    GatherRequest(TInfo tinfo, int remainder, int modulus) {
    +      this.remainder = remainder;
    +      this.modulus = modulus;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public SummaryCollection call() throws Exception {
    +      TSummaryRequest req = getRequest();
    +
    +      ClientContext cct = new ClientContext(ctx.getInstance(), ctx.getCredentials(), ctx.getConfiguration()) {
    +        @Override
    +        public long getClientTimeoutInMillis() {
    +          return Math.max(super.getClientTimeoutInMillis(), 10 * 60 * 1000);
    +        }
    +      };
    +
    +      TSummaries tSums = ServerClient.execute(cct, c -> c.getSummariesForPartition(tinfo, ctx.rpcCreds(), req, modulus, remainder));
    +      return new SummaryCollection(tSums);
    +    }
    +  }
    +
    +  public SummaryCollection gather() throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
    +
    +    ExecutorService es = Executors.newFixedThreadPool(ctx.getConnector().instanceOperations().getTabletServers().size());
    --- End diff --
    
    Sounds good.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r106905869
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java ---
    @@ -1812,7 +1884,26 @@ public TSummaries getSummariesFromFiles(TInfo tinfo, TCredentials credentials, T
           BlockCache summaryCache = resourceManager.getSummaryCache();
           BlockCache indexCache = resourceManager.getIndexCache();
           FileSystemResolver volMgr = p -> fs.getVolumeByPath(p).getFileSystem();
    -      return new Gatherer(TabletServer.this, request, tableCfg).processFiles(volMgr, files, summaryCache, indexCache, srp).toThrift();
    +      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();
    --- End diff --
    
    That exception is used in a few other places (that are not scan) to indicate no such session id.  I was just following that.  The exception should be named no such session exception.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103723011
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/SummaryRetriever.java ---
    @@ -0,0 +1,107 @@
    +/*
    + * 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 {
    +
    +  /**
    +   * Summary data is only retrieved from data that has been written to files. Data recently written to Accumulo may be in memory and there will not show up in
    +   * summary data. Setting this option to true force tablets in the range to minor compact before summary data is retrieved. By default the table will not be
    +   * flushed before retrieving summary data.
    +   *
    +   * @return this
    +   */
    +  SummaryRetriever flush(boolean shouldFlush);
    +
    +  /**
    +   * Allows optionally setting start row before retrieving data. The start row is not inclusive.
    --- End diff --
    
    Nit: Move "optionally" to it's own sentence (e.g. "The start row is not inclusive. Calling this method is optional.") or lead with it (e.g. "Optional, allows setting a start row").
    
    Same applies to the other methods.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103731048
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/FamilySummarizer.java ---
    @@ -0,0 +1,36 @@
    +/*
    + * 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.summary.CountingSummarizer;
    +import org.apache.accumulo.core.data.ArrayByteSequence;
    +import org.apache.accumulo.core.data.ByteSequence;
    +
    +public class FamilySummarizer extends CountingSummarizer<ByteSequence> {
    --- End diff --
    
    needs javadoc & since tag


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105253027
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java ---
    @@ -0,0 +1,513 @@
    +/*
    + * 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 java.io.UncheckedIOException;
    +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.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.function.Predicate;
    +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.impl.thrift.TRowRange;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaryRequest;
    +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.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.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.TextUtil;
    +import org.apache.accumulo.fate.util.UtilWaitThread;
    +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.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()}.
    + * <li> {@link #gather()} will call make RPC calls to multiple tservers to remotely execute {@link #processPartition(int, int)}
    + * <li> {@link #processPartition(int, int)} will make RPC calls to multiple tserver to remotely execute
    + * {@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<>();
    +    }
    +  }
    +
    +  private class FilesProcessor implements Callable<ProcessedFiles> {
    +
    +    HostAndPort location;
    +    Map<String,List<TRowRange>> allFiles;
    +    private TInfo tinfo;
    +
    +    public FilesProcessor(TInfo tinfo, HostAndPort location, Map<String,List<TRowRange>> allFiles) {
    +      this.location = location;
    +      this.allFiles = allFiles;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public ProcessedFiles call() throws Exception {
    +      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;
    +          }
    +
    +          TSummaries tSums = null;
    +          try {
    +            tSums = client.getSummariesFromFiles(tinfo, ctx.rpcCreds(), getRequest(), files);
    +          } catch (TApplicationException tae) {
    +            throw tae;
    +          } catch (TTransportException e) {
    +            pfiles.failedFiles.addAll(files.keySet());
    +            continue;
    +          }
    +
    +          pfiles.summaries.merge(new SummaryCollection(tSums), factory);
    +        }
    +      } finally {
    +        ThriftUtil.returnClient(client);
    +      }
    +
    +      return pfiles;
    +    }
    +  }
    +
    +  /**
    +   * 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 SummaryCollection processPartition(int modulus, int remainder) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
    +
    +    Predicate<String> hashingFileSelector = file -> Math.abs(Hashing.murmur3_32().hashString(file).asInt()) % modulus == remainder;
    +
    +    Map<String,Map<String,List<TRowRange>>> filesGBL = getFilesGroupedByLocation(hashingFileSelector);
    +
    +    SummaryCollection partitionSummaries = new SummaryCollection();
    +
    +    ExecutorService execSrv = Executors.newCachedThreadPool();
    +    try {
    +      TInfo tinfo = Tracer.traceInfo();
    +      while (true) {
    +        Set<String> failedFiles = new HashSet<>();
    +
    +        List<Future<ProcessedFiles>> futures = new ArrayList<>();
    +        for (Entry<String,Map<String,List<TRowRange>>> entry : filesGBL.entrySet()) {
    +          HostAndPort location = HostAndPort.fromString(entry.getKey());
    +          Map<String,List<TRowRange>> allFiles = entry.getValue();
    +
    +          Future<ProcessedFiles> future = execSrv.submit(new FilesProcessor(tinfo, location, allFiles));
    +          futures.add(future);
    +        }
    +
    +        for (Future<ProcessedFiles> future : futures) {
    +          try {
    +            ProcessedFiles pfiles = future.get();
    +            if (pfiles.summaries.getTotalFiles() > 0) {
    +              partitionSummaries.merge(pfiles.summaries, factory);
    +            }
    +
    +            failedFiles.addAll(pfiles.failedFiles);
    +          } catch (InterruptedException | ExecutionException e) {
    +            throw new AccumuloException(e);
    +          }
    +        }
    +
    +        if (failedFiles.size() > 0) {
    +          // get new locations just for failed files
    +          Predicate<String> fileSelector = hashingFileSelector.and(file -> failedFiles.contains(file));
    +          filesGBL = getFilesGroupedByLocation(fileSelector);
    +          UtilWaitThread.sleep(250);
    +        } else {
    +          break;
    +        }
    +      }
    +    } finally {
    +      execSrv.shutdownNow();
    +    }
    +
    +    return partitionSummaries;
    +  }
    +
    +  public static interface FileSystemResolver {
    +    FileSystem get(Path file);
    +  }
    +
    +  /**
    +   * This method will read summaries from a set of files.
    +   */
    +  public SummaryCollection processFiles(FileSystemResolver volMgr, Map<String,List<TRowRange>> files, BlockCache summaryCache, BlockCache indexCache,
    +      ExecutorService srp) {
    +    SummaryCollection fileSummaries = new SummaryCollection();
    +
    +    List<Future<SummaryCollection>> futures = new ArrayList<>();
    +    for (Entry<String,List<TRowRange>> entry : files.entrySet()) {
    +      Future<SummaryCollection> future = srp.submit(() -> {
    +        List<RowRange> rrl = Lists.transform(entry.getValue(), RowRange::new);
    +        return getSummaries(volMgr, entry.getKey(), rrl, summaryCache, indexCache);
    +      });
    +      futures.add(future);
    +    }
    +
    +    for (Future<SummaryCollection> future : futures) {
    +      try {
    +        fileSummaries.merge(future.get(), factory);
    +      } catch (ExecutionException | InterruptedException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +
    +    return fileSummaries;
    +  }
    +
    +  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 Callable<SummaryCollection> {
    +
    +    private int remainder;
    +    private int modulus;
    +    private TInfo tinfo;
    +
    +    GatherRequest(TInfo tinfo, int remainder, int modulus) {
    +      this.remainder = remainder;
    +      this.modulus = modulus;
    +      this.tinfo = tinfo;
    +    }
    +
    +    @Override
    +    public SummaryCollection call() throws Exception {
    +      TSummaryRequest req = getRequest();
    +
    +      ClientContext cct = new ClientContext(ctx.getInstance(), ctx.getCredentials(), ctx.getConfiguration()) {
    +        @Override
    +        public long getClientTimeoutInMillis() {
    +          return Math.max(super.getClientTimeoutInMillis(), 10 * 60 * 1000);
    +        }
    +      };
    +
    +      TSummaries tSums = ServerClient.execute(cct, c -> c.getSummariesForPartition(tinfo, ctx.rpcCreds(), req, modulus, remainder));
    +      return new SummaryCollection(tSums);
    +    }
    +  }
    +
    +  public SummaryCollection gather() throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
    +
    +    ExecutorService es = Executors.newFixedThreadPool(ctx.getConnector().instanceOperations().getTabletServers().size());
    +    try {
    +
    +      int numFiles = countFiles();
    +
    +      log.debug("Gathering summaries from {} files", numFiles);
    +
    +      // have each tablet server process ~100K files
    +      int numRequest = Math.max(numFiles / 100_000, 1);
    +
    +      List<Future<SummaryCollection>> futures = new ArrayList<>();
    +
    +      TInfo tinfo = Tracer.traceInfo();
    +      for (int i = 0; i < numRequest; i++) {
    +        futures.add(es.submit(new GatherRequest(tinfo, i, numRequest)));
    +      }
    +
    +      SummaryCollection allSummaries = new SummaryCollection();
    +      for (Future<SummaryCollection> future : futures) {
    +        try {
    +          allSummaries.merge(future.get(), factory);
    +        } catch (InterruptedException | ExecutionException e) {
    +          throw new AccumuloException(e);
    --- End diff --
    
    re-set interrupted state?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103727903
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/FileOperations.java ---
    @@ -301,6 +302,16 @@ public RateLimiter getRateLimiter() {
           return this;
         }
     
    +    public boolean isAccumuloStartEnabled() {
    +      return enableAccumuloStart;
    +    }
    +
    +    @Override
    +    public OpenWriterOperation setAccumuloStartEnabled(boolean enableAccumuloStart) {
    --- End diff --
    
    This is really vague to me. I don't know what this actually means..


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103731627
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/summary/SummarizerConfiguration.java ---
    @@ -0,0 +1,238 @@
    +/*
    + * 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();
    +    }
    +
    +    /**
    +     * Setting this is optional. If not set, an id is generated using hashing that will likely be unique.
    +     *
    +     * @param propId
    +     *          This is id is used when converting a {@link SummarizerConfiguration} to table properties. Since tables can have multiple summarizers, make sure
    +     *          its unique.
    +     */
    +    public Builder setPropertyId(String propId) {
    +      Preconditions.checkArgument(propId.matches("\\w+"), "Config Id %s is not alphanum", propId);
    +      this.configId = propId;
    +      return this;
    +    }
    +
    +    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;
    +    }
    +
    +    public Builder addOption(String key, long value) {
    +      return addOption(key, value + "");
    +    }
    +
    +    public Builder addOptions(String... keyValuePairs) {
    +      Preconditions.checkArgument(keyValuePairs.length % 2 == 0, "Require an even number of arguments, got %s", keyValuePairs.length);
    --- End diff --
    
    Check for positive numbers too?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103725643
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---
    @@ -1661,4 +1676,138 @@ public Locations locate(String tableName, Collection<Range> ranges) throws Accum
     
         return new LoctionsImpl(binnedRanges);
       }
    +
    +  @Override
    +  public SummaryRetriever getSummaries(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);
    +        }
    +
    +        ClientContext cct = new ClientContext(context.getInstance(), context.getCredentials(), context.getConfiguration()) {
    +          @Override
    +          public long getClientTimeoutInMillis() {
    +            return Math.max(super.getClientTimeoutInMillis(), 60 * 60 * 1000);
    +          }
    +        };
    +        TSummaries ret = ServerClient.execute(cct, c -> c.getSummaries(Tracer.traceInfo(), context.rpcCreds(), request));
    +        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)));
    --- End diff --
    
    `Collections.singleton` instead of `new HashSet<>`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103724151
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---
    @@ -1661,4 +1676,138 @@ public Locations locate(String tableName, Collection<Range> ranges) throws Accum
     
         return new LoctionsImpl(binnedRanges);
       }
    +
    +  @Override
    +  public SummaryRetriever getSummaries(String tableName) {
    +
    +    return new SummaryRetriever() {
    --- End diff --
    
    Pull this out to a private static class instead of doing it inline? Would help testing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105246613
  
    --- Diff: test/src/main/java/org/apache/accumulo/test/ShellServerIT.java ---
    @@ -1894,4 +1896,71 @@ private String getTableId(String tableName) throws Exception {
         return null;
       }
     
    +  @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("config -t summary -s table.summarizer.fam.opt."+CountingSummarizer.INGNORE_DELETES_OPT+"=false");
    +    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");
    +    Assert.assertTrue(output.matches("(?sm).*^.*deletes\\s+=\\s+0.*$.*"));
    --- End diff --
    
    Please provide some debugging here so that we would know what the output was if the assertion ever fails.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104507030
  
    --- Diff: 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.
    --- End diff --
    
    This is cool!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103722621
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/SummaryRetriever.java ---
    @@ -0,0 +1,107 @@
    +/*
    + * 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 {
    +
    +  /**
    +   * Summary data is only retrieved from data that has been written to files. Data recently written to Accumulo may be in memory and there will not show up in
    --- End diff --
    
    Using 3-person declarative (the style used on the other methods) is good for Javadoc, IMO.
    
    e.g. `Forces a flush of data in tablets before summary data is retrieved. ....:


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104702376
  
    --- Diff: 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));
    --- End diff --
    
    I am not sure about this.  The code is already so short.  But if I did do it, `LongSummarizer` is too generic of a name because it summing.   So maybe it could be called `SummingSummarizer`. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103783042
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---
    @@ -1661,4 +1676,138 @@ public Locations locate(String tableName, Collection<Range> ranges) throws Accum
     
         return new LoctionsImpl(binnedRanges);
       }
    +
    +  @Override
    +  public SummaryRetriever getSummaries(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);
    +        }
    +
    +        ClientContext cct = new ClientContext(context.getInstance(), context.getCredentials(), context.getConfiguration()) {
    +          @Override
    +          public long getClientTimeoutInMillis() {
    +            return Math.max(super.getClientTimeoutInMillis(), 60 * 60 * 1000);
    --- End diff --
    
    Ugh.  Glad you found this.  I meant to circle back to this, but forgot about it.  When I did this hack I Was worried about the case where getting summaries is slow (because none are in cache) and a result the client times and initiates a new operation (while threads for old operation are still churning).  I need a better solution.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104510474
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java ---
    @@ -117,6 +121,7 @@ String getFormatDescription() {
        * @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????");
    --- End diff --
    
    lol, maybe just a "this should not happen" instead ;)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103725852
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---
    @@ -1661,4 +1676,138 @@ public Locations locate(String tableName, Collection<Range> ranges) throws Accum
     
         return new LoctionsImpl(binnedRanges);
       }
    +
    +  @Override
    +  public SummaryRetriever getSummaries(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);
    +        }
    +
    +        ClientContext cct = new ClientContext(context.getInstance(), context.getCredentials(), context.getConfiguration()) {
    +          @Override
    +          public long getClientTimeoutInMillis() {
    +            return Math.max(super.getClientTimeoutInMillis(), 60 * 60 * 1000);
    +          }
    +        };
    +        TSummaries ret = ServerClient.execute(cct, c -> c.getSummaries(Tracer.traceInfo(), context.rpcCreds(), request));
    +        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());
    --- End diff --
    
    Such wizardy.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on the issue:

    https://github.com/apache/accumulo/pull/224
  
    @joshelser I have been thinking about the issues you brought up regarding the thread pools and RPC timeouts.  These issues will cause problems when there are problems on the cluster (and that never happens :).  I am going to work on fixing them.  Thinking of introducing configurable thread pools as needed.  Also going to make RPC work like scanner with sessions and sessions timeouts.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103817828
  
    --- Diff: docs/src/main/asciidoc/chapters/summaries.txt ---
    @@ -0,0 +1,209 @@
    +// 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 define 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
    --- End diff --
    
    I added a sentence mentioning what package to look in for summarizers that ship with Accumulo.  I really really with there was a good way to link to javadocs from the user manual.  Could create much better documentation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on the issue:

    https://github.com/apache/accumulo/pull/224
  
    TODO now that there is summary cache, need to include it in the tservers memory size sanity checks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104520129
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java ---
    @@ -0,0 +1,543 @@
    +/*
    + * 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()));
    --- End diff --
    
    Would the `symbolTable` ever not have a mapping for the key?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105750617
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---
    @@ -1661,4 +1676,138 @@ public Locations locate(String tableName, Collection<Range> ranges) throws Accum
     
         return new LoctionsImpl(binnedRanges);
       }
    +
    +  @Override
    +  public SummaryRetriever getSummaries(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);
    +        }
    +
    +        ClientContext cct = new ClientContext(context.getInstance(), context.getCredentials(), context.getConfiguration()) {
    +          @Override
    +          public long getClientTimeoutInMillis() {
    +            return Math.max(super.getClientTimeoutInMillis(), 60 * 60 * 1000);
    --- End diff --
    
    I switched this to a timeout of zero, which is what some other long RPC ops currently do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105268731
  
    --- Diff: 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
    +  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++;
    --- End diff --
    
    > Currently it would bork the compaction, so the counter issue would be moot.
    
    SGTM can deal with your other thoughts as a follow-on improvement.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105753424
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---
    @@ -1661,4 +1676,138 @@ public Locations locate(String tableName, Collection<Range> ranges) throws Accum
     
         return new LoctionsImpl(binnedRanges);
       }
    +
    +  @Override
    +  public SummaryRetriever getSummaries(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);
    +        }
    +
    +        ClientContext cct = new ClientContext(context.getInstance(), context.getCredentials(), context.getConfiguration()) {
    +          @Override
    +          public long getClientTimeoutInMillis() {
    +            return Math.max(super.getClientTimeoutInMillis(), 60 * 60 * 1000);
    --- End diff --
    
    Correct me if I'm wrong, but don't we usually do that for client-facing RPCs? As-in, while the client is still there (connected), we let the RPC sit. My worry is that if we do this inside the TabletServer, the risk is larger since that is a Java process which we expect to run for days/months/indefinitely.
    
    Precedence to the contrary is fine, as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104511336
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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 overTableId(String tableId);
    +
    +    ColumnOptions overTableId(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 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() {
    +            Iterator<TabletMetadata> iter = tmi.iterator();
    +            return new Iterator<TabletMetadata>() {
    --- End diff --
    
    Maybe lift this `Iterator` into it's own class for test-ability. I think the inline `Iterable` wrapper is fine. @


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103731380
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/summary/SummarizerConfiguration.java ---
    @@ -0,0 +1,238 @@
    +/*
    + * 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();
    +    }
    +
    +    /**
    +     * Setting this is optional. If not set, an id is generated using hashing that will likely be unique.
    +     *
    +     * @param propId
    +     *          This is id is used when converting a {@link SummarizerConfiguration} to table properties. Since tables can have multiple summarizers, make sure
    +     *          its unique.
    +     */
    +    public Builder setPropertyId(String propId) {
    +      Preconditions.checkArgument(propId.matches("\\w+"), "Config Id %s is not alphanum", propId);
    +      this.configId = propId;
    +      return this;
    +    }
    +
    +    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;
    +    }
    +
    +    public Builder addOption(String key, long value) {
    +      return addOption(key, value + "");
    --- End diff --
    
    `Integer.toString(value)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103826212
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java ---
    @@ -121,8 +128,22 @@ public NewTableConfiguration setProperties(Map<String,String> prop) {
        */
       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, summarizerProps);
    +    this.samplerProps = tmp;
    --- End diff --
    
    I cleaned this up and simplified it in commit 40ed96f.  That removed the unnecessary checks and hopefully made it a little more clear.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105246402
  
    --- Diff: 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_STAMP_STAT = "minStamp";
    +  public static final String MAX_STAMP_STAT = "maxStamp";
    --- End diff --
    
    s/STAMP/TIMESTAMP/ ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103815757
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java ---
    @@ -92,10 +102,11 @@ public RFileWriter build() throws IOException {
           } 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);
    --- End diff --
    
    It presumed that this code us running in a client process, so do not want to use Accumulo start to load classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103730257
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/summary/Summary.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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
    + */
    --- End diff --
    
    needs since tag


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105245989
  
    --- Diff: test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java ---
    @@ -0,0 +1,759 @@
    +/*
    + * 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.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.Set;
    +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.Summarizer;
    +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.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(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().getSummaries(table).flush(false).retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    LongSummaryStatistics stats = getTimestampStats(table, c);
    +
    +    summaries = c.tableOperations().getSummaries(table).flush(true).retrieve();
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 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().getSummaries(table).retrieve();
    +
    +    checkSummaries(summaries, sc1, 2, 0, 0, "total", 100_002l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 1l);
    +
    +    bw.close();
    +
    +    c.tableOperations().compact(table, new CompactionConfig().setWait(true));
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // split tablet into two
    +    String sp1 = String.format("r%09x", 50_000);
    +    addSplits(table, c, sp1);
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // compact 2nd tablet
    +    c.tableOperations().compact(table, new CompactionConfig().setStartRow(new Text(sp1)).setWait(true));
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    checkSummaries(summaries, sc1, 2, 0, 1, "total", 113_999l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // get summaries for first tablet
    +    stats = getTimestampStats(table, c, sp1, null);
    +    summaries = c.tableOperations().getSummaries(table).startRow(sp1).retrieve();
    +    checkSummaries(summaries, sc1, 1, 0, 0, "total", 49_999l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // compact all tablets and regenerate all summaries
    +    c.tableOperations().compact(table, new CompactionConfig());
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    stats = getTimestampStats(table, c);
    +    checkSummaries(summaries, sc1, 2, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    summaries = c.tableOperations().getSummaries(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");
    +    Assert.assertTrue(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().getSummaries(table).retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    c.tableOperations().addSummarizers(table, sc1);
    +    c.tableOperations().compact(table, new CompactionConfig().setWait(true));
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    checkSummaries(summaries, sc1, 2, 0, 0, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 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().getSummaries(table).withConfiguration(sc2).retrieve();
    +    Assert.assertEquals(1, summaries.size());
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    summaries = c.tableOperations().getSummaries(table).withConfiguration(sc1).retrieve();
    +    Assert.assertEquals(1, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    // retrieve a non-existant summary
    +    SummarizerConfiguration sc3 = SummarizerConfiguration.builder(KeySizeSummarizer.class.getName()).addOption("maxLen", "256").build();
    +    summaries = c.tableOperations().getSummaries(table).withConfiguration(sc3).retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    summaries = c.tableOperations().getSummaries(table).withConfiguration(sc1, sc2).retrieve();
    +    Assert.assertEquals(2, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    summaries = c.tableOperations().getSummaries(table).retrieve();
    +    Assert.assertEquals(2, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*BasicSummarizer \\{\\}.*").retrieve();
    +    Assert.assertEquals(1, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +
    +    summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*KeySizeSummarizer \\{maxLen=512\\}.*").retrieve();
    +    Assert.assertEquals(1, summaries.size());
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*KeySizeSummarizer \\{maxLen=256\\}.*").retrieve();
    +    Assert.assertEquals(0, summaries.size());
    +
    +    summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*BasicSummarizer \\{\\}.*").withConfiguration(sc2).retrieve();
    +    Assert.assertEquals(2, summaries.size());
    +    checkSummary(summaries, sc1, "total", 100_000l, "minStamp", stats.getMin(), "maxStamp", stats.getMax(), "deletes", 0l);
    +    checkSummary(summaries, sc2, "len=14", 100_000l);
    +
    +    // Ensure a bad regex fails fast.
    +    try {
    +      summaries = c.tableOperations().getSummaries(table).withMatchingConfiguration(".*KeySizeSummarizer {maxLen=256}.*").retrieve();
    +      Assert.fail("Bad regex should have caused exception");
    +    } catch (PatternSyntaxException e) {}
    +  }
    +
    +  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));
    +    }
    +  }
    +
    +  public static class FooFilter extends Filter {
    +    @Override
    +    public boolean accept(Key k, Value v) {
    +      return !k.getRowData().toString().contains("foo");
    +    }
    +  }
    +
    +  public static class FooCS extends CompactionStrategy {
    --- End diff --
    
    Can you provide a better name and/or javadoc?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105248600
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java ---
    @@ -0,0 +1,513 @@
    +/*
    + * 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 java.io.UncheckedIOException;
    +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.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.function.Predicate;
    +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.impl.thrift.TRowRange;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaries;
    +import org.apache.accumulo.core.client.impl.thrift.TSummaryRequest;
    +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.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.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.TextUtil;
    +import org.apache.accumulo.fate.util.UtilWaitThread;
    +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.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()}.
    + * <li> {@link #gather()} will call make RPC calls to multiple tservers to remotely execute {@link #processPartition(int, int)}
    + * <li> {@link #processPartition(int, int)} will make RPC calls to multiple tserver to remotely execute
    + * {@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();
    --- End diff --
    
    Oh my...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105245446
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java ---
    @@ -808,4 +812,64 @@ void setSamplerConfiguration(String tableName, SamplerConfiguration samplerConfi
        * @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. The
    +   * size of this cache is determined by the property {code tserver.cache.summary.size}. When summary data for a file is not present, it will be retrieved using
    +   * threads on the tserver. The property {@code tserver.summary.retrieval.threads} determines the max number of threads the tserver will use for this.
    +   *
    +   * <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 getSummaries(String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
    --- End diff --
    
    Is there a better method name than `getSummaries`? I didn't notice it was strange until I got to the tests where you have (paraphrasing)
    
    `conn.tableOperations().getSummaries("my_table").retrieve()`
    
    The `SummaryRetriever` is more like a builder, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105704840
  
    --- Diff: test/src/main/java/org/apache/accumulo/test/ShellServerIT.java ---
    @@ -1894,4 +1896,71 @@ private String getTableId(String tableName) throws Exception {
         return null;
       }
     
    +  @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("config -t summary -s table.summarizer.fam.opt."+CountingSummarizer.INGNORE_DELETES_OPT+"=false");
    +    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");
    +    Assert.assertTrue(output.matches("(?sm).*^.*deletes\\s+=\\s+0.*$.*"));
    --- End diff --
    
    In 7a9fa75 I created a new utility method called `assertMatches` that prints the regex and output on failure.  A nice improvement.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103726513
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java ---
    @@ -134,6 +136,20 @@ public static void setSampler(Job job, SamplerConfiguration samplerConfig) {
         FileOutputConfigurator.setSampler(CLASS, job.getConfiguration(), samplerConfig);
       }
     
    +  /**
    +   * Specify a list of summarizer configurations to create summary data in the output file. Each Key Value written will be passed to the configured
    --- End diff --
    
    s/Specify/Specifies/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103726395
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java ---
    @@ -136,6 +138,20 @@ public static void setSampler(JobConf job, SamplerConfiguration samplerConfig) {
         FileOutputConfigurator.setSampler(CLASS, job, samplerConfig);
       }
     
    +  /**
    +   * Specify a list of summarizer configurations to create summary data in the output file. Each Key Value written will be passed to the configured
    --- End diff --
    
    s/Specify/Specifies/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104511568
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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 overTableId(String tableId);
    +
    +    ColumnOptions overTableId(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 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() {
    +            Iterator<TabletMetadata> iter = tmi.iterator();
    +            return new Iterator<TabletMetadata>() {
    +
    +              private boolean sawLast = false;
    +
    +              @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;
    +              }
    +            };
    +          }
    +        };
    +      } 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 overTableId(String tableId) {
    --- End diff --
    
    How about `overUserTableId` instead? Implies that providing the root or metadata table IDs instead would throw an error.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103824011
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java ---
    @@ -121,8 +128,22 @@ public NewTableConfiguration setProperties(Map<String,String> prop) {
        */
       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, summarizerProps);
    +    this.samplerProps = tmp;
    --- End diff --
    
    Checking summarizerProps is completely unnecessary here. The function is used in three places.  Only in one of those three places does it actually need to check both.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105683470
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java ---
    @@ -48,6 +53,83 @@ void gatherInformation(MajorCompactionRequest request) {}
         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);
    +
    +        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);
    --- End diff --
    
    Yikes!  Thats a really nice find.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104563673
  
    --- Diff: docs/src/main/asciidoc/chapters/summaries.txt ---
    @@ -0,0 +1,211 @@
    +// 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.
    +
    +=== Configuring
    +
    +The following tablet server and table properties configure summarization.
    +
    +* <<appendices/config.txt#_tserver_cache_summary_size>>
    +* <<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 : 
    --- End diff --
    
    So cool.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r104519924
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java ---
    @@ -0,0 +1,543 @@
    +/*
    + * 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 diff --
    
    Is there a lower-bound on when you want to do this? (aside from the sanity check of <= 1 from above)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103723297
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/SummaryRetriever.java ---
    @@ -0,0 +1,107 @@
    +/*
    + * 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 {
    +
    +  /**
    +   * Summary data is only retrieved from data that has been written to files. Data recently written to Accumulo may be in memory and there will not show up in
    +   * summary data. Setting this option to true force tablets in the range to minor compact before summary data is retrieved. By default the table will not be
    +   * flushed before retrieving summary data.
    +   *
    +   * @return this
    +   */
    +  SummaryRetriever flush(boolean shouldFlush);
    +
    +  /**
    +   * Allows optionally setting start row before retrieving data. The start row is not inclusive.
    +   */
    +  SummaryRetriever startRow(Text startRow);
    +
    +  /**
    +   * Allows optionally setting start row before retrieving data. The start row is not inclusive.
    +   */
    +  SummaryRetriever startRow(CharSequence startRow);
    +
    +  /**
    +   * Allows optionally setting end row before retrieving data. The end row is inclusive.
    +   */
    +  SummaryRetriever endRow(Text endRow);
    +
    +  /**
    +   * Allows optionally setting end row before retrieving data. The end row is inclusive.
    +   */
    +  SummaryRetriever endRow(CharSequence endRow);
    +
    +  /**
    +   * Gets summaries generated with a configuration that matches the given regex. For a given SummarizationConfiguration it is matched in exactly the following
    +   * way. This allows the regex to match on classname and options.
    --- End diff --
    
    It's not obvious to me what this method is doing under the hood and I don't think the code snippet helps.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r105276306
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java ---
    @@ -808,4 +812,64 @@ void setSamplerConfiguration(String tableName, SamplerConfiguration samplerConfi
        * @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. The
    +   * size of this cache is determined by the property {code tserver.cache.summary.size}. When summary data for a file is not present, it will be retrieved using
    +   * threads on the tserver. The property {@code tserver.summary.retrieval.threads} determines the max number of threads the tserver will use for this.
    +   *
    +   * <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 getSummaries(String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
    --- End diff --
    
    Yeah the current name does not communicate well what it is.  I really don't like the current name.  I tried to think of something better and got sidetracked.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #224: ACCUMULO-4500 ACCUMULO-96 Added summarization

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/224#discussion_r103723473
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java ---
    @@ -808,4 +812,64 @@ void setSamplerConfiguration(String tableName, SamplerConfiguration samplerConfi
        * @since 1.8.0
        */
       SamplerConfiguration getSamplerConfiguration(String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
    +
    +  /**
    +   * This is a entry point for retrieving summaries with optional restrictions.
    +   *
    +   * <p>
    +   * Inorder to retrieve Summaries, the Accumulo user making the request will need the {@link TablePermission#GET_SUMMARIES} table permission.
    --- End diff --
    
    s/Inorder/In order/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---