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 2022/12/19 21:08:32 UTC

[GitHub] [accumulo] EdColeman opened a new pull request, #3137: Removing replication info necessary for upgrade - resolves issue #3097

EdColeman opened a new pull request, #3137:
URL: https://github.com/apache/accumulo/pull/3137

   Removes the replication information to allow the upgrade process to complete.  With these changes the upgrade process can successfully run from 2.1 to 3.0.
   
   - removes replication iterator from metadata table.
   - removes replication properties from values saved in ZooKeeper.
   
   This PR adds a new data version (from 10 to 11) to separate the replication removal from other upgrade ops that occur.  With 3.0, most (if not all) of the previous upgrade ops can be assumed to have occurred be cause 3.0 will only support upgrades from 2.1.  The new data version may then be unnecessary - but it may make merging between versions easier.  The removal of unnecessary upgrades ops will be addressed as a separate PR.
   
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
EdColeman commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1062608411


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java:
##########
@@ -105,8 +106,11 @@ public boolean isParentLevelUpgraded(KeyExtent extent) {
   private static Logger log = LoggerFactory.getLogger(UpgradeCoordinator.class);
 
   private int currentVersion;
-  private Map<Integer,Upgrader> upgraders = Map.of(AccumuloDataVersion.SHORTEN_RFILE_KEYS,
-      new Upgrader8to9(), AccumuloDataVersion.CRYPTO_CHANGES, new Upgrader9to10());
+  // map of "current version" -> upgrader to next version.
+  private Map<Integer,

Review Comment:
   Added final in 686aaa176d



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
EdColeman commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1062522508


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java:
##########
@@ -179,15 +186,22 @@ public synchronized Future<Void> upgradeMetadata(ServerContext context,
           .submit(() -> {
             try {
               for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-                log.info("Upgrading Root from data version {}", v);
-                upgraders.get(v).upgradeRoot(context);
+                log.info("Upgrading Root - current version {} as step towards target version {}", v,
+                    AccumuloDataVersion.get());
+                if (upgraders.get(v) != null) {
+                  upgraders.get(v).upgradeRoot(context);
+                }

Review Comment:
   5436f22281  changes the null check to match  the ZooKeeper  sectionn - using rquireNonNull - while the check may be redundant if the previous check(s) pass it isolates issues with execution order.  The reason behind the check is to provide a better error message - during development a misconfiguration was throwing a null pointer and having a log statement helped.  
   
   Originally, the code skipped that section if null and it was pointed out by @ctubbsii that it would be better to fail.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
EdColeman commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1053674800


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZNAMESPACES;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    deleteReplicationConfigs(zrw, iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");
+    deleteReplMetadataEntries(context);
+  }
+
+  /**
+   * remove +rep entries from metadata.
+   */
+  private void deleteReplMetadataEntries(final ServerContext context) {
+    try (BatchDeleter deleter =
+        context.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 10)) {
+      deleter.setRanges(Collections.singletonList(
+          new Range(REPLICATION_ID.canonical() + ";", REPLICATION_ID.canonical() + "<")));
+      deleter.delete();

Review Comment:
   Maybe?  Although I configured replication I did not actually set-up and run two full instances (yet) - so I have not seen those entries. I think removing the replication iterator from the configs is the minimum required to get an upgraded instance to run - there maybe additional artifacts, but as far as I know, they do not impact functionality.
   
   As other entries are identified, they can be added to this PR is done as a follow-on.  As-is this is sufficient to upgrade and run 3.0 code as it stands now (well, that's the goal)



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman merged pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
EdColeman merged PR #3137:
URL: https://github.com/apache/accumulo/pull/3137


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
EdColeman commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1058645623


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZNAMESPACES;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.RESERVED_PREFIX;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.volume.Volume;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    deleteReplicationConfigs(zrw, iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");
+    deleteReplMetadataEntries(context);
+    deleteReplHdfsFiles(context);
+  }
+
+  /**
+   * remove +rep entries from metadata.
+   */
+  private void deleteReplMetadataEntries(final ServerContext context) {
+    try (BatchDeleter deleter =
+        context.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 10)) {
+
+      Range repTableRange =
+          new Range(REPLICATION_ID.canonical() + ";", true, REPLICATION_ID.canonical() + "<", true);
+      // copied from MetadataSchema 2.1 (removed in 3.0)
+      Range repWalRange =
+          new Range(RESERVED_PREFIX + "repl", true, RESERVED_PREFIX + "repm", false);
+
+      deleter.setRanges(List.of(repTableRange, repWalRange));
+      deleter.delete();
+    } catch (TableNotFoundException | MutationsRejectedException ex) {
+      throw new IllegalStateException("failed to remove replication info from metadata table", ex);
+    }
+  }
+
+  @VisibleForTesting
+  void deleteReplHdfsFiles(final ServerContext context) {
+    try {
+      for (Volume volume : context.getVolumeManager().getVolumes()) {
+        String dirUri = volume.getBasePath() + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
+            + REPLICATION_ID.canonical();

Review Comment:
   Addressed with 8490bde61d - reads the metadata table for replication table files and then deletes the files individually.  Un-referenced files and the dir structure is left alone.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1062427530


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java:
##########
@@ -179,15 +186,22 @@ public synchronized Future<Void> upgradeMetadata(ServerContext context,
           .submit(() -> {
             try {
               for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-                log.info("Upgrading Root from data version {}", v);
-                upgraders.get(v).upgradeRoot(context);
+                log.info("Upgrading Root - current version {} as step towards target version {}", v,
+                    AccumuloDataVersion.get());
+                if (upgraders.get(v) != null) {
+                  upgraders.get(v).upgradeRoot(context);
+                }

Review Comment:
   I think you should replace lines 191-193 with `upgraders.get(v).upgradeRoot(context);`. You added the `requireNonNull` check in the commit you mentioned below, so it shouldn't be null. If it null here, then this code will hide it and move on.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1056729173


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java:
##########
@@ -152,8 +155,13 @@ public synchronized void upgradeZookeeper(ServerContext context,
         abortIfFateTransactions(context);
 
         for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-          log.info("Upgrading Zookeeper from data version {}", v);
-          upgraders.get(v).upgradeZookeeper(context);
+          log.info("Upgrading Zookeeper from data version {} target version {}", v,
+              AccumuloDataVersion.get());
+          var upgrader = upgraders.get(v);
+          log.info("UPGRADER: version:{} upgrader: {}", v, upgrader);

Review Comment:
   Is this expecting the upgrader to have a meaningful toString? If we want it to log here, in a consistent way, we should add a dedicated method for getting the description for logging. Or, we can create an upgrader base class that logs itself when run.



##########
server/base/src/main/java/org/apache/accumulo/server/AccumuloDataVersion.java:
##########
@@ -36,6 +36,11 @@
  */
 public class AccumuloDataVersion {
 
+  /**
+   * version (11) reflects removal of replication starting with 3.0
+   */
+  public static final int REMOVE_REPLICATION = 11;

Review Comment:
   I suspect that this will be just one of several things that this upgrade will perform. It might be better to rename this to something more broadly applicable, such as `REMOVE_DEPRECATIONS_FOR_VERSION_3`. Can always enumerate the details in the javadoc, if desired.



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZNAMESPACES;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.RESERVED_PREFIX;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.volume.Volume;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    deleteReplicationConfigs(zrw, iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");

Review Comment:
   This is upgrading at the info level, which is very visible. It should have a slightly more verbose description if it's going to appear in user logs or on the console, like upgrading metadata entries to do what? or as part of what? As is, these feel like debugging log statements, with minimal information, just used in the course of development and left in unintentionally. I think they should be either meaningful for the user or removed, or reduced to the trace or debug level.



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java:
##########
@@ -179,15 +187,21 @@ public synchronized Future<Void> upgradeMetadata(ServerContext context,
           .submit(() -> {
             try {
               for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-                log.info("Upgrading Root from data version {}", v);
-                upgraders.get(v).upgradeRoot(context);
+                log.info("Upgrading Root from data version {} target version {}", v,
+                    AccumuloDataVersion.get());
+                if (upgraders.get(v) != null) {
+                  upgraders.get(v).upgradeRoot(context);
+                }
               }
 
               setStatus(UpgradeStatus.UPGRADED_ROOT, eventCoordinator);
 
               for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-                log.info("Upgrading Metadata from data version {}", v);
-                upgraders.get(v).upgradeMetadata(context);
+                log.info("Upgrading Metadata from data version {} target version {}", v,
+                    AccumuloDataVersion.get());
+                if (upgraders.get(v) != null) {
+                  upgraders.get(v).upgradeMetadata(context);
+                }

Review Comment:
   Why the null checks here? Under what circumstances would we find an upgrader to be null? If that happens, we don't want to completely ignore it, but we should fail hard. That's not expected.



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZNAMESPACES;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.RESERVED_PREFIX;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.volume.Volume;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    deleteReplicationConfigs(zrw, iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");
+    deleteReplMetadataEntries(context);
+    deleteReplHdfsFiles(context);
+  }
+
+  /**
+   * remove +rep entries from metadata.
+   */
+  private void deleteReplMetadataEntries(final ServerContext context) {
+    try (BatchDeleter deleter =
+        context.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 10)) {
+
+      Range repTableRange =
+          new Range(REPLICATION_ID.canonical() + ";", true, REPLICATION_ID.canonical() + "<", true);
+      // copied from MetadataSchema 2.1 (removed in 3.0)
+      Range repWalRange =
+          new Range(RESERVED_PREFIX + "repl", true, RESERVED_PREFIX + "repm", false);
+
+      deleter.setRanges(List.of(repTableRange, repWalRange));
+      deleter.delete();
+    } catch (TableNotFoundException | MutationsRejectedException ex) {
+      throw new IllegalStateException("failed to remove replication info from metadata table", ex);
+    }
+  }
+
+  @VisibleForTesting
+  void deleteReplHdfsFiles(final ServerContext context) {
+    try {
+      for (Volume volume : context.getVolumeManager().getVolumes()) {
+        String dirUri = volume.getBasePath() + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
+            + REPLICATION_ID.canonical();

Review Comment:
   This is making assumptions that there are no other files in these directories other than files used by the replication table and not used by any other table. That doesn't seem like a safe assumption. For example, if the replication table had been cloned, some of its files might now be referenced by the clone.
   
   Instead of deleting these directories, we should instead read the file entriess in the replication table's tablets and create corresponding `~del` entries for the `accumulo-gc` to delete later when it determines they no longer need to be preserved. This also allows the actual deletion to be deferred by the user by avoiding running the `accumulo-gc` during upgrade (which might be one way they could upgrade more cautiously) or by utilizing the HDFS trash that the `accumulo-gc` is configured to use.



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java:
##########
@@ -152,8 +155,13 @@ public synchronized void upgradeZookeeper(ServerContext context,
         abortIfFateTransactions(context);
 
         for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-          log.info("Upgrading Zookeeper from data version {}", v);
-          upgraders.get(v).upgradeZookeeper(context);
+          log.info("Upgrading Zookeeper from data version {} target version {}", v,
+              AccumuloDataVersion.get());

Review Comment:
   I think this log message change could be misleading. These are incremental upgrades, but the target version is just the final version. For example, the message may imply that it is upgrading from 5 to 9, but is really doing 5 to 6, then 6 to 7, etc. until it gets to 8 to 9.



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZNAMESPACES;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.RESERVED_PREFIX;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.volume.Volume;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    deleteReplicationConfigs(zrw, iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");
+    deleteReplMetadataEntries(context);
+    deleteReplHdfsFiles(context);
+  }
+
+  /**
+   * remove +rep entries from metadata.
+   */
+  private void deleteReplMetadataEntries(final ServerContext context) {
+    try (BatchDeleter deleter =
+        context.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 10)) {
+
+      Range repTableRange =
+          new Range(REPLICATION_ID.canonical() + ";", true, REPLICATION_ID.canonical() + "<", true);
+      // copied from MetadataSchema 2.1 (removed in 3.0)
+      Range repWalRange =
+          new Range(RESERVED_PREFIX + "repl", true, RESERVED_PREFIX + "repm", false);
+
+      deleter.setRanges(List.of(repTableRange, repWalRange));
+      deleter.delete();
+    } catch (TableNotFoundException | MutationsRejectedException ex) {
+      throw new IllegalStateException("failed to remove replication info from metadata table", ex);
+    }
+  }
+
+  @VisibleForTesting
+  void deleteReplHdfsFiles(final ServerContext context) {
+    try {
+      for (Volume volume : context.getVolumeManager().getVolumes()) {
+        String dirUri = volume.getBasePath() + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
+            + REPLICATION_ID.canonical();
+        Path replPath = new Path(dirUri);
+        if (volume.getFileSystem().exists(replPath)) {
+          try {
+            log.debug("Removing replication dir and files in hdfs {}", replPath);
+            volume.getFileSystem().delete(replPath, true);
+          } catch (IOException ex) {
+            log.error("Unable to remove replication dir and files from " + replPath + ": " + ex);
+          }
+        }
+      }
+    } catch (IOException ex) {
+      log.error("Unable to remove replication dir and files: " + ex);
+    }
+  }
+
+  private boolean checkReplicationTableInZk(final InstanceId iid, final ZooReaderWriter zrw) {
+    try {
+      String path = buildRepTablePath(iid);
+      return zrw.exists(path);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException("ZooKeeper error - cannot determine replication table status",
+          ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("interrupted reading replication state from ZooKeeper", ex);
+    }
+  }
+
+  /**
+   * To protect against removing replication information if replication is being used and possible
+   * active, check the replication table state in Zookeeper to see if it is ONLINE (active) or
+   * OFFLINE (inactive). If the state node does not exist, then the status is considered as OFFLINE.
+   *
+   * @return true if the replication table state is OFFLINE, false otherwise
+   */
+  private boolean checkReplicationOffline(final InstanceId iid, final ZooReaderWriter zrw) {
+    try {
+      String path = buildRepTablePath(iid) + ZTABLE_STATE;
+      byte[] bytes = zrw.getData(path);
+      if (bytes != null && bytes.length > 0) {
+        String status = new String(bytes, UTF_8);
+        return TableState.OFFLINE.name().equals(status);
+      }
+      return false;
+    } catch (KeeperException ex) {
+      throw new IllegalStateException("ZooKeeper error - cannot determine replication table status",
+          ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("interrupted reading replication state from ZooKeeper", ex);
+    }
+  }
+
+  /**
+   * Utility method to build the ZooKeeper replication table path. The path resolves to
+   * {@code /accumulo/INSTANCE_ID/tables/+rep}
+   */
+  static String buildRepTablePath(final InstanceId iid) {
+    return ZooUtil.getRoot(iid) + ZTABLES + "/" + REPLICATION_ID.canonical();
+  }
+
+  private void deleteReplicationTableZkEntries(ZooReaderWriter zrw, InstanceId iid) {
+    String repTablePath = buildRepTablePath(iid);
+    try {
+      zrw.recursiveDelete(repTablePath, ZooUtil.NodeMissingPolicy.SKIP);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException(
+          "ZooKeeper error - failed recursive deletion on " + repTablePath, ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("interrupted deleting " + repTablePath + " from ZooKeeper",
+          ex);
+    }
+  }
+
+  private void deleteReplicationConfigs(ZooReaderWriter zrw, InstanceId iid, PropStore propStore) {
+    List<PropStoreKey<?>> ids = getPropKeysFromZkIds(zrw, iid);

Review Comment:
   Is this just removing them from the cache? It seems that these will be deleted from ZK later when we delete everything from ZK for this table. So, I'm not sure if this is strictly necessary to do separately. And, deleting the configs from ZK may cause the per-table configuration to enter a strange state, where its configuration now is the default per-table configuration or that set in the site configuration file or in the accumulo namespace config in ZK... which may not be what we want, because it could cause all sorts of things to change for the table that's supposed to be offline.



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZNAMESPACES;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    deleteReplicationConfigs(zrw, iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");
+    deleteReplMetadataEntries(context);
+  }
+
+  /**
+   * remove +rep entries from metadata.
+   */
+  private void deleteReplMetadataEntries(final ServerContext context) {
+    try (BatchDeleter deleter =
+        context.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 10)) {
+      deleter.setRanges(Collections.singletonList(
+          new Range(REPLICATION_ID.canonical() + ";", REPLICATION_ID.canonical() + "<")));
+      deleter.delete();

Review Comment:
   The repTableRange would be deleting the metadata entries that define the replication table. The repWalRange would be deleting the additionally tracked WALs that are preserved because they are in use, and intended to prevent the accumulo-gc from deleting the still-referenced WALs. I think that's all we're expecting.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
EdColeman commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1057914369


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZNAMESPACES;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.RESERVED_PREFIX;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.volume.Volume;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    deleteReplicationConfigs(zrw, iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");
+    deleteReplMetadataEntries(context);
+    deleteReplHdfsFiles(context);
+  }
+
+  /**
+   * remove +rep entries from metadata.
+   */
+  private void deleteReplMetadataEntries(final ServerContext context) {
+    try (BatchDeleter deleter =
+        context.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 10)) {
+
+      Range repTableRange =
+          new Range(REPLICATION_ID.canonical() + ";", true, REPLICATION_ID.canonical() + "<", true);
+      // copied from MetadataSchema 2.1 (removed in 3.0)
+      Range repWalRange =
+          new Range(RESERVED_PREFIX + "repl", true, RESERVED_PREFIX + "repm", false);
+
+      deleter.setRanges(List.of(repTableRange, repWalRange));
+      deleter.delete();
+    } catch (TableNotFoundException | MutationsRejectedException ex) {
+      throw new IllegalStateException("failed to remove replication info from metadata table", ex);
+    }
+  }
+
+  @VisibleForTesting
+  void deleteReplHdfsFiles(final ServerContext context) {
+    try {
+      for (Volume volume : context.getVolumeManager().getVolumes()) {
+        String dirUri = volume.getBasePath() + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
+            + REPLICATION_ID.canonical();
+        Path replPath = new Path(dirUri);
+        if (volume.getFileSystem().exists(replPath)) {
+          try {
+            log.debug("Removing replication dir and files in hdfs {}", replPath);
+            volume.getFileSystem().delete(replPath, true);
+          } catch (IOException ex) {
+            log.error("Unable to remove replication dir and files from " + replPath + ": " + ex);
+          }
+        }
+      }
+    } catch (IOException ex) {
+      log.error("Unable to remove replication dir and files: " + ex);
+    }
+  }
+
+  private boolean checkReplicationTableInZk(final InstanceId iid, final ZooReaderWriter zrw) {
+    try {
+      String path = buildRepTablePath(iid);
+      return zrw.exists(path);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException("ZooKeeper error - cannot determine replication table status",
+          ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("interrupted reading replication state from ZooKeeper", ex);
+    }
+  }
+
+  /**
+   * To protect against removing replication information if replication is being used and possible
+   * active, check the replication table state in Zookeeper to see if it is ONLINE (active) or
+   * OFFLINE (inactive). If the state node does not exist, then the status is considered as OFFLINE.
+   *
+   * @return true if the replication table state is OFFLINE, false otherwise
+   */
+  private boolean checkReplicationOffline(final InstanceId iid, final ZooReaderWriter zrw) {
+    try {
+      String path = buildRepTablePath(iid) + ZTABLE_STATE;
+      byte[] bytes = zrw.getData(path);
+      if (bytes != null && bytes.length > 0) {
+        String status = new String(bytes, UTF_8);
+        return TableState.OFFLINE.name().equals(status);
+      }
+      return false;
+    } catch (KeeperException ex) {
+      throw new IllegalStateException("ZooKeeper error - cannot determine replication table status",
+          ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("interrupted reading replication state from ZooKeeper", ex);
+    }
+  }
+
+  /**
+   * Utility method to build the ZooKeeper replication table path. The path resolves to
+   * {@code /accumulo/INSTANCE_ID/tables/+rep}
+   */
+  static String buildRepTablePath(final InstanceId iid) {
+    return ZooUtil.getRoot(iid) + ZTABLES + "/" + REPLICATION_ID.canonical();
+  }
+
+  private void deleteReplicationTableZkEntries(ZooReaderWriter zrw, InstanceId iid) {
+    String repTablePath = buildRepTablePath(iid);
+    try {
+      zrw.recursiveDelete(repTablePath, ZooUtil.NodeMissingPolicy.SKIP);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException(
+          "ZooKeeper error - failed recursive deletion on " + repTablePath, ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("interrupted deleting " + repTablePath + " from ZooKeeper",
+          ex);
+    }
+  }
+
+  private void deleteReplicationConfigs(ZooReaderWriter zrw, InstanceId iid, PropStore propStore) {
+    List<PropStoreKey<?>> ids = getPropKeysFromZkIds(zrw, iid);

Review Comment:
   No, it is more that removing them from the cache - `propStore.removeProperties`  updates ZooKeeper.  The issue is when there are replication iterator configs specified in the the configuration - particularly the metadata table. But this is also checking the other ZooKeeper configs to make sure that if the replication iterator was set it is removed.
   
   In 2.1, the ZooKeeper config for the metadata table has these replication entries:
   
   ```
   root@uno> config -t accumulo.metadata -f replc
   -----------+-------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------
   SCOPE      | NAME                                            | VALUE
   -----------+-------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------
   table      | table.iterator.majc.replcombiner .............. | 9,org.apache.accumulo.server.replication.StatusCombiner
   table      | table.iterator.majc.replcombiner.opt.columns .. | stat
   table      | table.iterator.minc.replcombiner .............. | 9,org.apache.accumulo.server.replication.StatusCombiner
   table      | table.iterator.minc.replcombiner.opt.columns .. | stat
   table      | table.iterator.scan.replcombiner .............. | 9,org.apache.accumulo.server.replication.StatusCombiner
   table      | table.iterator.scan.replcombiner.opt.columns .. | stat
   -----------+-------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------
   
   ```
   
   And, the StatusCombiner, if present, will cause 3.0 code to fail because they have been removed from the code base. 
   
   And at this point, only the metadata and root tables are online.
   
   Not sure what to do if replication iterators have been set in the system config file - that's a manual removal - but if present they would likely cause the manager to go offline, if the upgrade sequence could complete.  The easiest way is for the upgrade instruction to include removing any replication configuration values with 2.1 down , but before starting 3.0.  There will likely be manual steps to completely remove replication remnants  - but functionally they will not prevent Accumulo from running. 
   



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
EdColeman commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1057914629


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java:
##########
@@ -179,15 +187,21 @@ public synchronized Future<Void> upgradeMetadata(ServerContext context,
           .submit(() -> {
             try {
               for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-                log.info("Upgrading Root from data version {}", v);
-                upgraders.get(v).upgradeRoot(context);
+                log.info("Upgrading Root from data version {} target version {}", v,
+                    AccumuloDataVersion.get());
+                if (upgraders.get(v) != null) {
+                  upgraders.get(v).upgradeRoot(context);
+                }
               }
 
               setStatus(UpgradeStatus.UPGRADED_ROOT, eventCoordinator);
 
               for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-                log.info("Upgrading Metadata from data version {}", v);
-                upgraders.get(v).upgradeMetadata(context);
+                log.info("Upgrading Metadata from data version {} target version {}", v,
+                    AccumuloDataVersion.get());
+                if (upgraders.get(v) != null) {
+                  upgraders.get(v).upgradeMetadata(context);
+                }

Review Comment:
   Addressed with b3ca1d9b65



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
EdColeman commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1057784097


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZNAMESPACES;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.RESERVED_PREFIX;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.volume.Volume;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    deleteReplicationConfigs(zrw, iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");

Review Comment:
   This appears once - during initialization.  Along with the classn ame in the log message highlights the upgrade steps and seems sufficient.
   
   ```
   2022-12-23T21:01:37,482 [upgrade.Upgrader10to11] INFO : upgrade metadata entries
   ...
   022-12-23T21:01:37,482 [upgrade.Upgrader10to11] INFO : upgrade root - skipping, nothing to do
   ...
   2022-12-23T21:01:37,482 [upgrade.Upgrader10to11] INFO : upgrade metadata entries
   ```
   
   Having these stand-out beyond debug seems appropriate and would help if things do not proceed as expected.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
EdColeman commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1062607877


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.RESERVED_PREFIX;
+import static org.apache.accumulo.server.util.MetadataTableUtil.EMPTY_TEXT;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.hadoop.io.Text;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  private static final Range REP_TABLE_RANGE =
+      new Range(REPLICATION_ID.canonical() + ";", true, REPLICATION_ID.canonical() + "<", true);
+
+  // copied from MetadataSchema 2.1 (removed in 3.0)
+  private static final Range REP_WAL_RANGE =
+      new Range(RESERVED_PREFIX + "repl", true, RESERVED_PREFIX + "repm", false);
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    cleanMetaConfig(iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");
+    List<String> replTableFiles = readReplFilesFromMetadata(context);
+    deleteReplMetadataEntries(context);
+    deleteReplTableFiles(context, replTableFiles);
+  }
+
+  List<String> readReplFilesFromMetadata(final ServerContext context) {
+    List<String> results = new ArrayList<>();
+    try (Scanner scanner = context.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME);
+      scanner.setRange(REP_TABLE_RANGE);
+      for (Map.Entry<Key,Value> entry : scanner) {
+        String f = entry.getKey()
+            .getColumnQualifier(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME).toString();
+        results.add(f);
+      }
+    } catch (TableNotFoundException ex) {
+      throw new IllegalStateException("failed to read replication files from metadata", ex);
+    }
+    return results;
+  }
+
+  void deleteReplTableFiles(final ServerContext context, final List<String> replTableFiles) {
+    // write delete mutations

Review Comment:
   Address in 686aaa176d



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1062426873


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java:
##########
@@ -179,15 +187,21 @@ public synchronized Future<Void> upgradeMetadata(ServerContext context,
           .submit(() -> {
             try {
               for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-                log.info("Upgrading Root from data version {}", v);
-                upgraders.get(v).upgradeRoot(context);
+                log.info("Upgrading Root from data version {} target version {}", v,
+                    AccumuloDataVersion.get());
+                if (upgraders.get(v) != null) {
+                  upgraders.get(v).upgradeRoot(context);
+                }
               }
 
               setStatus(UpgradeStatus.UPGRADED_ROOT, eventCoordinator);
 
               for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-                log.info("Upgrading Metadata from data version {}", v);
-                upgraders.get(v).upgradeMetadata(context);
+                log.info("Upgrading Metadata from data version {} target version {}", v,
+                    AccumuloDataVersion.get());
+                if (upgraders.get(v) != null) {
+                  upgraders.get(v).upgradeMetadata(context);
+                }

Review Comment:
   I think you should replace lines 202 - 204 with `upgraders.get(v).upgradeMetadata(context);`. You added the `requireNonNull` check in the commit you mentioned, so it shouldn't be null. If it null here, then this code will hide it and move on.



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java:
##########
@@ -105,8 +106,11 @@ public boolean isParentLevelUpgraded(KeyExtent extent) {
   private static Logger log = LoggerFactory.getLogger(UpgradeCoordinator.class);
 
   private int currentVersion;
-  private Map<Integer,Upgrader> upgraders = Map.of(AccumuloDataVersion.SHORTEN_RFILE_KEYS,
-      new Upgrader8to9(), AccumuloDataVersion.CRYPTO_CHANGES, new Upgrader9to10());
+  // map of "current version" -> upgrader to next version.
+  private Map<Integer,

Review Comment:
   `Map.of` returns an unmodifiable map. If you use `final` here, then the `upgraders` variable can't be reassigned. This should give you confidence in the map contents in the code below.



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.RESERVED_PREFIX;
+import static org.apache.accumulo.server.util.MetadataTableUtil.EMPTY_TEXT;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.hadoop.io.Text;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  private static final Range REP_TABLE_RANGE =
+      new Range(REPLICATION_ID.canonical() + ";", true, REPLICATION_ID.canonical() + "<", true);
+
+  // copied from MetadataSchema 2.1 (removed in 3.0)
+  private static final Range REP_WAL_RANGE =
+      new Range(RESERVED_PREFIX + "repl", true, RESERVED_PREFIX + "repm", false);
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    cleanMetaConfig(iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");
+    List<String> replTableFiles = readReplFilesFromMetadata(context);
+    deleteReplMetadataEntries(context);
+    deleteReplTableFiles(context, replTableFiles);
+  }
+
+  List<String> readReplFilesFromMetadata(final ServerContext context) {
+    List<String> results = new ArrayList<>();
+    try (Scanner scanner = context.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME);
+      scanner.setRange(REP_TABLE_RANGE);
+      for (Map.Entry<Key,Value> entry : scanner) {
+        String f = entry.getKey()
+            .getColumnQualifier(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME).toString();
+        results.add(f);
+      }
+    } catch (TableNotFoundException ex) {
+      throw new IllegalStateException("failed to read replication files from metadata", ex);
+    }
+    return results;
+  }
+
+  void deleteReplTableFiles(final ServerContext context, final List<String> replTableFiles) {
+    // write delete mutations

Review Comment:
   you could short-circuit here if `replTableFiles` is empty



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java:
##########
@@ -179,15 +186,22 @@ public synchronized Future<Void> upgradeMetadata(ServerContext context,
           .submit(() -> {
             try {
               for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-                log.info("Upgrading Root from data version {}", v);
-                upgraders.get(v).upgradeRoot(context);
+                log.info("Upgrading Root - current version {} as step towards target version {}", v,
+                    AccumuloDataVersion.get());
+                if (upgraders.get(v) != null) {
+                  upgraders.get(v).upgradeRoot(context);
+                }

Review Comment:
   I think you should replace lines 191-1932 with `upgraders.get(v).upgradeRoot(context);`. You added the `requireNonNull` check in the commit you mentioned below, so it shouldn't be null. If it null here, then this code will hide it and move on.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
EdColeman commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1057914921


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java:
##########
@@ -152,8 +155,13 @@ public synchronized void upgradeZookeeper(ServerContext context,
         abortIfFateTransactions(context);
 
         for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-          log.info("Upgrading Zookeeper from data version {}", v);
-          upgraders.get(v).upgradeZookeeper(context);
+          log.info("Upgrading Zookeeper from data version {} target version {}", v,
+              AccumuloDataVersion.get());

Review Comment:
   Addressed with b3ca1d9b65



##########
server/base/src/main/java/org/apache/accumulo/server/AccumuloDataVersion.java:
##########
@@ -36,6 +36,11 @@
  */
 public class AccumuloDataVersion {
 
+  /**
+   * version (11) reflects removal of replication starting with 3.0
+   */
+  public static final int REMOVE_REPLICATION = 11;

Review Comment:
   Changed in b3ca1d9b65



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1056727912


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZNAMESPACES;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    deleteReplicationConfigs(zrw, iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");
+    deleteReplMetadataEntries(context);
+  }
+
+  /**
+   * remove +rep entries from metadata.
+   */
+  private void deleteReplMetadataEntries(final ServerContext context) {
+    try (BatchDeleter deleter =
+        context.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 10)) {
+      deleter.setRanges(Collections.singletonList(
+          new Range(REPLICATION_ID.canonical() + ";", REPLICATION_ID.canonical() + "<")));
+      deleter.delete();

Review Comment:
   I don't know what other tables would have replication entries. There's just the replication table (in its entirety) and these entries in the metadata table, as far as I can tell.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1053666306


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZNAMESPACES;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    deleteReplicationConfigs(zrw, iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");
+    deleteReplMetadataEntries(context);
+  }
+
+  /**
+   * remove +rep entries from metadata.
+   */
+  private void deleteReplMetadataEntries(final ServerContext context) {
+    try (BatchDeleter deleter =
+        context.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 10)) {
+      deleter.setRanges(Collections.singletonList(
+          new Range(REPLICATION_ID.canonical() + ";", REPLICATION_ID.canonical() + "<")));
+      deleter.delete();

Review Comment:
   This deletes the `+rep` table entries, right? Do we also need to delete any replication entries within other tables? Reference: https://github.com/apache/accumulo/blob/2.1/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java#L404



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
EdColeman commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1057798068


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZNAMESPACES;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.RESERVED_PREFIX;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.volume.Volume;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    deleteReplicationConfigs(zrw, iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");
+    deleteReplMetadataEntries(context);
+    deleteReplHdfsFiles(context);
+  }
+
+  /**
+   * remove +rep entries from metadata.
+   */
+  private void deleteReplMetadataEntries(final ServerContext context) {
+    try (BatchDeleter deleter =
+        context.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 10)) {
+
+      Range repTableRange =
+          new Range(REPLICATION_ID.canonical() + ";", true, REPLICATION_ID.canonical() + "<", true);
+      // copied from MetadataSchema 2.1 (removed in 3.0)
+      Range repWalRange =
+          new Range(RESERVED_PREFIX + "repl", true, RESERVED_PREFIX + "repm", false);
+
+      deleter.setRanges(List.of(repTableRange, repWalRange));
+      deleter.delete();
+    } catch (TableNotFoundException | MutationsRejectedException ex) {
+      throw new IllegalStateException("failed to remove replication info from metadata table", ex);
+    }
+  }
+
+  @VisibleForTesting
+  void deleteReplHdfsFiles(final ServerContext context) {
+    try {
+      for (Volume volume : context.getVolumeManager().getVolumes()) {
+        String dirUri = volume.getBasePath() + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
+            + REPLICATION_ID.canonical();

Review Comment:
   Will need to look at this - during the upgrade, and depending on the phase, there is limited information - what is in zookeeper, what is in the root table and what is in the metadata - other tables have not been brought online. 
   
   One way to handle this could be through instructions.
   
   The replication table must be offline for the upgrade to proceed.  So, if someone is using replication, the instructions could include the warning that any tables that may have been cloned / shared data with the replication table, they should be compacted prior to upgrading so they do not share any files.
   
   Alternatively, the files could be left in place and leave it up to the user to delete the hdfs directory if they want.  
   
   Removing the zookeeper and metadata entries during the upgrade process seems to be required because of the "special" status as a system table, but also closes off some other options.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3137: Removing replication info necessary for upgrade - resolves issue #3097

Posted by GitBox <gi...@apache.org>.
EdColeman commented on code in PR #3137:
URL: https://github.com/apache/accumulo/pull/3137#discussion_r1057914849


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java:
##########
@@ -152,8 +155,13 @@ public synchronized void upgradeZookeeper(ServerContext context,
         abortIfFateTransactions(context);
 
         for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-          log.info("Upgrading Zookeeper from data version {}", v);
-          upgraders.get(v).upgradeZookeeper(context);
+          log.info("Upgrading Zookeeper from data version {} target version {}", v,
+              AccumuloDataVersion.get());
+          var upgrader = upgraders.get(v);
+          log.info("UPGRADER: version:{} upgrader: {}", v, upgrader);

Review Comment:
   The was leftover from development - addressed with b3ca1d9b65



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org