You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2020/03/06 03:39:28 UTC

[GitHub] [accumulo] ctubbsii opened a new pull request #1551: Use sets instead of arrays for volumes

ctubbsii opened a new pull request #1551: Use sets instead of arrays for volumes
URL: https://github.com/apache/accumulo/pull/1551
 
 
   Originally ACCUMULO-3376, this change replaces the use of `String[]`
   with `Set<String>` throughout the VolumeChooser and VolumeManager code.
   This makes the API for VolumeChooser a bit more friendly, making it
   easier to use Streams and lambdas to filter and transform volumes
   throughout the code.
   
   This change also adds some more sanity checks on the instance volumes
   configuration (checking for empty list and duplicates).
   
   This also includes a utility to warn about planned API changes in
   internal pluggable code (such as the VolumeChooser), but without being
   too spammy.
   
   Update the default `VolumeChooser.choosable()` to return the full set of
   volumes, so that all are checked for flush and sync support on tserver
   startup. This choosable mechanism is a bit dubious anyway, because
   choosers could make different decisions over time (in response to
   configuration changes, for example) than they do at startup. So,
   defaulting to checking for the maximal set seems like a better strategy
   out-of-the-box.
   
   Make ServerConstants.baseUris unmodifiable.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [accumulo] milleruntime commented on a change in pull request #1551: Use sets instead of arrays for volumes

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #1551: Use sets instead of arrays for volumes
URL: https://github.com/apache/accumulo/pull/1551#discussion_r388969624
 
 

 ##########
 File path: core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java
 ##########
 @@ -107,29 +110,31 @@ public static String getConfiguredBaseDir(AccumuloConfiguration conf,
 
         try {
           // pass through URI to unescape hex encoded chars (e.g. convert %2C to "," char)
-          configuredBaseDirs[i++] = new Path(new URI(namespace)).toString();
+          configuredBaseDirs.add(new Path(new URI(namespace)).toString());
         } catch (URISyntaxException e) {
           throw new IllegalArgumentException(Property.INSTANCE_VOLUMES.getKey() + " contains "
               + namespace + " which has a syntax error", e);
         }
       }
     }
 
