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/07 21:53:59 UTC

[45/50] [abbrv] git commit: ACCUMULO-2438 TabletGroupWatchers need to watch separate sets of tables for merges

ACCUMULO-2438 TabletGroupWatchers need to 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/21ab9724
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/21ab9724
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/21ab9724

Branch: refs/heads/ACCUMULO-2061
Commit: 21ab9724f47a036a08a0683096ed4896ba79e24d
Parents: a4e5626
Author: Eric Newton <er...@gmail.com>
Authored: Fri Mar 7 11:27:59 2014 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Fri Mar 7 11:28:08 2014 -0500

----------------------------------------------------------------------
 .../accumulo/master/TabletGroupWatcher.java     |  9 ++++-
 .../apache/accumulo/test/TestDeleteRowsIT.java  | 42 ++++++++++++++++++++
 2 files changed, 49 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/21ab9724/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
index cc52e45..d72abd2 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
@@ -123,9 +123,10 @@ class TabletGroupWatcher extends Daemon {
       ClosableIterator<TabletLocationState> iter = null;
       try {
         Map<Text,MergeStats> mergeStatsCache = new HashMap<Text,MergeStats>();
+        Map<Text,MergeStats> currentMerges = new HashMap<Text,MergeStats>();
         for (MergeInfo merge : master.merges()) {
           if (merge.getExtent() != null) {
-            mergeStatsCache.put(merge.getExtent().getTableId(), new MergeStats(merge));
+            currentMerges.put(merge.getExtent().getTableId(), new MergeStats(merge));
           }
         }
         
@@ -179,7 +180,11 @@ class TabletGroupWatcher extends Daemon {
           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 = this.master.getGoalState(tls, mergeStats.getMergeInfo());
           TServerInstance server = tls.getServer();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/21ab9724/test/src/test/java/org/apache/accumulo/test/TestDeleteRowsIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/TestDeleteRowsIT.java b/test/src/test/java/org/apache/accumulo/test/TestDeleteRowsIT.java
new file mode 100644
index 0000000..f33a617
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/TestDeleteRowsIT.java
@@ -0,0 +1,42 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.test.functional.FunctionalTestUtils;
+import org.apache.accumulo.test.functional.SimpleMacIT;
+import org.junit.Test;
+
+public class TestDeleteRowsIT extends SimpleMacIT {
+  
+  @Test(timeout = 60 * 1000)
+  public void test() throws Exception {
+    Connector c = getConnector();
+    String[] tableNames = this.getTableNames(20);
+    for (String tableName : tableNames) {
+      c.tableOperations().create(tableName);
+      c.tableOperations().deleteRows(tableName, null, null);
+      Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY);
+      assertEquals(0, FunctionalTestUtils.count(scanner));
+    }
+  }
+
+}