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/03/10 04:50:26 UTC

[1/6] git commit: ACCUMULO-2438 Backport the changes to ensure that we watch separate sets of tables for merges

Repository: accumulo
Updated Branches:
  refs/heads/1.5.2-SNAPSHOT 18006d245 -> 12b53b043
  refs/heads/1.6.0-SNAPSHOT 0297276e6 -> a63e7c9b9
  refs/heads/master 96d54e673 -> f3083c7bb


ACCUMULO-2438 Backport the changes to ensure that we watch separate sets of tables for merges


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

Branch: refs/heads/1.5.2-SNAPSHOT
Commit: 12b53b043b60368fb647db9c0a15916171d4504b
Parents: 18006d2
Author: Josh Elser <el...@apache.org>
Authored: Sun Mar 9 23:34:18 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Mar 9 23:48:44 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/server/master/Master.java   |  9 ++-
 .../apache/accumulo/test/DeleteRowsTest.java    | 78 ++++++++++++++++++++
 2 files changed, 85 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/12b53b04/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index e72fe3f..8c4c864 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -1288,9 +1288,10 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         int unloaded = 0;
         try {
           Map<Text,MergeStats> mergeStatsCache = new HashMap<Text,MergeStats>();
+          Map<Text,MergeStats> currentMerges = new HashMap<Text,MergeStats>();
           for (MergeInfo merge : merges()) {
             if (merge.getRange() != null) {
-              mergeStatsCache.put(merge.getRange().getTableId(), new MergeStats(merge));
+              currentMerges.put(merge.getRange().getTableId(), new MergeStats(merge));
             }
           }
           
@@ -1339,7 +1340,11 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
             Text tableId = tls.extent.getTableId();
             MergeStats mergeStats = mergeStatsCache.get(tableId);
             if (mergeStats == null) {
-              mergeStatsCache.put(tableId, mergeStats = new MergeStats(new MergeInfo()));
+              mergeStats = currentMerges.get(tableId);
+              if (mergeStats == null) {
+                mergeStats = new MergeStats(new MergeInfo());
+              }
+              mergeStatsCache.put(tableId, mergeStats);
             }
             TabletGoalState goal = getGoalState(tls, mergeStats.getMergeInfo());
             TServerInstance server = tls.getServer();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/12b53b04/test/src/test/java/org/apache/accumulo/test/DeleteRowsTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/DeleteRowsTest.java b/test/src/test/java/org/apache/accumulo/test/DeleteRowsTest.java
new file mode 100644
index 0000000..a627dc1
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/DeleteRowsTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import com.google.common.collect.Iterables;
+
+public class DeleteRowsTest {
+  private static final Logger log = Logger.getLogger(DeleteRowsTest.class);
+
+  @Rule
+  public TestName testName = new TestName();
+
+  private static String secret = "superSecret";
+  public static TemporaryFolder folder = new TemporaryFolder();
+  public static MiniAccumuloCluster cluster;
+  
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    folder.create();
+    MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("miniAccumulo"), secret);
+    cfg.setNumTservers(1);
+    cluster = new MiniAccumuloCluster(cfg);
+    cluster.start();
+    
+  }
+  
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    cluster.stop();
+    folder.delete();
+  }
+
+  @Test(timeout = 120 * 1000)
+  public void test() throws Exception {
+    ZooKeeperInstance zk = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector c = zk.getConnector("root", new PasswordToken(secret));
+    for (int i = 0; i < 20; i++) {
+      final String tableName = testName.getMethodName() + i;
+      log.debug("Creating " + tableName);
+      c.tableOperations().create(tableName);
+      log.debug("Deleting rows from " + tableName);
+      c.tableOperations().deleteRows(tableName, null, null);
+      log.debug("Verifying no rows were found");
+      Scanner scanner = c.createScanner(tableName, new Authorizations());
+      assertEquals(0, Iterables.size(scanner));
+    }
+  }
+}


[3/6] git commit: ACCUMULO-2438 Backport the changes to ensure that we watch separate sets of tables for merges

Posted by el...@apache.org.
ACCUMULO-2438 Backport the changes to ensure that we watch separate sets of tables for merges


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

