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

accumulo git commit: ACCUMULO-3521: created tests and minor improvements for iterators:

Repository: accumulo
Updated Branches:
  refs/heads/master 1743ee2e8 -> 34532dcc6


ACCUMULO-3521: created tests and minor improvements for iterators:

	modified:   core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java
	modified:   core/src/main/java/org/apache/accumulo/core/iterators/TypedValueCombiner.java
	modified:   core/src/test/java/org/apache/accumulo/core/iterators/FirstEntryInRowIteratorTest.java
	modified:   core/src/test/java/org/apache/accumulo/core/iterators/user/CombinerTest.java


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

Branch: refs/heads/master
Commit: 34532dcc620fc638685cebd9cbe222ec3f9a5b93
Parents: 1743ee2
Author: Mike Miller <mm...@apache.org>
Authored: Wed Mar 29 16:45:56 2017 -0400
Committer: Mike Miller <mm...@apache.org>
Committed: Tue Apr 4 16:20:49 2017 -0400

----------------------------------------------------------------------
 .../core/iterators/FirstEntryInRowIterator.java | 13 +++---
 .../core/iterators/TypedValueCombiner.java      | 11 +----
 .../iterators/FirstEntryInRowIteratorTest.java  | 22 ++++++----
 .../core/iterators/user/CombinerTest.java       | 46 ++++++++++++++++++++
 4 files changed, 67 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/34532dcc/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java
