You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "dlmarion (via GitHub)" <gi...@apache.org> on 2023/04/21 14:54:52 UTC

[GitHub] [accumulo] dlmarion opened a new pull request, #3327: Modified user initiated tablet hosting goal to use FATE

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

   Closes #3303


-- 
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 #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1173979044


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);
+    LOG.debug("Setting tablet hosting goal to {} requested for: {} ", goal, extents);
+    try (TabletsMutator mutator = manager.getContext().getAmple().mutateTablets()) {
+      extents.forEach(e -> mutator.mutateTablet(e).setHostingGoal(goal).mutate());

Review Comment:
   Do splits happen without a FATE table lock?



-- 
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 merged pull request #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion merged PR #3327:
URL: https://github.com/apache/accumulo/pull/3327


-- 
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] keith-turner commented on a diff in pull request #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1174021451


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);

Review Comment:
   That worked nicely.



-- 
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] keith-turner commented on a diff in pull request #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1174031465


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);
+    LOG.debug("Setting tablet hosting goal to {} requested for: {} ", goal, extents);
+    try (TabletsMutator mutator = manager.getContext().getAmple().mutateTablets()) {
+      extents.forEach(e -> mutator.mutateTablet(e).setHostingGoal(goal).mutate());

Review Comment:
   I think the closest example is bulk import.  It makes rpcs to the tablet and this helps avoid concurrent problems with splits.  The tablet on the tablet server writes the data to the metadata table.  The fate op keeps scanning until it sees the data in all tablets in the range.  The table lock helps avoid concurrent problems with destructive operations like delete table. 
   
   So it avoids problems with concurrent operations with a combo of repeated metadata scans, tablet RPCs, and a table lock.
   
   So in the old model of how Accumulo worked we probably would have added an RPC to the tablet to write the new hosting 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] dlmarion commented on pull request #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#issuecomment-1521671841

   Draft follow-on issue created in the Elasticity project


-- 
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 #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1174036727


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);
+    LOG.debug("Setting tablet hosting goal to {} requested for: {} ", goal, extents);
+    try (TabletsMutator mutator = manager.getContext().getAmple().mutateTablets()) {
+      extents.forEach(e -> mutator.mutateTablet(e).setHostingGoal(goal).mutate());

Review Comment:
   Right, so it likely makes sense to make this a follow-on task and *not* add more RPCs to the Tablet/TabletServer now.
   



-- 
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 #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1174014694


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);

Review Comment:
   I moved the logic in `findExtentsForRange` into the `call` method so that we did not create a collection of KeyExtents at runtime.



