You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/10/10 00:47:19 UTC

[02/10] git commit: ACCUMULO-3217 Make KeyValue extend SimpleImmutableEntry instead of implement Map.Entry.

ACCUMULO-3217 Make KeyValue extend SimpleImmutableEntry instead of implement Map.Entry.

We get toString, hashCode and equals for free by extending SimpleImmutableEntry
over implementing Map.Entry. Without this change, users couldn't properly
use the Entry<Key,Value> objects that were returned by Scanners and BatchScanners.


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

Branch: refs/heads/1.6
Commit: 854dce9fffb9766a65776e5c61af7710f1df0b75
Parents: 8b17144
Author: Josh Elser <el...@apache.org>
Authored: Thu Oct 9 17:35:41 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu Oct 9 17:35:41 2014 -0400

----------------------------------------------------------------------
 .../java/org/apache/accumulo/core/data/Key.java | 14 +--
 .../org/apache/accumulo/core/data/KeyValue.java | 72 ++++++++-------
 .../accumulo/server/tabletserver/Tablet.java    | 14 ++-
 .../server/tabletserver/TabletServer.java       |  2 +-
 .../accumulo/test/KeyValueEqualityTest.java     | 94 ++++++++++++++++++++
 5 files changed, 146 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/854dce9f/core/src/main/java/org/apache/accumulo/core/data/Key.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/Key.java b/core/src/main/java/org/apache/accumulo/core/data/Key.java