-    return configuredBaseDirs;
+    LinkedHashSet<String> deduplicated = new LinkedHashSet<>();
+    deduplicated.addAll(configuredBaseDirs);
+    if (deduplicated.isEmpty()) {
+      throw new IllegalArgumentException(
+          Property.INSTANCE_VOLUMES.getKey() + " contains no volumes (" + ns + ")");
+    }
+    if (deduplicated.size() < configuredBaseDirs.size()) {
+      throw new IllegalArgumentException(
+          Property.INSTANCE_VOLUMES.getKey() + " contains duplicate volumes (" + ns + ")");
 
 Review comment:
   Can never have enough sanity checks!  Could create unit tests for these checks, one where two configured Volumes get normalized to the same Path.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [accumulo] milleruntime commented on a change in pull request #1551: Use sets instead of arrays for volumes

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #1551: Use sets instead of arrays for volumes
URL: https://github.com/apache/accumulo/pull/1551#discussion_r388956871
 
 

 ##########
 File path: server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
 ##########
 @@ -385,10 +385,10 @@ public boolean isReady() throws IOException {
   }
 
   @Override
-  public Path matchingFileSystem(Path source, String[] options) {
+  public Path matchingFileSystem(Path source, Set<String> options) {
     try {
       if (ViewFSUtils.isViewFS(source, hadoopConf)) {
-        return ViewFSUtils.matchingFileSystem(source, options, hadoopConf);
+        return ViewFSUtils.matchingFileSystem(source, options.toArray(new String[0]), hadoopConf);
 
 Review comment:
   Could make ```ViewFSUtils``` take a Set as well.  Looks like this is the only class that uses it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [accumulo] milleruntime commented on a change in pull request #1551: Use sets instead of arrays for volumes

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #1551: Use sets instead of arrays for volumes
URL: https://github.com/apache/accumulo/pull/1551#discussion_r388953853
 
 

 ##########
 File path: server/master/src/main/java/org/apache/accumulo/master/tableOps/tableImport/CreateImportDir.java
 ##########
 @@ -47,10 +47,9 @@
     UniqueNameAllocator namer = master.getContext().getUniqueNameAllocator();
 
     Path exportDir = new Path(tableInfo.exportDir);
-    String[] tableDirs = ServerConstants.getTablesDirs(master.getContext());
+    Set<String> tableDirs = ServerConstants.getTablesDirs(master.getContext());
 
-    log.info("Looking for matching filesystem for " + exportDir + " from options "
-        + Arrays.toString(tableDirs));
+    log.info("Looking for matching filesystem for " + exportDir + " from options " + tableDirs);
 
 Review comment:
   Could use sl4j parameterized log statement. 
   ```suggestion
       log.info("Looking for matching filesystem for {} from options {}", exportDir, tableDirs);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [accumulo] ctubbsii commented on a change in pull request #1551: Use sets instead of arrays for volumes

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1551: Use sets instead of arrays for volumes
URL: https://github.com/apache/accumulo/pull/1551#discussion_r389026181
 
 

 ##########
 File path: core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java
 ##########
 @@ -107,29 +110,31 @@ public static String getConfiguredBaseDir(AccumuloConfiguration conf,
 
         try {
           // pass through URI to unescape hex encoded chars (e.g. convert %2C to "," char)
-          configuredBaseDirs[i++] = new Path(new URI(namespace)).toString();
+          configuredBaseDirs.add(new Path(new URI(namespace)).toString());
         } catch (URISyntaxException e) {
           throw new IllegalArgumentException(Property.INSTANCE_VOLUMES.getKey() + " contains "
               + namespace + " which has a syntax error", e);
         }
       }
     }
 
-    return configuredBaseDirs;
+    LinkedHashSet<String> deduplicated = new LinkedHashSet<>();
+    deduplicated.addAll(configuredBaseDirs);
+    if (deduplicated.isEmpty()) {
+      throw new IllegalArgumentException(
+          Property.INSTANCE_VOLUMES.getKey() + " contains no volumes (" + ns + ")");
+    }
+    if (deduplicated.size() < configuredBaseDirs.size()) {
+      throw new IllegalArgumentException(
+          Property.INSTANCE_VOLUMES.getKey() + " contains duplicate volumes (" + ns + ")");
 
 Review comment:
   This code is likely to change with #1397, so I didn't want to spend too much time adding more stuff like that just yet, but can do so with #1397.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [accumulo] milleruntime commented on a change in pull request #1551: Use sets instead of arrays for volumes

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #1551: Use sets instead of arrays for volumes
URL: https://github.com/apache/accumulo/pull/1551#discussion_r388966232
 
 

 ##########
 File path: server/base/src/main/java/org/apache/accumulo/server/fs/InterfaceEvolutionWarner.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.server.fs;
+
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class InterfaceEvolutionWarner {
 
 Review comment:
   This is neat.  The name is a little long though, could just call it ```ApiWarner``` or ```ApiNotifier```.  I wonder if there are other deprecated APIs that we want to warn about?  Could be a follow on task.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [accumulo] ctubbsii commented on a change in pull request #1551: Use sets instead of arrays for volumes

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1551: Use sets instead of arrays for volumes
URL: https://github.com/apache/accumulo/pull/1551#discussion_r389022038
 
 

 ##########
 File path: server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
 ##########
 @@ -385,10 +385,10 @@ public boolean isReady() throws IOException {
   }
 
   @Override
-  public Path matchingFileSystem(Path source, String[] options) {
+  public Path matchingFileSystem(Path source, Set<String> options) {
     try {
       if (ViewFSUtils.isViewFS(source, hadoopConf)) {
-        return ViewFSUtils.matchingFileSystem(source, options, hadoopConf);
+        return ViewFSUtils.matchingFileSystem(source, options.toArray(new String[0]), hadoopConf);
 
 Review comment:
   I'm going to deal with this class separately... I think we can rip out the viewfs support tooling... since we explicitly ban it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [accumulo] ctubbsii merged pull request #1551: Use sets instead of arrays for volumes

Posted by GitBox <gi...@apache.org>.
ctubbsii merged pull request #1551: Use sets instead of arrays for volumes
URL: https://github.com/apache/accumulo/pull/1551
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [accumulo] ctubbsii commented on a change in pull request #1551: Use sets instead of arrays for volumes

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1551: Use sets instead of arrays for volumes
URL: https://github.com/apache/accumulo/pull/1551#discussion_r389021623
 
 

 ##########
 File path: server/master/src/main/java/org/apache/accumulo/master/tableOps/tableImport/CreateImportDir.java
 ##########
 @@ -47,10 +47,9 @@
     UniqueNameAllocator namer = master.getContext().getUniqueNameAllocator();
 
     Path exportDir = new Path(tableInfo.exportDir);
-    String[] tableDirs = ServerConstants.getTablesDirs(master.getContext());
+    Set<String> tableDirs = ServerConstants.getTablesDirs(master.getContext());
 
-    log.info("Looking for matching filesystem for " + exportDir + " from options "
-        + Arrays.toString(tableDirs));
+    log.info("Looking for matching filesystem for " + exportDir + " from options " + tableDirs);
 
 Review comment:
   This was preexisting, but good to change while we're in there.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [accumulo] ctubbsii commented on a change in pull request #1551: Use sets instead of arrays for volumes

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1551: Use sets instead of arrays for volumes
URL: https://github.com/apache/accumulo/pull/1551#discussion_r389023242
 
 

 ##########
 File path: server/base/src/main/java/org/apache/accumulo/server/fs/InterfaceEvolutionWarner.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.server.fs;
+
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class InterfaceEvolutionWarner {
 
 Review comment:
   Nothing is tied to the name, not even its logger, so we can rename, move, and reuse it however we want, as needed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [accumulo] ctubbsii commented on issue #1551: Use sets instead of arrays for volumes

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on issue #1551: Use sets instead of arrays for volumes
URL: https://github.com/apache/accumulo/pull/1551#issuecomment-595585118
 
 
   I wanted to knock this out because it will make a lot of the code cleanup of #1397 easier, I think, and I didn't want the pull request for that to get too big.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services