index 32e6464..17b8f43 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.hadoop.io.Text;
+import org.apache.commons.lang.math.NumberUtils;
 
 public class FirstEntryInRowIterator extends SkippingIterator implements OptionDescriber {
 
@@ -75,7 +76,7 @@ public class FirstEntryInRowIterator extends SkippingIterator implements OptionD
   // this is only ever called immediately after getting "next" entry
   @Override
   protected void consume() throws IOException {
-    if (finished == true || lastRowFound == null)
+    if (finished || lastRowFound == null)
       return;
     int count = 0;
     while (getSource().hasTop() && lastRowFound.equals(getSource().getTopKey().getRow())) {
@@ -139,13 +140,9 @@ public class FirstEntryInRowIterator extends SkippingIterator implements OptionD
 
   @Override
   public boolean validateOptions(Map<String,String> options) {
-    try {
-      String o = options.get(NUM_SCANS_STRING_NAME);
-      if (o != null)
-        Integer.parseInt(o);
-    } catch (Exception e) {
-      throw new IllegalArgumentException("bad integer " + NUM_SCANS_STRING_NAME + ":" + options.get(NUM_SCANS_STRING_NAME), e);
-    }
+    String o = options.get(NUM_SCANS_STRING_NAME);
+    if (o != null && !NumberUtils.isNumber(o))
+      throw new IllegalArgumentException("bad integer " + NUM_SCANS_STRING_NAME + ":" + options.get(NUM_SCANS_STRING_NAME));
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/34532dcc/core/src/main/java/org/apache/accumulo/core/iterators/TypedValueCombiner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/TypedValueCombiner.java b/core/src/main/java/org/apache/accumulo/core/iterators/TypedValueCombiner.java
index 03e4d88..05e3955 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/TypedValueCombiner.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/TypedValueCombiner.java
@@ -131,11 +131,7 @@ public abstract class TypedValueCombiner<V> extends Combiner {
       @SuppressWarnings("unchecked")
       Class<? extends Encoder<V>> clazz = (Class<? extends Encoder<V>>) AccumuloVFSClassLoader.loadClass(encoderClass, Encoder.class);
       encoder = clazz.newInstance();
-    } catch (ClassNotFoundException e) {
-      throw new IllegalArgumentException(e);
-    } catch (InstantiationException e) {
-      throw new IllegalArgumentException(e);
-    } catch (IllegalAccessException e) {
+    } catch (ClassNotFoundException | IllegalAccessException | InstantiationException e) {
       throw new IllegalArgumentException(e);
     }
   }
@@ -190,10 +186,7 @@ public abstract class TypedValueCombiner<V> extends Combiner {
 
   private void setLossyness(Map<String,String> options) {
     String loss = options.get(LOSSY);
-    if (loss == null)
-      lossy = false;
-    else
-      lossy = Boolean.parseBoolean(loss);
+    lossy = loss != null && Boolean.parseBoolean(loss);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/34532dcc/core/src/test/java/org/apache/accumulo/core/iterators/FirstEntryInRowIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/FirstEntryInRowIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/FirstEntryInRowIteratorTest.java
index 34b01bc..2956152 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/FirstEntryInRowIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/FirstEntryInRowIteratorTest.java
@@ -17,31 +17,33 @@
 package org.apache.accumulo.core.iterators;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.TreeMap;
 
+import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.impl.BaseIteratorEnvironment;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.system.CountingIterator;
+import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.junit.Test;
 
 public class FirstEntryInRowIteratorTest {
 
   @SuppressWarnings("unchecked")
-  private static long process(TreeMap<Key,Value> sourceMap, TreeMap<Key,Value> resultMap, Range range, int numScans) throws IOException {
+  private static long process(TreeMap<Key,Value> sourceMap, TreeMap<Key,Value> resultMap, Range range, IteratorSetting iteratorSetting) throws IOException {
     org.apache.accumulo.core.iterators.SortedMapIterator source = new SortedMapIterator(sourceMap);
     CountingIterator counter = new CountingIterator(source);
     FirstEntryInRowIterator feiri = new FirstEntryInRowIterator();
     IteratorEnvironment env = new BaseIteratorEnvironment();
 
-    feiri.init(counter, Collections.singletonMap(FirstEntryInRowIterator.NUM_SCANS_STRING_NAME, Integer.toString(numScans)), env);
+    feiri.init(counter, iteratorSetting.getOptions(), env);
 
-    feiri.seek(range, Collections.EMPTY_SET, false);
+    feiri.seek(range, LocalityGroupUtil.EMPTY_CF_SET, false);
     while (feiri.hasTop()) {
       resultMap.put(feiri.getTopKey(), feiri.getTopValue());
       feiri.next();
@@ -53,12 +55,15 @@ public class FirstEntryInRowIteratorTest {
   public void test() throws IOException {
     TreeMap<Key,Value> sourceMap = new TreeMap<>();
     Value emptyValue = new Value("".getBytes());
+    IteratorSetting iteratorSetting = new IteratorSetting(1, FirstEntryInRowIterator.class);
+    FirstEntryInRowIterator.setNumScansBeforeSeek(iteratorSetting, 10);
+    assertTrue(iteratorSetting.getOptions().containsKey(FirstEntryInRowIterator.NUM_SCANS_STRING_NAME));
     sourceMap.put(new Key("r1", "cf", "cq"), emptyValue);
     sourceMap.put(new Key("r2", "cf", "cq"), emptyValue);
     sourceMap.put(new Key("r3", "cf", "cq"), emptyValue);
     TreeMap<Key,Value> resultMap = new TreeMap<>();
     long numSourceEntries = sourceMap.size();
-    long numNexts = process(sourceMap, resultMap, new Range(), 10);
+    long numNexts = process(sourceMap, resultMap, new Range(), iteratorSetting);
     assertEquals(numNexts, numSourceEntries);
     assertEquals(sourceMap.size(), resultMap.size());
 
@@ -66,17 +71,18 @@ public class FirstEntryInRowIteratorTest {
       sourceMap.put(new Key("r2", "cf", "cq" + i), emptyValue);
     }
     resultMap.clear();
-    numNexts = process(sourceMap, resultMap, new Range(new Key("r1"), (new Key("r2")).followingKey(PartialKey.ROW)), 10);
+
+    numNexts = process(sourceMap, resultMap, new Range(new Key("r1"), (new Key("r2")).followingKey(PartialKey.ROW)), iteratorSetting);
     assertEquals(numNexts, resultMap.size() + 10);
     assertEquals(resultMap.size(), 2);
 
     resultMap.clear();
-    numNexts = process(sourceMap, resultMap, new Range(new Key("r1"), new Key("r2", "cf2")), 10);
+    numNexts = process(sourceMap, resultMap, new Range(new Key("r1"), new Key("r2", "cf2")), iteratorSetting);
     assertEquals(numNexts, resultMap.size() + 10);
     assertEquals(resultMap.size(), 2);
 
     resultMap.clear();
-    numNexts = process(sourceMap, resultMap, new Range(new Key("r1"), new Key("r4")), 10);
+    numNexts = process(sourceMap, resultMap, new Range(new Key("r1"), new Key("r4")), iteratorSetting);
     assertEquals(numNexts, resultMap.size() + 10);
     assertEquals(resultMap.size(), 3);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/34532dcc/core/src/test/java/org/apache/accumulo/core/iterators/user/CombinerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/CombinerTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/CombinerTest.java
index ec740a6..c996f5d 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/CombinerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/CombinerTest.java
@@ -48,6 +48,7 @@ import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.iterators.TypedValueCombiner;
 import org.apache.accumulo.core.iterators.TypedValueCombiner.Encoder;
+import org.apache.accumulo.core.iterators.ValueFormatException;
 import org.apache.accumulo.core.iterators.system.MultiIterator;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
@@ -922,4 +923,49 @@ public class CombinerTest {
     runDeleteHandlingTest(input, expected, null, paritalMajcIe, ".*ERROR.*SummingCombiner.*ACCUMULO-2232.*");
     runDeleteHandlingTest(input, expected, null, fullMajcIe, ".*ERROR.*SummingCombiner.*ACCUMULO-2232.*");
   }
+
+  /**
+   * Tests the Lossy option will ignore errors in TypedValueCombiner. Uses SummingArrayCombiner to generate error.
+   */
+  @Test
+  public void testLossyOption() throws IOException, IllegalAccessException, InstantiationException {
+    Encoder<List<Long>> encoder = new SummingArrayCombiner.VarLongArrayEncoder();
+
+    TreeMap<Key,Value> tm1 = new TreeMap<>();
+
+    // keys that aggregate
+    tm1.put(newKey(1, 1, 1, 1, false), new Value("badValue"));
+    newKeyValue(tm1, 1, 1, 1, 2, false, nal(3l, 4l, 5l), encoder);
+    newKeyValue(tm1, 1, 1, 1, 3, false, nal(), encoder);
+
+    SummingArrayCombiner summingArrayCombiner = new SummingArrayCombiner();
+    IteratorSetting iteratorSetting = new IteratorSetting(1, SummingArrayCombiner.class);
+    SummingArrayCombiner.setEncodingType(iteratorSetting, SummingArrayCombiner.Type.VARLEN);
+    Combiner.setColumns(iteratorSetting, Collections.singletonList(new IteratorSetting.Column("cf001")));
+
+    // lossy = true so ignore bad value
+    TypedValueCombiner.setLossyness(iteratorSetting, true);
+    assertTrue(summingArrayCombiner.validateOptions(iteratorSetting.getOptions()));
+
+    summingArrayCombiner.init(new SortedMapIterator(tm1), iteratorSetting.getOptions(), SCAN_IE);
+    summingArrayCombiner.seek(new Range(), EMPTY_COL_FAMS, false);
+
+    assertTrue(summingArrayCombiner.hasTop());
+    assertEquals(newKey(1, 1, 1, 3), summingArrayCombiner.getTopKey());
+    assertBytesEqual(encoder.encode(nal(3l, 4l, 5l)), summingArrayCombiner.getTopValue().get());
+
+    summingArrayCombiner.next();
+
+    assertFalse(summingArrayCombiner.hasTop());
+
+    // lossy = false throw error for bad value
+    TypedValueCombiner.setLossyness(iteratorSetting, false);
+    assertTrue(summingArrayCombiner.validateOptions(iteratorSetting.getOptions()));
+
+    summingArrayCombiner.init(new SortedMapIterator(tm1), iteratorSetting.getOptions(), SCAN_IE);
+    try {
+      summingArrayCombiner.seek(new Range(), EMPTY_COL_FAMS, false);
+      Assert.fail("ValueFormatException should have been thrown");
+    } catch (ValueFormatException e) {}
+  }
 }