-- 
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] keith-turner commented on a diff in pull request #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1173971306


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);
+    LOG.debug("Setting tablet hosting goal to {} requested for: {} ", goal, extents);
+    try (TabletsMutator mutator = manager.getContext().getAmple().mutateTablets()) {
+      extents.forEach(e -> mutator.mutateTablet(e).setHostingGoal(goal).mutate());

Review Comment:
   We need a follow on issue to address concurrency issues.  Current fate ops handle that w/ table lock and detection of concurrent split.  However I am not sure if its worthwhile to write code for this new fate op to do all of that as we may replace that w/ conditional mutations in the future.  



##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);

Review Comment:
   If its possible it would be nice to avoid materializing this into manager memory.  If we could instead get an Iterator/Iterable/Stream of KeyExtents that need their goal updated that would be better for avoid memory problems on the manager.  The Iterable could be backed by a scanner that filters to things within in the range.
   
   While this would be better for the managers runtime memory, I don't know what the code would look like.  So a bit uncertain if it ends up being better for manager memory and worse for code maintainability.
   
   This could be a follow on issue.



-- 
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] keith-turner commented on a diff in pull request #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1174146582


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);
+    LOG.debug("Setting tablet hosting goal to {} requested for: {} ", goal, extents);
+    try (TabletsMutator mutator = manager.getContext().getAmple().mutateTablets()) {
+      extents.forEach(e -> mutator.mutateTablet(e).setHostingGoal(goal).mutate());

Review Comment:
   
   Thinking through how this might be handled using only conditional mutations (and no table locks), something like the following may work using multiple fate steps.  The algorithm tries to reserve tablets before making the changes inorder to detect concurrent collisions with other FATE transactions trying to set the goal on the same range.
   
   The following is the first fate step.
   
   ```
   while(true) {
      foreach tablet in range {
           Do a conditional mutation settting coulumn changingGoal=<Fate tx id> IF column changingGoal is absent
      }  
   
      if conditional mutations failed {
         if there are other fate tx also changingGoal
            if the other fate tx id is lower then ours then delete our changingGoal columns and wait for the other fate tx to go away
            if the other fate tx id is higher than our then wait for them to go away
      } else {
        break;
      }
   }
   ```
   
   The second fate step would set the goals.  When we get to this step, all of the tablets are reserved for changing the goal so can go ahead and do it.
   
   ```
   while(true) {
      
      foreach tablet {
           Do a conditional mutation settting goal <Fate tx id> IF changing goal is set to my transaction id.
      }
     
      if conditional mutations failed {
        if changingGoal column is not set, then throw an error this is unexpected.
      } else {
         break;
      } 
   }
   ```
   
   The third fate step would delete the changingGoal columns in the range.



-- 
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] keith-turner commented on a diff in pull request #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1174038415


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);
+    LOG.debug("Setting tablet hosting goal to {} requested for: {} ", goal, extents);
+    try (TabletsMutator mutator = manager.getContext().getAmple().mutateTablets()) {
+      extents.forEach(e -> mutator.mutateTablet(e).setHostingGoal(goal).mutate());

Review Comment:
   In the new model of how Accumulo might work it would write a conditional mutation that requires an absent operation.  Like 
   
   https://github.com/keith-turner/accumulo/blob/f06a87be124ed51eb31a06ca15135ae07ae8c2f8/server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java#L86
   
   This would ensure there is no concurrent merge, delete, or split running.   It would have keep scanning until all conditional mutations are successful.



-- 
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 #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1174047221


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);
+    LOG.debug("Setting tablet hosting goal to {} requested for: {} ", goal, extents);
+    try (TabletsMutator mutator = manager.getContext().getAmple().mutateTablets()) {
+      extents.forEach(e -> mutator.mutateTablet(e).setHostingGoal(goal).mutate());

Review Comment:
   > For this particular operation we also want to consider the case of concurrency with itself. Like two concurrent setTabletHostingGoal API calls that overlap the same range but set different goal states.
   
   Doesn't the table lock prevent that?



-- 
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 #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1174022243


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);
+    LOG.debug("Setting tablet hosting goal to {} requested for: {} ", goal, extents);
+    try (TabletsMutator mutator = manager.getContext().getAmple().mutateTablets()) {
+      extents.forEach(e -> mutator.mutateTablet(e).setHostingGoal(goal).mutate());

Review Comment:
   Do you have an example of where this concurrency check is done currently?



-- 
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] keith-turner commented on a diff in pull request #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1174020278


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);
+    LOG.debug("Setting tablet hosting goal to {} requested for: {} ", goal, extents);
+    try (TabletsMutator mutator = manager.getContext().getAmple().mutateTablets()) {
+      extents.forEach(e -> mutator.mutateTablet(e).setHostingGoal(goal).mutate());

Review Comment:
   yeah splits can happen w/o a table lock



-- 
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] keith-turner commented on a diff in pull request #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1174044971


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);
+    LOG.debug("Setting tablet hosting goal to {} requested for: {} ", goal, extents);
+    try (TabletsMutator mutator = manager.getContext().getAmple().mutateTablets()) {
+      extents.forEach(e -> mutator.mutateTablet(e).setHostingGoal(goal).mutate());

Review Comment:
   For this particular operation we also want to consider the case of concurrency with itself.  Like  two concurrent setTabletHostingGoal API calls that overlap the same range but set different goal states. 



-- 
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] keith-turner commented on a diff in pull request #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1174031465


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);
+    LOG.debug("Setting tablet hosting goal to {} requested for: {} ", goal, extents);
+    try (TabletsMutator mutator = manager.getContext().getAmple().mutateTablets()) {
+      extents.forEach(e -> mutator.mutateTablet(e).setHostingGoal(goal).mutate());

Review Comment:
   I think the closest example is bulk import.  It makes rpcs to the tablet and this helps avoid concurrent problems with splits.  The tablet on the tablet server writes the data to the metadata table.  The fate op keeps scanning until it sees the data in all tablets in the range.  The table lock helps avoid concurrent problems with destructive operations like delete table. 
   So it avoids problems with concurrent operations with a combo of repeated metadata scans, tablet RPCs, and a table lock.



-- 
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] keith-turner commented on a diff in pull request #3327: Modified user initiated tablet hosting goal to use FATE

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3327:
URL: https://github.com/apache/accumulo/pull/3327#discussion_r1174102262


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/goal/SetHostingGoal.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.tableOps.goal;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.manager.tableOps.ManagerRepo;
+import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SetHostingGoal extends ManagerRepo {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(SetHostingGoal.class);
+
+  private final TableId tableId;
+  private final NamespaceId namespaceId;
+  private final TRange range;
+  private final TabletHostingGoal goal;
+
+  public SetHostingGoal(TableId tableId, NamespaceId namespaceId, TRange range,
+      TabletHostingGoal goal) {
+    this.tableId = tableId;
+    this.namespaceId = namespaceId;
+    this.range = range;
+    this.goal = goal;
+  }
+
+  @Override
+  public long isReady(long tid, Manager manager) throws Exception {
+    return Utils.reserveNamespace(manager, namespaceId, tid, false, true,
+        TableOperation.SET_HOSTING_GOAL)
+        + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.SET_HOSTING_GOAL);
+  }
+
+  @Override
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    final Range r = new Range(range);
+    final List<KeyExtent> extents = findExtentsForRange(manager.getContext(), tableId, r);
+    LOG.debug("Setting tablet hosting goal to {} requested for: {} ", goal, extents);
+    try (TabletsMutator mutator = manager.getContext().getAmple().mutateTablets()) {
+      extents.forEach(e -> mutator.mutateTablet(e).setHostingGoal(goal).mutate());

Review Comment:
   > Doesn't the table lock prevent that?
   
   Yeah, a table write lock would.  It also prevents all other operations from running (except for splits).



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