Branch: refs/heads/master
Commit: 12b53b043b60368fb647db9c0a15916171d4504b
Parents: 18006d2
Author: Josh Elser <el...@apache.org>
Authored: Sun Mar 9 23:34:18 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Mar 9 23:48:44 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/server/master/Master.java   |  9 ++-
 .../apache/accumulo/test/DeleteRowsTest.java    | 78 ++++++++++++++++++++
 2 files changed, 85 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/12b53b04/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index e72fe3f..8c4c864 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -1288,9 +1288,10 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         int unloaded = 0;
         try {
           Map<Text,MergeStats> mergeStatsCache = new HashMap<Text,MergeStats>();
+          Map<Text,MergeStats> currentMerges = new HashMap<Text,MergeStats>();
           for (MergeInfo merge : merges()) {
             if (merge.getRange() != null) {
-              mergeStatsCache.put(merge.getRange().getTableId(), new MergeStats(merge));
+              currentMerges.put(merge.getRange().getTableId(), new MergeStats(merge));
             }
           }
           
@@ -1339,7 +1340,11 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
             Text tableId = tls.extent.getTableId();
             MergeStats mergeStats = mergeStatsCache.get(tableId);
             if (mergeStats == null) {
-              mergeStatsCache.put(tableId, mergeStats = new MergeStats(new MergeInfo()));
+              mergeStats = currentMerges.get(tableId);
+              if (mergeStats == null) {
+                mergeStats = new MergeStats(new MergeInfo());
+              }
+              mergeStatsCache.put(tableId, mergeStats);
             }
             TabletGoalState goal = getGoalState(tls, mergeStats.getMergeInfo());
             TServerInstance server = tls.getServer();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/12b53b04/test/src/test/java/org/apache/accumulo/test/DeleteRowsTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/DeleteRowsTest.java b/test/src/test/java/org/apache/accumulo/test/DeleteRowsTest.java
new file mode 100644
index 0000000..a627dc1
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/DeleteRowsTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import com.google.common.collect.Iterables;
+
+public class DeleteRowsTest {
+  private static final Logger log = Logger.getLogger(DeleteRowsTest.class);
+
+  @Rule
+  public TestName testName = new TestName();
+
+  private static String secret = "superSecret";
+  public static TemporaryFolder folder = new TemporaryFolder();
+  public static MiniAccumuloCluster cluster;
+  
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    folder.create();
+    MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("miniAccumulo"), secret);
+    cfg.setNumTservers(1);
+    cluster = new MiniAccumuloCluster(cfg);
+    cluster.start();
+    
+  }
+  
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    cluster.stop();
+    folder.delete();
+  }
+
+  @Test(timeout = 120 * 1000)
+  public void test() throws Exception {
+    ZooKeeperInstance zk = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector c = zk.getConnector("root", new PasswordToken(secret));
+    for (int i = 0; i < 20; i++) {
+      final String tableName = testName.getMethodName() + i;
+      log.debug("Creating " + tableName);
+      c.tableOperations().create(tableName);
+      log.debug("Deleting rows from " + tableName);
+      c.tableOperations().deleteRows(tableName, null, null);
+      log.debug("Verifying no rows were found");
+      Scanner scanner = c.createScanner(tableName, new Authorizations());
+      assertEquals(0, Iterables.size(scanner));
+    }
+  }
+}


[5/6] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.0-SNAPSHOT

Posted by el...@apache.org.
Merge branch '1.5.2-SNAPSHOT' into 1.6.0-SNAPSHOT

Conflicts:
	server/src/main/java/org/apache/accumulo/server/master/Master.java


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: a63e7c9b9767c0585956616597385f32463be688
Parents: 0297276 12b53b0
Author: Josh Elser <el...@apache.org>
Authored: Sun Mar 9 23:49:47 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Mar 9 23:49:47 2014 -0400

----------------------------------------------------------------------

----------------------------------------------------------------------



[4/6] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.0-SNAPSHOT

Posted by el...@apache.org.
Merge branch '1.5.2-SNAPSHOT' into 1.6.0-SNAPSHOT

Conflicts:
	server/src/main/java/org/apache/accumulo/server/master/Master.java


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

