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 2015/05/10 23:06:19 UTC

[1/9] accumulo git commit: ACCUMULO-3790 Ensure replication table is online before test proceeds

Repository: accumulo
Updated Branches:
  refs/heads/1.7 2cde56226 -> f91385488
  refs/heads/master d0e6c0be3 -> 688fa0419


ACCUMULO-3790 Ensure replication table is online before test proceeds


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

Branch: refs/heads/1.7
Commit: 671d4da164994a9c0025b246245b8385468778ea
Parents: 2cde562
Author: Josh Elser <el...@apache.org>
Authored: Sat May 9 20:52:29 2015 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sat May 9 20:52:29 2015 -0400

----------------------------------------------------------------------
 .../replication/MultiInstanceReplicationIT.java | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/671d4da1/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
index 3912e98..7dd31c0 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
@@ -225,6 +225,11 @@ public class MultiInstanceReplicationIT extends ConfigurableMacIT {
       Iterators.size(ReplicationTable.getScanner(connMaster).iterator());
       log.info("TabletServer is online");
 
+      while (!ReplicationTable.isOnline(connMaster)) {
+        log.info("Replication table still offline, waiting");
+        Thread.sleep(5000);
+      }
+
       log.info("");
       log.info("Fetching metadata records:");
       for (Entry<Key,Value> kv : connMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
@@ -409,6 +414,11 @@ public class MultiInstanceReplicationIT extends ConfigurableMacIT {
       // Read the data -- the tserver is back up and running
       Iterators.size(connMaster.createScanner(masterTable1, Authorizations.EMPTY).iterator());
 
+      while (!ReplicationTable.isOnline(connMaster)) {
+        log.info("Replication table still offline, waiting");
+        Thread.sleep(5000);
+      }
+
       // Wait for both tables to be replicated
       log.info("Waiting for {} for {}", filesFor1, masterTable1);
       connMaster.replicationOperations().drain(masterTable1, filesFor1);
@@ -516,6 +526,11 @@ public class MultiInstanceReplicationIT extends ConfigurableMacIT {
 
     cluster.exec(TabletServer.class);
 
+    while (!ReplicationTable.isOnline(connMaster)) {
+      log.info("Replication table still offline, waiting");
+      Thread.sleep(5000);
+    }
+
     Iterators.size(connMaster.createScanner(masterTable, Authorizations.EMPTY).iterator());
 
     for (Entry<Key,Value> kv : ReplicationTable.getScanner(connMaster)) {
@@ -638,6 +653,11 @@ public class MultiInstanceReplicationIT extends ConfigurableMacIT {
 
       cluster.exec(TabletServer.class);
 
+      while (!ReplicationTable.isOnline(connMaster)) {
+        log.info("Replication table still offline, waiting");
+        Thread.sleep(5000);
+      }
+
       // Wait until we fully replicated something
       boolean fullyReplicated = false;
       for (int i = 0; i < 10 && !fullyReplicated; i++) {


[4/9] accumulo git commit: ACCUMULO-3788 Recreate batchscanner in ContinuousBatchWalker after its closed.

Posted by el...@apache.org.
ACCUMULO-3788 Recreate batchscanner in ContinuousBatchWalker after its closed.


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

Branch: refs/heads/master
Commit: 757c1cb5569d02e6b155f1d5dd2f66f877ec0cd9
Parents: 671d4da
Author: Josh Elser <el...@apache.org>
Authored: Sat May 9 20:55:23 2015 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sat May 9 20:55:23 2015 -0400

----------------------------------------------------------------------
 .../apache/accumulo/test/continuous/ContinuousBatchWalker.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/757c1cb5/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java b/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
index 5c54b08..2c32176 100644
--- a/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
+++ b/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
@@ -63,10 +63,10 @@ public class ContinuousBatchWalker {
     Scanner scanner = ContinuousUtil.createScanner(conn, clientOpts.getTableName(), auths);
     scanner.setBatchSize(scanOpts.scanBatchSize);
 
-    BatchScanner bs = conn.createBatchScanner(clientOpts.getTableName(), auths, bsOpts.scanThreads);
-    bs.setTimeout(bsOpts.scanTimeout, TimeUnit.MILLISECONDS);
-
     while (true) {
+      BatchScanner bs = conn.createBatchScanner(clientOpts.getTableName(), auths, bsOpts.scanThreads);
+      bs.setTimeout(bsOpts.scanTimeout, TimeUnit.MILLISECONDS);
+
       Set<Text> batch = getBatch(scanner, opts.min, opts.max, scanOpts.scanBatchSize, r);
       List<Range> ranges = new ArrayList<Range>(batch.size());
 


[9/9] accumulo git commit: Merge branch '1.7'

Posted by el...@apache.org.
Merge branch '1.7'


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

Branch: refs/heads/master
Commit: 688fa0419c83f821cfe146ecafac72274032c4df
Parents: d0e6c0b f913854
Author: Josh Elser <el...@apache.org>
Authored: Sun May 10 17:06:06 2015 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun May 10 17:06:06 2015 -0400

----------------------------------------------------------------------
 .../client/lexicoder/BigIntegerLexicoder.java   | 10 +++
 .../core/client/lexicoder/DateLexicoder.java    | 10 +++
 .../core/client/lexicoder/DoubleLexicoder.java  | 10 +++
 .../core/client/lexicoder/IntegerLexicoder.java | 10 +++
 .../core/client/lexicoder/ListLexicoder.java    | 10 +++
 .../core/client/lexicoder/PairLexicoder.java    | 10 +++
 .../core/client/lexicoder/StringLexicoder.java  | 10 +++
 .../core/client/lexicoder/TextLexicoder.java    | 10 +++
 .../client/lexicoder/UIntegerLexicoder.java     | 10 +++
 .../core/client/lexicoder/ULongLexicoder.java   | 10 +++
 .../core/client/lexicoder/UUIDLexicoder.java    | 10 +++
 .../accumulo/core/iterators/LongCombiner.java   | 30 +++++++++
 .../core/iterators/user/BigDecimalCombiner.java | 10 +++
 .../iterators/user/SummingArrayCombiner.java    | 20 ++++++
 .../tabletserver/LargestFirstMemoryManager.java | 13 ++++
 .../LargestFirstMemoryManagerTest.java          | 67 ++++++++++++++++++++
 .../test/continuous/ContinuousBatchWalker.java  |  6 +-
 .../replication/MultiInstanceReplicationIT.java | 20 ++++++
 18 files changed, 273 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/688fa041/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
----------------------------------------------------------------------


[3/9] accumulo git commit: ACCUMULO-3788 Recreate batchscanner in ContinuousBatchWalker after its closed.

Posted by el...@apache.org.
ACCUMULO-3788 Recreate batchscanner in ContinuousBatchWalker after its closed.


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

Branch: refs/heads/1.7
Commit: 757c1cb5569d02e6b155f1d5dd2f66f877ec0cd9
Parents: 671d4da
Author: Josh Elser <el...@apache.org>
Authored: Sat May 9 20:55:23 2015 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sat May 9 20:55:23 2015 -0400

----------------------------------------------------------------------
 .../apache/accumulo/test/continuous/ContinuousBatchWalker.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/757c1cb5/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java b/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
index 5c54b08..2c32176 100644
--- a/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
+++ b/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
@@ -63,10 +63,10 @@ public class ContinuousBatchWalker {
     Scanner scanner = ContinuousUtil.createScanner(conn, clientOpts.getTableName(), auths);
     scanner.setBatchSize(scanOpts.scanBatchSize);
 
-    BatchScanner bs = conn.createBatchScanner(clientOpts.getTableName(), auths, bsOpts.scanThreads);
-    bs.setTimeout(bsOpts.scanTimeout, TimeUnit.MILLISECONDS);
-
     while (true) {
+      BatchScanner bs = conn.createBatchScanner(clientOpts.getTableName(), auths, bsOpts.scanThreads);
+      bs.setTimeout(bsOpts.scanTimeout, TimeUnit.MILLISECONDS);
+
       Set<Text> batch = getBatch(scanner, opts.min, opts.max, scanOpts.scanBatchSize, r);
       List<Range> ranges = new ArrayList<Range>(batch.size());
 


[6/9] accumulo git commit: ACCUMULO-3777 Add existence check to MemoryManager

Posted by el...@apache.org.
ACCUMULO-3777 Add existence check to MemoryManager

Somehow we got into a situation where we were checking tablets
for a deleted table. Add a quick check for existence before
running the extent through the memory manager.


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

Branch: refs/heads/master
Commit: 719aac433e0428f969130820427d62404d5642e2
Parents: 757c1cb
Author: Josh Elser <el...@apache.org>
Authored: Sun May 10 01:09:19 2015 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun May 10 01:09:19 2015 -0400

----------------------------------------------------------------------
 .../tabletserver/LargestFirstMemoryManager.java | 13 ++++
 .../LargestFirstMemoryManagerTest.java          | 67 ++++++++++++++++++++
 2 files changed, 80 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/719aac43/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java b/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java
index 323e59d..a39c8b6 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java
@@ -22,6 +22,8 @@ import java.util.List;
 import java.util.Map.Entry;
 import java.util.TreeMap;
 
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.server.conf.ServerConfiguration;
@@ -143,11 +145,16 @@ public class LargestFirstMemoryManager implements MemoryManager {
     return mincIdleThresholds.get(tableId);
   }
 
+  boolean tableExists(Instance instance, String tableId) {
+    return Tables.exists(instance, tableId);
+  }
+
   @Override
   public MemoryManagementActions getMemoryManagementActions(List<TabletState> tablets) {
     if (maxMemory < 0)
       throw new IllegalStateException("need to initialize " + LargestFirstMemoryManager.class.getName());
 
+    final Instance instance = config.getInstance();
     final int maxMinCs = maxConcurrentMincs * numWaitingMultiplier;
 
     mincIdleThresholds.clear();
@@ -164,6 +171,12 @@ public class LargestFirstMemoryManager implements MemoryManager {
 
     // find the largest and most idle tablets
     for (TabletState ts : tablets) {
+      // Make sure that the table still exists
+      if (!tableExists(instance, ts.getExtent().getTableId().toString())) {
+        log.info("Ignoring extent for deleted table: " + ts.getExtent());
+        continue;
+      }
+
       final long memTabletSize = ts.getMemTableSize();
       final long minorCompactingSize = ts.getMinorCompactingMemTableSize();
       final long idleTime = now - Math.max(ts.getLastCommitTime(), ZERO_TIME);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/719aac43/server/tserver/src/test/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManagerTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManagerTest.java b/server/tserver/src/test/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManagerTest.java
index 7c15cfc..b08b980 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManagerTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManagerTest.java
@@ -33,6 +33,8 @@ import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
+import com.google.common.base.Function;
+
 public class LargestFirstMemoryManagerTest {
 
   private static final long ZERO = System.currentTimeMillis();
@@ -160,6 +162,52 @@ public class LargestFirstMemoryManagerTest {
     assertEquals(k("b"), result.tabletsToMinorCompact.get(0));
   }
 
+  @Test
+  public void testDeletedTable() throws Exception {
+    final String deletedTableId = "1";
+    Function<String,Boolean> existenceCheck = new Function<String,Boolean>() {
+      public Boolean apply(String tableId) {
+        return !deletedTableId.equals(tableId);
+      }
+    };
+    LargestFirstMemoryManagerWithExistenceCheck mgr = new LargestFirstMemoryManagerWithExistenceCheck(existenceCheck);
+    ServerConfiguration config = new ServerConfiguration() {
+      ServerConfigurationFactory delegate = new ServerConfigurationFactory(new MockInstance());
+
+      @Override
+      public AccumuloConfiguration getConfiguration() {
+        return DefaultConfiguration.getInstance();
+      }
+
+      @Override
+      public TableConfiguration getTableConfiguration(String tableId) {
+        return delegate.getTableConfiguration(tableId);
+      }
+
+      @Override
+      public TableConfiguration getTableConfiguration(KeyExtent extent) {
+        return delegate.getTableConfiguration(extent);
+      }
+
+      @Override
+      public NamespaceConfiguration getNamespaceConfiguration(String namespaceId) {
+        return delegate.getNamespaceConfiguration(namespaceId);
+      }
+
+      @Override
+      public Instance getInstance() {
+        return delegate.getInstance();
+      }
+    };
+    mgr.init(config);
+    MemoryManagementActions result;
+    // one tablet is really big and the other is for a nonexistent table
+    KeyExtent extent = new KeyExtent(new Text("2"), new Text("j"), null);
+    result = mgr.getMemoryManagementActions(tablets(t(extent, ZERO, ONE_GIG, 0), t(k("j"), ZERO, ONE_GIG, 0)));
+    assertEquals(1, result.tabletsToMinorCompact.size());
+    assertEquals(extent, result.tabletsToMinorCompact.get(0));
+  }
+
   private static class LargestFirstMemoryManagerUnderTest extends LargestFirstMemoryManager {
 
     public long currentTime = ZERO;
@@ -174,6 +222,25 @@ public class LargestFirstMemoryManagerTest {
       return 15 * 60 * 1000;
     }
 
+    @Override
+    boolean tableExists(Instance instance, String tableId) {
+      return true;
+    }
+  }
+
+  private static class LargestFirstMemoryManagerWithExistenceCheck extends LargestFirstMemoryManagerUnderTest {
+
+    Function<String,Boolean> existenceCheck;
+
+    public LargestFirstMemoryManagerWithExistenceCheck(Function<String,Boolean> existenceCheck) {
+      super();
+      this.existenceCheck = existenceCheck;
+    }
+
+    @Override
+    boolean tableExists(Instance instance, String tableId) {
+      return existenceCheck.apply(tableId);
+    }
   }
 
   private static KeyExtent k(String endRow) {


[8/9] accumulo git commit: ACCUMULO-3789: Code compiled against 1.6 lexicoders can't run in 1.7

Posted by el...@apache.org.
ACCUMULO-3789: Code compiled against 1.6 lexicoders can't run in 1.7

Added concrete implementation of decode to all client Lexicoders. Implementation just delegates to decode method in AbstractEncoder.

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/1.7
Commit: f91385488b93e87513cc63ed78e6b74a512f0a3e
Parents: 719aac4
Author: Russ Weeks <rw...@newbrightidea.com>
Authored: Sat May 9 22:55:34 2015 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Sun May 10 13:34:28 2015 -0400

----------------------------------------------------------------------
 .../client/lexicoder/BigIntegerLexicoder.java   | 10 +++++++
 .../core/client/lexicoder/DateLexicoder.java    | 10 +++++++
 .../core/client/lexicoder/DoubleLexicoder.java  | 10 +++++++
 .../core/client/lexicoder/IntegerLexicoder.java | 10 +++++++
 .../core/client/lexicoder/ListLexicoder.java    | 10 +++++++
 .../core/client/lexicoder/PairLexicoder.java    | 10 +++++++
 .../core/client/lexicoder/StringLexicoder.java  | 10 +++++++
 .../core/client/lexicoder/TextLexicoder.java    | 10 +++++++
 .../client/lexicoder/UIntegerLexicoder.java     | 10 +++++++
 .../core/client/lexicoder/ULongLexicoder.java   | 10 +++++++
 .../core/client/lexicoder/UUIDLexicoder.java    | 10 +++++++
 .../accumulo/core/iterators/LongCombiner.java   | 30 ++++++++++++++++++++
 .../core/iterators/user/BigDecimalCombiner.java | 10 +++++++
 .../iterators/user/SummingArrayCombiner.java    | 20 +++++++++++++
 14 files changed, 170 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
index 577e33e..59db11c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
@@ -63,6 +63,16 @@ public class BigIntegerLexicoder extends AbstractLexicoder<BigInteger> implement
 
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public BigInteger decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected BigInteger decodeUnchecked(byte[] b, int offset, int origLen) throws ValueFormatException {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DateLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DateLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DateLexicoder.java
index 2d17d37..2d928b4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DateLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DateLexicoder.java
@@ -34,6 +34,16 @@ public class DateLexicoder extends AbstractLexicoder<Date> implements Lexicoder<
     return longEncoder.encode(data.getTime());
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public Date decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected Date decodeUnchecked(byte[] data, int offset, int len) {
     return new Date(longEncoder.decodeUnchecked(data, offset, len));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DoubleLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DoubleLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DoubleLexicoder.java
index 14413c0..c473f5f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DoubleLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DoubleLexicoder.java
@@ -38,6 +38,16 @@ public class DoubleLexicoder extends AbstractLexicoder<Double> implements Lexico
     return longEncoder.encode(l);
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public Double decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected Double decodeUnchecked(byte[] data, int offset, int len) {
     long l = longEncoder.decodeUnchecked(data, offset, len);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/IntegerLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/IntegerLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/IntegerLexicoder.java
index f96b41d..f3c481d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/IntegerLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/IntegerLexicoder.java
@@ -33,6 +33,16 @@ public class IntegerLexicoder extends AbstractLexicoder<Integer> implements Lexi
     return uil.encode(i ^ 0x80000000);
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public Integer decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected Integer decodeUnchecked(byte[] data, int offset, int len) {
     return uil.decodeUnchecked(data, offset, len) ^ 0x80000000;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java
index 3c20401..99ffe3b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java
@@ -56,6 +56,16 @@ public class ListLexicoder<LT> extends AbstractLexicoder<List<LT>> implements Le
     return concat(encElements);
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public List<LT> decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected List<LT> decodeUnchecked(byte[] b, int offset, int len) {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
index f28912c..679e823 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
@@ -65,6 +65,16 @@ public class PairLexicoder<A extends Comparable<A>,B extends Comparable<B>> exte
     return concat(escape(firstLexicoder.encode(data.getFirst())), escape(secondLexicoder.encode(data.getSecond())));
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public ComparablePair<A,B> decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected ComparablePair<A,B> decodeUnchecked(byte[] data, int offset, int len) {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/StringLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/StringLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/StringLexicoder.java
index f7b1877..94558ce 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/StringLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/StringLexicoder.java
@@ -34,6 +34,16 @@ public class StringLexicoder extends AbstractLexicoder<String> implements Lexico
     return data.getBytes(UTF_8);
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public String decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected String decodeUnchecked(byte[] data, int offset, int len) {
     return new String(data, offset, len, UTF_8);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/TextLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/TextLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/TextLexicoder.java
index bb29857..55702b6 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/TextLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/TextLexicoder.java
@@ -34,6 +34,16 @@ public class TextLexicoder extends AbstractLexicoder<Text> implements Lexicoder<
     return TextUtil.getBytes(data);
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public Text decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected Text decodeUnchecked(byte[] data, int offset, int len) {
     Text text = new Text();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UIntegerLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UIntegerLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UIntegerLexicoder.java
index e99c3b1..8194244 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UIntegerLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UIntegerLexicoder.java
@@ -53,6 +53,16 @@ public class UIntegerLexicoder extends AbstractLexicoder<Integer> implements Lex
 
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public Integer decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected Integer decodeUnchecked(byte[] data, int offset, int len) {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ULongLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ULongLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ULongLexicoder.java
index a69ed34..72d4a52 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ULongLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ULongLexicoder.java
@@ -73,4 +73,14 @@ public class ULongLexicoder extends AbstractLexicoder<Long> implements Lexicoder
 
     return l;
   }
+
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public Long decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
index e8e2c06..423c371 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
@@ -56,6 +56,16 @@ public class UUIDLexicoder extends AbstractLexicoder<UUID> implements Lexicoder<
     }
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public UUID decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected UUID decodeUnchecked(byte[] b, int offset, int len) throws ValueFormatException {
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java b/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java
index cbc76ab..7bd8079 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java
@@ -133,6 +133,16 @@ public abstract class LongCombiner extends TypedValueCombiner<Long> {
       return baos.toByteArray();
     }
 
+    /**
+     * @deprecated Since 1.7.0
+     */
+    @Override
+    @Deprecated
+    public Long decode(byte[] b) {
+      // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+      return super.decode(b);
+    }
+
     @Override
     protected Long decodeUnchecked(byte[] b, int offset, int len) {
       DataInputStream dis = new DataInputStream(new ByteArrayInputStream(b, offset, len));
@@ -162,6 +172,16 @@ public abstract class LongCombiner extends TypedValueCombiner<Long> {
       return b;
     }
 
+    /**
+     * @deprecated Since 1.7.0
+     */
+    @Override
+    @Deprecated
+    public Long decode(byte[] b) {
+      // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+      return super.decode(b);
+    }
+
     @Override
     protected Long decodeUnchecked(byte[] b, int offset, int len) {
       return decodeStatic(b, offset, len);
@@ -192,6 +212,16 @@ public abstract class LongCombiner extends TypedValueCombiner<Long> {
       return Long.toString(v).getBytes(UTF_8);
     }
 
+    /**
+     * @deprecated Since 1.7.0
+     */
+    @Override
+    @Deprecated
+    public Long decode(byte[] b) {
+      // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+      return super.decode(b);
+    }
+
     @Override
     protected Long decodeUnchecked(byte[] b, int offset, int len) {
       try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java
index e90e87e..f2c03f3 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java
@@ -108,6 +108,16 @@ public abstract class BigDecimalCombiner extends TypedValueCombiner<BigDecimal>
       return v.toString().getBytes(UTF_8);
     }
 
+    /**
+     * @deprecated Since 1.7.0
+     */
+    @Override
+    @Deprecated
+    public BigDecimal decode(byte[] b) {
+      // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+      return super.decode(b);
+    }
+
     @Override
     protected BigDecimal decodeUnchecked(byte[] b, int offset, int len) throws ValueFormatException {
       try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java
index 04c0af1..32b7a7a 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java
@@ -162,6 +162,16 @@ public class SummingArrayCombiner extends TypedValueCombiner<List<Long>> {
       return baos.toByteArray();
     }
 
+    /**
+     * @deprecated Since 1.7.0
+     */
+    @Override
+    @Deprecated
+    public List<V> decode(byte[] b) {
+      // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+      return super.decode(b);
+    }
+
     @Override
     protected List<V> decodeUnchecked(byte[] b, int offset, int origLen) {
       DataInputStream dis = new DataInputStream(new ByteArrayInputStream(b, offset, origLen));
@@ -215,6 +225,16 @@ public class SummingArrayCombiner extends TypedValueCombiner<List<Long>> {
       return sb.toString().getBytes(UTF_8);
     }
 
+    /**
+     * @deprecated Since 1.7.0
+     */
+    @Override
+    @Deprecated
+    public List<Long> decode(byte[] b) {
+      // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+      return super.decode(b);
+    }
+
     @Override
     protected List<Long> decodeUnchecked(byte[] b, int offset, int len) {
       String[] longstrs = new String(b, offset, len, UTF_8).split(",");


[7/9] accumulo git commit: ACCUMULO-3789: Code compiled against 1.6 lexicoders can't run in 1.7

Posted by el...@apache.org.
ACCUMULO-3789: Code compiled against 1.6 lexicoders can't run in 1.7

Added concrete implementation of decode to all client Lexicoders. Implementation just delegates to decode method in AbstractEncoder.

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/master
Commit: f91385488b93e87513cc63ed78e6b74a512f0a3e
Parents: 719aac4
Author: Russ Weeks <rw...@newbrightidea.com>
Authored: Sat May 9 22:55:34 2015 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Sun May 10 13:34:28 2015 -0400

----------------------------------------------------------------------
 .../client/lexicoder/BigIntegerLexicoder.java   | 10 +++++++
 .../core/client/lexicoder/DateLexicoder.java    | 10 +++++++
 .../core/client/lexicoder/DoubleLexicoder.java  | 10 +++++++
 .../core/client/lexicoder/IntegerLexicoder.java | 10 +++++++
 .../core/client/lexicoder/ListLexicoder.java    | 10 +++++++
 .../core/client/lexicoder/PairLexicoder.java    | 10 +++++++
 .../core/client/lexicoder/StringLexicoder.java  | 10 +++++++
 .../core/client/lexicoder/TextLexicoder.java    | 10 +++++++
 .../client/lexicoder/UIntegerLexicoder.java     | 10 +++++++
 .../core/client/lexicoder/ULongLexicoder.java   | 10 +++++++
 .../core/client/lexicoder/UUIDLexicoder.java    | 10 +++++++
 .../accumulo/core/iterators/LongCombiner.java   | 30 ++++++++++++++++++++
 .../core/iterators/user/BigDecimalCombiner.java | 10 +++++++
 .../iterators/user/SummingArrayCombiner.java    | 20 +++++++++++++
 14 files changed, 170 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
index 577e33e..59db11c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
@@ -63,6 +63,16 @@ public class BigIntegerLexicoder extends AbstractLexicoder<BigInteger> implement
 
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public BigInteger decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected BigInteger decodeUnchecked(byte[] b, int offset, int origLen) throws ValueFormatException {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DateLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DateLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DateLexicoder.java
index 2d17d37..2d928b4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DateLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DateLexicoder.java
@@ -34,6 +34,16 @@ public class DateLexicoder extends AbstractLexicoder<Date> implements Lexicoder<
     return longEncoder.encode(data.getTime());
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public Date decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected Date decodeUnchecked(byte[] data, int offset, int len) {
     return new Date(longEncoder.decodeUnchecked(data, offset, len));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DoubleLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DoubleLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DoubleLexicoder.java
index 14413c0..c473f5f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DoubleLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/DoubleLexicoder.java
@@ -38,6 +38,16 @@ public class DoubleLexicoder extends AbstractLexicoder<Double> implements Lexico
     return longEncoder.encode(l);
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public Double decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected Double decodeUnchecked(byte[] data, int offset, int len) {
     long l = longEncoder.decodeUnchecked(data, offset, len);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/IntegerLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/IntegerLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/IntegerLexicoder.java
index f96b41d..f3c481d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/IntegerLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/IntegerLexicoder.java
@@ -33,6 +33,16 @@ public class IntegerLexicoder extends AbstractLexicoder<Integer> implements Lexi
     return uil.encode(i ^ 0x80000000);
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public Integer decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected Integer decodeUnchecked(byte[] data, int offset, int len) {
     return uil.decodeUnchecked(data, offset, len) ^ 0x80000000;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java
index 3c20401..99ffe3b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java
@@ -56,6 +56,16 @@ public class ListLexicoder<LT> extends AbstractLexicoder<List<LT>> implements Le
     return concat(encElements);
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public List<LT> decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected List<LT> decodeUnchecked(byte[] b, int offset, int len) {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
index f28912c..679e823 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
@@ -65,6 +65,16 @@ public class PairLexicoder<A extends Comparable<A>,B extends Comparable<B>> exte
     return concat(escape(firstLexicoder.encode(data.getFirst())), escape(secondLexicoder.encode(data.getSecond())));
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public ComparablePair<A,B> decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected ComparablePair<A,B> decodeUnchecked(byte[] data, int offset, int len) {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/StringLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/StringLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/StringLexicoder.java
index f7b1877..94558ce 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/StringLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/StringLexicoder.java
@@ -34,6 +34,16 @@ public class StringLexicoder extends AbstractLexicoder<String> implements Lexico
     return data.getBytes(UTF_8);
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public String decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected String decodeUnchecked(byte[] data, int offset, int len) {
     return new String(data, offset, len, UTF_8);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/TextLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/TextLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/TextLexicoder.java
index bb29857..55702b6 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/TextLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/TextLexicoder.java
@@ -34,6 +34,16 @@ public class TextLexicoder extends AbstractLexicoder<Text> implements Lexicoder<
     return TextUtil.getBytes(data);
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public Text decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected Text decodeUnchecked(byte[] data, int offset, int len) {
     Text text = new Text();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UIntegerLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UIntegerLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UIntegerLexicoder.java
index e99c3b1..8194244 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UIntegerLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UIntegerLexicoder.java
@@ -53,6 +53,16 @@ public class UIntegerLexicoder extends AbstractLexicoder<Integer> implements Lex
 
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public Integer decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected Integer decodeUnchecked(byte[] data, int offset, int len) {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ULongLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ULongLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ULongLexicoder.java
index a69ed34..72d4a52 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ULongLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ULongLexicoder.java
@@ -73,4 +73,14 @@ public class ULongLexicoder extends AbstractLexicoder<Long> implements Lexicoder
 
     return l;
   }
+
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public Long decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
index e8e2c06..423c371 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
@@ -56,6 +56,16 @@ public class UUIDLexicoder extends AbstractLexicoder<UUID> implements Lexicoder<
     }
   }
 
+  /**
+   * @deprecated Since 1.7.0
+   */
+  @Override
+  @Deprecated
+  public UUID decode(byte[] b) {
+    // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+    return super.decode(b);
+  }
+
   @Override
   protected UUID decodeUnchecked(byte[] b, int offset, int len) throws ValueFormatException {
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java b/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java
index cbc76ab..7bd8079 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java
@@ -133,6 +133,16 @@ public abstract class LongCombiner extends TypedValueCombiner<Long> {
       return baos.toByteArray();
     }
 
+    /**
+     * @deprecated Since 1.7.0
+     */
+    @Override
+    @Deprecated
+    public Long decode(byte[] b) {
+      // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+      return super.decode(b);
+    }
+
     @Override
     protected Long decodeUnchecked(byte[] b, int offset, int len) {
       DataInputStream dis = new DataInputStream(new ByteArrayInputStream(b, offset, len));
@@ -162,6 +172,16 @@ public abstract class LongCombiner extends TypedValueCombiner<Long> {
       return b;
     }
 
+    /**
+     * @deprecated Since 1.7.0
+     */
+    @Override
+    @Deprecated
+    public Long decode(byte[] b) {
+      // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+      return super.decode(b);
+    }
+
     @Override
     protected Long decodeUnchecked(byte[] b, int offset, int len) {
       return decodeStatic(b, offset, len);
@@ -192,6 +212,16 @@ public abstract class LongCombiner extends TypedValueCombiner<Long> {
       return Long.toString(v).getBytes(UTF_8);
     }
 
+    /**
+     * @deprecated Since 1.7.0
+     */
+    @Override
+    @Deprecated
+    public Long decode(byte[] b) {
+      // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+      return super.decode(b);
+    }
+
     @Override
     protected Long decodeUnchecked(byte[] b, int offset, int len) {
       try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java
index e90e87e..f2c03f3 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java
@@ -108,6 +108,16 @@ public abstract class BigDecimalCombiner extends TypedValueCombiner<BigDecimal>
       return v.toString().getBytes(UTF_8);
     }
 
+    /**
+     * @deprecated Since 1.7.0
+     */
+    @Override
+    @Deprecated
+    public BigDecimal decode(byte[] b) {
+      // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+      return super.decode(b);
+    }
+
     @Override
     protected BigDecimal decodeUnchecked(byte[] b, int offset, int len) throws ValueFormatException {
       try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f9138548/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java
index 04c0af1..32b7a7a 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java
@@ -162,6 +162,16 @@ public class SummingArrayCombiner extends TypedValueCombiner<List<Long>> {
       return baos.toByteArray();
     }
 
+    /**
+     * @deprecated Since 1.7.0
+     */
+    @Override
+    @Deprecated
+    public List<V> decode(byte[] b) {
+      // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+      return super.decode(b);
+    }
+
     @Override
     protected List<V> decodeUnchecked(byte[] b, int offset, int origLen) {
       DataInputStream dis = new DataInputStream(new ByteArrayInputStream(b, offset, origLen));
@@ -215,6 +225,16 @@ public class SummingArrayCombiner extends TypedValueCombiner<List<Long>> {
       return sb.toString().getBytes(UTF_8);
     }
 
+    /**
+     * @deprecated Since 1.7.0
+     */
+    @Override
+    @Deprecated
+    public List<Long> decode(byte[] b) {
+      // This concrete implementation is provided for backwards compatibility with 1.6; it can be removed in 2.0. See ACCUMULO-3789.
+      return super.decode(b);
+    }
+
     @Override
     protected List<Long> decodeUnchecked(byte[] b, int offset, int len) {
       String[] longstrs = new String(b, offset, len, UTF_8).split(",");


[5/9] accumulo git commit: ACCUMULO-3777 Add existence check to MemoryManager

Posted by el...@apache.org.
ACCUMULO-3777 Add existence check to MemoryManager

Somehow we got into a situation where we were checking tablets
for a deleted table. Add a quick check for existence before
running the extent through the memory manager.


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

Branch: refs/heads/1.7
Commit: 719aac433e0428f969130820427d62404d5642e2
Parents: 757c1cb
Author: Josh Elser <el...@apache.org>
Authored: Sun May 10 01:09:19 2015 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun May 10 01:09:19 2015 -0400

----------------------------------------------------------------------
 .../tabletserver/LargestFirstMemoryManager.java | 13 ++++
 .../LargestFirstMemoryManagerTest.java          | 67 ++++++++++++++++++++
 2 files changed, 80 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/719aac43/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java b/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java
index 323e59d..a39c8b6 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java
@@ -22,6 +22,8 @@ import java.util.List;
 import java.util.Map.Entry;
 import java.util.TreeMap;
 
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.server.conf.ServerConfiguration;
@@ -143,11 +145,16 @@ public class LargestFirstMemoryManager implements MemoryManager {
     return mincIdleThresholds.get(tableId);
   }
 
+  boolean tableExists(Instance instance, String tableId) {
+    return Tables.exists(instance, tableId);
+  }
+
   @Override
   public MemoryManagementActions getMemoryManagementActions(List<TabletState> tablets) {
     if (maxMemory < 0)
       throw new IllegalStateException("need to initialize " + LargestFirstMemoryManager.class.getName());
 
+    final Instance instance = config.getInstance();
     final int maxMinCs = maxConcurrentMincs * numWaitingMultiplier;
 
     mincIdleThresholds.clear();
@@ -164,6 +171,12 @@ public class LargestFirstMemoryManager implements MemoryManager {
 
     // find the largest and most idle tablets
     for (TabletState ts : tablets) {
+      // Make sure that the table still exists
+      if (!tableExists(instance, ts.getExtent().getTableId().toString())) {
+        log.info("Ignoring extent for deleted table: " + ts.getExtent());
+        continue;
+      }
+
       final long memTabletSize = ts.getMemTableSize();
       final long minorCompactingSize = ts.getMinorCompactingMemTableSize();
       final long idleTime = now - Math.max(ts.getLastCommitTime(), ZERO_TIME);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/719aac43/server/tserver/src/test/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManagerTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManagerTest.java b/server/tserver/src/test/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManagerTest.java
index 7c15cfc..b08b980 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManagerTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManagerTest.java
@@ -33,6 +33,8 @@ import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
+import com.google.common.base.Function;
+
 public class LargestFirstMemoryManagerTest {
 
   private static final long ZERO = System.currentTimeMillis();
@@ -160,6 +162,52 @@ public class LargestFirstMemoryManagerTest {
     assertEquals(k("b"), result.tabletsToMinorCompact.get(0));
   }
 
+  @Test
+  public void testDeletedTable() throws Exception {
+    final String deletedTableId = "1";
+    Function<String,Boolean> existenceCheck = new Function<String,Boolean>() {
+      public Boolean apply(String tableId) {
+        return !deletedTableId.equals(tableId);
+      }
+    };
+    LargestFirstMemoryManagerWithExistenceCheck mgr = new LargestFirstMemoryManagerWithExistenceCheck(existenceCheck);
+    ServerConfiguration config = new ServerConfiguration() {
+      ServerConfigurationFactory delegate = new ServerConfigurationFactory(new MockInstance());
+
+      @Override
+      public AccumuloConfiguration getConfiguration() {
+        return DefaultConfiguration.getInstance();
+      }
+
+      @Override
+      public TableConfiguration getTableConfiguration(String tableId) {
+        return delegate.getTableConfiguration(tableId);
+      }
+
+      @Override
+      public TableConfiguration getTableConfiguration(KeyExtent extent) {
+        return delegate.getTableConfiguration(extent);
+      }
+
+      @Override
+      public NamespaceConfiguration getNamespaceConfiguration(String namespaceId) {
+        return delegate.getNamespaceConfiguration(namespaceId);
+      }
+
+      @Override
+      public Instance getInstance() {
+        return delegate.getInstance();
+      }
+    };
+    mgr.init(config);
+    MemoryManagementActions result;
+    // one tablet is really big and the other is for a nonexistent table
+    KeyExtent extent = new KeyExtent(new Text("2"), new Text("j"), null);
+    result = mgr.getMemoryManagementActions(tablets(t(extent, ZERO, ONE_GIG, 0), t(k("j"), ZERO, ONE_GIG, 0)));
+    assertEquals(1, result.tabletsToMinorCompact.size());
+    assertEquals(extent, result.tabletsToMinorCompact.get(0));
+  }
+
   private static class LargestFirstMemoryManagerUnderTest extends LargestFirstMemoryManager {
 
     public long currentTime = ZERO;
@@ -174,6 +222,25 @@ public class LargestFirstMemoryManagerTest {
       return 15 * 60 * 1000;
     }
 
+    @Override
+    boolean tableExists(Instance instance, String tableId) {
+      return true;
+    }
+  }
+
+  private static class LargestFirstMemoryManagerWithExistenceCheck extends LargestFirstMemoryManagerUnderTest {
+
+    Function<String,Boolean> existenceCheck;
+
+    public LargestFirstMemoryManagerWithExistenceCheck(Function<String,Boolean> existenceCheck) {
+      super();
+      this.existenceCheck = existenceCheck;
+    }
+
+    @Override
+    boolean tableExists(Instance instance, String tableId) {
+      return existenceCheck.apply(tableId);
+    }
   }
 
   private static KeyExtent k(String endRow) {


[2/9] accumulo git commit: ACCUMULO-3790 Ensure replication table is online before test proceeds

Posted by el...@apache.org.
ACCUMULO-3790 Ensure replication table is online before test proceeds


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

Branch: refs/heads/master
Commit: 671d4da164994a9c0025b246245b8385468778ea
Parents: 2cde562
Author: Josh Elser <el...@apache.org>
Authored: Sat May 9 20:52:29 2015 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sat May 9 20:52:29 2015 -0400

----------------------------------------------------------------------
 .../replication/MultiInstanceReplicationIT.java | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/671d4da1/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
index 3912e98..7dd31c0 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
@@ -225,6 +225,11 @@ public class MultiInstanceReplicationIT extends ConfigurableMacIT {
       Iterators.size(ReplicationTable.getScanner(connMaster).iterator());
       log.info("TabletServer is online");
 
+      while (!ReplicationTable.isOnline(connMaster)) {
+        log.info("Replication table still offline, waiting");
+        Thread.sleep(5000);
+      }
+
       log.info("");
       log.info("Fetching metadata records:");
       for (Entry<Key,Value> kv : connMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
@@ -409,6 +414,11 @@ public class MultiInstanceReplicationIT extends ConfigurableMacIT {
       // Read the data -- the tserver is back up and running
       Iterators.size(connMaster.createScanner(masterTable1, Authorizations.EMPTY).iterator());
 
+      while (!ReplicationTable.isOnline(connMaster)) {
+        log.info("Replication table still offline, waiting");
+        Thread.sleep(5000);
+      }
+
       // Wait for both tables to be replicated
       log.info("Waiting for {} for {}", filesFor1, masterTable1);
       connMaster.replicationOperations().drain(masterTable1, filesFor1);
@@ -516,6 +526,11 @@ public class MultiInstanceReplicationIT extends ConfigurableMacIT {
 
     cluster.exec(TabletServer.class);
 
+    while (!ReplicationTable.isOnline(connMaster)) {
+      log.info("Replication table still offline, waiting");
+      Thread.sleep(5000);
+    }
+
     Iterators.size(connMaster.createScanner(masterTable, Authorizations.EMPTY).iterator());
 
     for (Entry<Key,Value> kv : ReplicationTable.getScanner(connMaster)) {
@@ -638,6 +653,11 @@ public class MultiInstanceReplicationIT extends ConfigurableMacIT {
 
       cluster.exec(TabletServer.class);
 
+      while (!ReplicationTable.isOnline(connMaster)) {
+        log.info("Replication table still offline, waiting");
+        Thread.sleep(5000);
+      }
+
       // Wait until we fully replicated something
       boolean fullyReplicated = false;
       for (int i = 0; i < 10 && !fullyReplicated; i++) {