You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2023/05/09 13:13:09 UTC

[accumulo] branch elasticity updated: adds tests to ensure operations against offline tables fail

This is an automated email from the ASF dual-hosted git repository.

kturner pushed a commit to branch elasticity
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/elasticity by this push:
     new 0a5845e6d6 adds tests to ensure operations against offline tables fail
0a5845e6d6 is described below

commit 0a5845e6d68d785437b38f899ff6147b382bc80a
Author: Keith Turner <kt...@apache.org>
AuthorDate: Tue May 9 09:10:46 2023 -0400

    adds tests to ensure operations against offline tables fail
    
    this commit pulls some of the test changes from #3143 that are still relevant
---
 .../org/apache/accumulo/test/OfflineTableIT.java   | 107 +++++++++++++++++++++
 .../org/apache/accumulo/test/ScanServerIT.java     |  19 ----
 .../accumulo/test/ScanServerIT_NoServers.java      |  19 ----
 3 files changed, 107 insertions(+), 38 deletions(-)

diff --git a/test/src/main/java/org/apache/accumulo/test/OfflineTableIT.java b/test/src/main/java/org/apache/accumulo/test/OfflineTableIT.java
new file mode 100644
index 0000000000..1ddab10e52
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/OfflineTableIT.java
@@ -0,0 +1,107 @@
+/*
+ * 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
+ *
+ *   https://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.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+public class OfflineTableIT extends SharedMiniClusterBase {
+
+  private static class OfflineTableITConfiguration implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      // Timeout scan sessions after being idle for 3 seconds
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    OfflineTableITConfiguration c = new OfflineTableITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  @Test
+  public void testScanOffline() throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      ScanServerIT.createTableAndIngest(client, tableName, null, 10, 10, "colf");
+      client.tableOperations().offline(tableName, true);
+      assertFalse(client.tableOperations().isOnline(tableName));
+
+      assertThrows(TableOfflineException.class,
+          () -> client.createScanner(tableName, Authorizations.EMPTY));
+    }
+  }
+
+  @Test
+  public void testBatchScanOffline() throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      ScanServerIT.createTableAndIngest(client, tableName, null, 10, 10, "colf");
+      client.tableOperations().offline(tableName, true);
+      assertFalse(client.tableOperations().isOnline(tableName));
+
+      assertThrows(TableOfflineException.class,
+          () -> client.createBatchScanner(tableName, Authorizations.EMPTY));
+    }
+  }
+
+  @Test
+  public void testSplitOffline() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+      client.tableOperations().create(tableName);
+      client.tableOperations().offline(tableName, true);
+      assertFalse(client.tableOperations().isOnline(tableName));
+      TreeSet<Text> splits = new TreeSet<>();
+      splits.add(new Text("m"));
+      assertThrows(TableOfflineException.class, () -> {
+        client.tableOperations().addSplits(tableName, splits);
+      });
+    }
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java
index 528fd7f441..7576413129 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java
@@ -36,7 +36,6 @@ import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
-import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.TimedOutException;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.conf.ClientProperty;
@@ -148,24 +147,6 @@ public class ScanServerIT extends SharedMiniClusterBase {
     }
   }
 
-  @Test
-  public void testScanOfflineTable() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = getUniqueNames(1)[0];
-
-      createTableAndIngest(client, tableName, null, 10, 10, "colf");
-      client.tableOperations().offline(tableName, true);
-
-      assertThrows(TableOfflineException.class, () -> {
-        try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
-          scanner.setRange(new Range());
-          scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
-          assertEquals(100, Iterables.size(scanner));
-        } // when the scanner is closed, all open sessions should be closed
-      });
-    }
-  }
-
   @Test
   @Disabled("Scanner.setTimeout does not work, issue #2606")
   @Timeout(value = 20)
diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java b/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java
index 204e7bb695..782cb56739 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java
@@ -34,7 +34,6 @@ import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
-import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.TimedOutException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Range;
@@ -127,24 +126,6 @@ public class ScanServerIT_NoServers extends SharedMiniClusterBase {
     }
   }
 
-  @Test
-  public void testScanOfflineTable() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = getUniqueNames(1)[0];
-
-      createTableAndIngest(client, tableName, null, 10, 10, "colf");
-      client.tableOperations().offline(tableName, true);
-
-      assertThrows(TableOfflineException.class, () -> {
-        try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
-          scanner.setRange(new Range());
-          scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
-          assertEquals(100, Iterables.size(scanner));
-        } // when the scanner is closed, all open sessions should be closed
-      });
-    }
-  }
-
   @Test
   public void testScanWithNoTserverFallback() throws Exception {