Branch: refs/heads/master
Commit: a63e7c9b9767c0585956616597385f32463be688
Parents: 0297276 12b53b0
Author: Josh Elser <el...@apache.org>
Authored: Sun Mar 9 23:49:47 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Mar 9 23:49:47 2014 -0400

----------------------------------------------------------------------

----------------------------------------------------------------------



[6/6] git commit: Merge branch '1.6.0-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.0-SNAPSHOT'


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

Branch: refs/heads/master
Commit: f3083c7bbd437025e2adc5751f53d600bd4691b2
Parents: 96d54e6 a63e7c9
Author: Josh Elser <el...@apache.org>
Authored: Sun Mar 9 23:50:09 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Mar 9 23:50:09 2014 -0400

----------------------------------------------------------------------

----------------------------------------------------------------------



[2/6] git commit: ACCUMULO-2438 Backport the changes to ensure that we watch separate sets of tables for merges

Posted by el...@apache.org.
ACCUMULO-2438 Backport the changes to ensure that we watch separate sets of tables for merges


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 12b53b043b60368fb647db9c0a15916171d4504b
Parents: 18006d2
Author: Josh Elser <el...@apache.org>
Authored: Sun Mar 9 23:34:18 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Mar 9 23:48:44 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/server/master/Master.java   |  9 ++-
 .../apache/accumulo/test/DeleteRowsTest.java    | 78 ++++++++++++++++++++
 2 files changed, 85 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/12b53b04/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index e72fe3f..8c4c864 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -1288,9 +1288,10 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         int unloaded = 0;
         try {
           Map<Text,MergeStats> mergeStatsCache = new HashMap<Text,MergeStats>();
+          Map<Text,MergeStats> currentMerges = new HashMap<Text,MergeStats>();
           for (MergeInfo merge : merges()) {
             if (merge.getRange() != null) {
-              mergeStatsCache.put(merge.getRange().getTableId(), new MergeStats(merge));
+              currentMerges.put(merge.getRange().getTableId(), new MergeStats(merge));
             }
           }
           
@@ -1339,7 +1340,11 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
             Text tableId = tls.extent.getTableId();
             MergeStats mergeStats = mergeStatsCache.get(tableId);
             if (mergeStats == null) {
-              mergeStatsCache.put(tableId, mergeStats = new MergeStats(new MergeInfo()));
+              mergeStats = currentMerges.get(tableId);
+              if (mergeStats == null) {
+                mergeStats = new MergeStats(new MergeInfo());
+              }
+              mergeStatsCache.put(tableId, mergeStats);
             }
             TabletGoalState goal = getGoalState(tls, mergeStats.getMergeInfo());
             TServerInstance server = tls.getServer();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/12b53b04/test/src/test/java/org/apache/accumulo/test/DeleteRowsTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/DeleteRowsTest.java b/test/src/test/java/org/apache/accumulo/test/DeleteRowsTest.java
new file mode 100644
index 0000000..a627dc1
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/DeleteRowsTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import com.google.common.collect.Iterables;
+
+public class DeleteRowsTest {
+  private static final Logger log = Logger.getLogger(DeleteRowsTest.class);
+
+  @Rule
+  public TestName testName = new TestName();
+
+  private static String secret = "superSecret";
+  public static TemporaryFolder folder = new TemporaryFolder();
+  public static MiniAccumuloCluster cluster;
+  
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    folder.create();
+    MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("miniAccumulo"), secret);
+    cfg.setNumTservers(1);
+    cluster = new MiniAccumuloCluster(cfg);
+    cluster.start();
+    
+  }
+  
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    cluster.stop();
+    folder.delete();
+  }
+
+  @Test(timeout = 120 * 1000)
+  public void test() throws Exception {
+    ZooKeeperInstance zk = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector c = zk.getConnector("root", new PasswordToken(secret));
+    for (int i = 0; i < 20; i++) {
+      final String tableName = testName.getMethodName() + i;
+      log.debug("Creating " + tableName);
+      c.tableOperations().create(tableName);
+      log.debug("Deleting rows from " + tableName);
+      c.tableOperations().deleteRows(tableName, null, null);
+      log.debug("Verifying no rows were found");
+      Scanner scanner = c.createScanner(tableName, new Authorizations());
+      assertEquals(0, Iterables.size(scanner));
+    }
+  }
+}