index 2b44359..c71a624 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/Key.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/Key.java
@@ -768,13 +768,13 @@ public class Key implements WritableComparable<Key>, Cloneable {
     List<TKeyValue> tkvl = Arrays.asList(new TKeyValue[param.size()]);
     
     if (param.size() > 0)
-      tkvl.set(0, new TKeyValue(param.get(0).key.toThrift(), ByteBuffer.wrap(param.get(0).value)));
-    
+      tkvl.set(0, new TKeyValue(param.get(0).getKey().toThrift(), ByteBuffer.wrap(param.get(0).getValue().get())));
+
     for (int i = param.size() - 1; i > 0; i--) {
-      Key prevKey = param.get(i - 1).key;
+      Key prevKey = param.get(i - 1).getKey();
       KeyValue kv = param.get(i);
-      Key key = kv.key;
-      
+      Key key = kv.getKey();
+
       TKey newKey = null;
       
       if (isEqual(prevKey.row, key.row)) {
@@ -802,8 +802,8 @@ public class Key implements WritableComparable<Key>, Cloneable {
       
       if (newKey == null)
         newKey = key.toThrift();
-      
-      tkvl.set(i, new TKeyValue(newKey, ByteBuffer.wrap(kv.value)));
+
+      tkvl.set(i, new TKeyValue(newKey, ByteBuffer.wrap(kv.getValue().get())));
     }
     
     return tkvl;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/854dce9f/core/src/main/java/org/apache/accumulo/core/data/KeyValue.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/KeyValue.java b/core/src/main/java/org/apache/accumulo/core/data/KeyValue.java
index 90b2d7c..ab17a8c 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/KeyValue.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/KeyValue.java
@@ -16,45 +16,49 @@
  */
 package org.apache.accumulo.core.data;
 
-import static org.apache.accumulo.core.util.ByteBufferUtil.toBytes;
-
 import java.nio.ByteBuffer;
-import java.util.Map;
+import java.util.AbstractMap.SimpleImmutableEntry;
+
+/**
+ * A key/value pair. The key and value may not be set after construction.
+ */
+public class KeyValue extends SimpleImmutableEntry<Key,Value> {
 
-import org.apache.accumulo.core.Constants;
+  private static final long serialVersionUID = 1L;
 
-public class KeyValue implements Map.Entry<Key,Value> {
-  
-  public Key key;
-  public byte[] value;
-  
+  /**
+   * Creates a new key/value pair.
+   *
+   * @param key
+   *          key
+   * @param value
+   *          bytes of value
+   */
   public KeyValue(Key key, byte[] value) {
-    this.key = key;
-    this.value = value;
+    super(key, new Value(value));
   }
-  
+
+  /**
+   * Creates a new key/value pair.
+   *
+   * @param key
+   *          key
+   * @param value
+   *          buffer containing bytes of value
+   */
   public KeyValue(Key key, ByteBuffer value) {
-    this.key = key;
-    this.value = toBytes(value);
+    super(key, new Value(value));
   }
-  
-  @Override
-  public Key getKey() {
-    return key;
-  }
-  
-  @Override
-  public Value getValue() {
-    return new Value(value);
-  }
-  
-  @Override
-  public Value setValue(Value value) {
-    throw new UnsupportedOperationException();
-  }
-  
-  public String toString() {
-    return key + " " + new String(value, Constants.UTF8);
+
+  /**
+   * Creates a new key/value pair.
+   *
+   * @param key
+   *          key
+   * @param value
+   *          buffer containing bytes of value
+   */
+  public KeyValue(Key key, Value value) {
+    super(key, value);
   }
-  
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/854dce9f/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
index a7a9d4e..7b0bb90 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
@@ -1583,20 +1583,18 @@ public class Tablet {
   }
   
   public static class KVEntry extends KeyValue {
+    private static final long serialVersionUID = 1L;
+
     public KVEntry(Key k, Value v) {
       super(new Key(k), Arrays.copyOf(v.get(), v.get().length));
     }
-    
-    public String toString() {
-      return key.toString() + "=" + getValue();
-    }
-    
+
     int numBytes() {
-      return key.getSize() + getValue().get().length;
+      return getKey().getSize() + getValue().get().length;
     }
     
     int estimateMemoryUsed() {
-      return key.getSize() + getValue().get().length + (9 * 32); // overhead is 32 per object
+      return getKey().getSize() + getValue().get().length + (9 * 32); // overhead is 32 per object
     }
   }
   
@@ -1682,7 +1680,7 @@ public class Tablet {
       throw new IllegalStateException("tablet should not exceed memory usage or close, not both");
     
     if (entriesAdded > 0)
-      addUnfinishedRange(lookupResult, range, results.get(results.size() - 1).key, false);
+      addUnfinishedRange(lookupResult, range, results.get(results.size() - 1).getKey(), false);
     else
       lookupResult.unfinishedRanges.add(range);
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/854dce9f/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
index a730d9f..08de4a1 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
@@ -1085,7 +1085,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
           // convert everything to thrift before adding result
           List<TKeyValue> retResults = new ArrayList<TKeyValue>();
           for (KVEntry entry : results)
-            retResults.add(new TKeyValue(entry.key.toThrift(), ByteBuffer.wrap(entry.value)));
+            retResults.add(new TKeyValue(entry.getKey().toThrift(), ByteBuffer.wrap(entry.getValue().get())));
           Map<TKeyExtent,List<TRange>> retFailures = Translator.translate(failures, Translators.KET, new Translator.ListTranslator<Range,TRange>(Translators.RT));
           List<TKeyExtent> retFullScans = Translator.translate(fullScans, Translators.KET);
           TKeyExtent retPartScan = null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/854dce9f/test/src/test/java/org/apache/accumulo/test/KeyValueEqualityTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/KeyValueEqualityTest.java b/test/src/test/java/org/apache/accumulo/test/KeyValueEqualityTest.java
new file mode 100644
index 0000000..7dc6be9
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/KeyValueEqualityTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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;
+
+import java.util.Iterator;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.Constants;
+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.ZooKeeperInstance;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class KeyValueEqualityTest {
+  public static TemporaryFolder folder = new TemporaryFolder();
+  public static MiniAccumuloCluster cluster;
+  private static final PasswordToken password = new PasswordToken("secret");
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    folder.create();
+    MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("miniAccumulo"), new String(password.getPassword()));
+    cluster = new MiniAccumuloCluster(cfg);
+    cluster.start();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    cluster.stop();
+    folder.delete();
+  }
+
+  @Test
+  public void testEquality() throws Exception {
+    final ZooKeeperInstance instance = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    final Connector conn = instance.getConnector("root", password);
+    final BatchWriterConfig config = new BatchWriterConfig();
+
+    final String table1 = "table1", table2 = "table2";
+    final TableOperations tops = conn.tableOperations();
+    tops.create(table1);
+    tops.create(table2);
+
+    final BatchWriter bw1 = conn.createBatchWriter(table1, config), bw2 = conn.createBatchWriter(table2, config);
+
+    for (int row = 0; row < 100; row++) {
+      Mutation m = new Mutation(Integer.toString(row));
+      for (int col = 0; col < 10; col++) {
+        m.put(Integer.toString(col), "", System.currentTimeMillis(), Integer.toString(col * 2));
+      }
+      bw1.addMutation(m);
+      bw2.addMutation(m);
+    }
+
+    bw1.close();
+    bw2.close();
+
+    Iterator<Entry<Key,Value>> t1 = conn.createScanner(table1, Constants.NO_AUTHS).iterator(), t2 = conn.createScanner(table2, Constants.NO_AUTHS).iterator();
+    while (t1.hasNext() && t2.hasNext()) {
+      // KeyValue, the implementation of Entry<Key,Value>, should support equality and hashCode properly
+      Entry<Key,Value> e1 = t1.next(), e2 = t2.next();
+      Assert.assertEquals(e1, e2);
+      Assert.assertEquals(e1.hashCode(), e2.hashCode());
+    }
+    Assert.assertFalse("table1 had more data to read", t1.hasNext());
+    Assert.assertFalse("table2 had more data to read", t2.hasNext());
+  }
+}