cshannon commented on code in PR #3929: URL: https://github.com/apache/accumulo/pull/3929#discussion_r1382613926
########## server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/ReserveTablets.java: ########## @@ -0,0 +1,110 @@ +/* + * 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.delete; + +import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION; +import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.OPID; +import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW; + +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiConsumer; + +import org.apache.accumulo.core.data.NamespaceId; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.fate.Repo; +import org.apache.accumulo.core.metadata.schema.Ample; +import org.apache.accumulo.core.metadata.schema.TabletOperationId; +import org.apache.accumulo.core.metadata.schema.TabletOperationType; +import org.apache.accumulo.manager.Manager; +import org.apache.accumulo.manager.tableOps.ManagerRepo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ReserveTablets extends ManagerRepo { + + private static final Logger log = LoggerFactory.getLogger(ReserveTablets.class); + + private static final long serialVersionUID = 1L; + + private final TableId tableId; + private final NamespaceId namespaceId; + + public ReserveTablets(TableId tableId, NamespaceId namespaceId) { + this.tableId = tableId; + this.namespaceId = namespaceId; + } + + @Override + public long isReady(long tid, Manager manager) throws Exception { + + var opid = TabletOperationId.from(TabletOperationType.DELETING, tid); + + // The consumer may be called in another thread so use an AtomicLong + AtomicLong accepted = new AtomicLong(0); + BiConsumer<KeyExtent,Ample.ConditionalResult> resultsConsumer = (extent, result) -> { + if (result.getStatus() == Ample.ConditionalResult.Status.ACCEPTED) { + accepted.incrementAndGet(); + } else { + log.debug("Failed to set operation id {} {}", opid, extent); + } + }; + + long locations = 0; + long otherOps = 0; + long submitted = 0; + long tabletsSeen = 0; + + try ( + var tablets = manager.getContext().getAmple().readTablets().forTable(tableId) + .fetch(OPID, PREV_ROW, LOCATION).checkConsistency().build(); + var conditionalMutator = + manager.getContext().getAmple().conditionallyMutateTablets(resultsConsumer)) { + tabletsSeen++; Review Comment: I'm assuming this counter should be incremented inside the for loop? right now it's only ever going to be 1 as it's incremented one time before the loop. ########## server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/ReserveTablets.java: ########## @@ -0,0 +1,110 @@ +/* + * 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.delete; + +import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION; +import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.OPID; +import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW; + +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiConsumer; + +import org.apache.accumulo.core.data.NamespaceId; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.fate.Repo; +import org.apache.accumulo.core.metadata.schema.Ample; +import org.apache.accumulo.core.metadata.schema.TabletOperationId; +import org.apache.accumulo.core.metadata.schema.TabletOperationType; +import org.apache.accumulo.manager.Manager; +import org.apache.accumulo.manager.tableOps.ManagerRepo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ReserveTablets extends ManagerRepo { + + private static final Logger log = LoggerFactory.getLogger(ReserveTablets.class); + + private static final long serialVersionUID = 1L; + + private final TableId tableId; + private final NamespaceId namespaceId; + + public ReserveTablets(TableId tableId, NamespaceId namespaceId) { + this.tableId = tableId; + this.namespaceId = namespaceId; + } + + @Override + public long isReady(long tid, Manager manager) throws Exception { + + var opid = TabletOperationId.from(TabletOperationType.DELETING, tid); + + // The consumer may be called in another thread so use an AtomicLong + AtomicLong accepted = new AtomicLong(0); + BiConsumer<KeyExtent,Ample.ConditionalResult> resultsConsumer = (extent, result) -> { + if (result.getStatus() == Ample.ConditionalResult.Status.ACCEPTED) { + accepted.incrementAndGet(); + } else { + log.debug("Failed to set operation id {} {}", opid, extent); + } + }; + + long locations = 0; + long otherOps = 0; + long submitted = 0; + long tabletsSeen = 0; + + try ( + var tablets = manager.getContext().getAmple().readTablets().forTable(tableId) + .fetch(OPID, PREV_ROW, LOCATION).checkConsistency().build(); + var conditionalMutator = + manager.getContext().getAmple().conditionallyMutateTablets(resultsConsumer)) { + tabletsSeen++; + for (var tabletMeta : tablets) { Review Comment: This will scan over all tablets each time it's executed but I was thinking maybe as a future enhancement it might be nice to have a filter on what is returned in case we are waiting on an existing op to finish. It may not make it any faster since the iterator still needs to skip the rows but it could make the code a bit cleaner. I'm thinking we could enhance the builder and add a new filter method that could be applied to the row iterator (just like we already filter just like we can filter by extents [here](https://github.com/apache/accumulo/blob/84419e33fc601e52626df25a1032dc0a20488eac/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java#L249)). Adding a generic filter method could be nice for future use cases to filter the tablets returned by some arbitrary condition. For example, in this case the filter could be to return tablets that don't have the delete op id set already: ```Java var tablets = manager.getContext().getAmple().readTablets().forTable(tableId) // new filter method added to TableRangeOptions .filter(tm -> !opid.equals(tm.getOperationId()) .fetch(OPID, PREV_ROW, LOCATION).checkConsistency().build(); ``` Then you'd only see tablets that need to be set. ########## server/base/src/main/java/org/apache/accumulo/server/metadata/AsyncConditionalTabletsMutatorImpl.java: ########## @@ -0,0 +1,95 @@ +/* + * 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.server.metadata; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.function.BiConsumer; + +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.metadata.schema.Ample; +import org.apache.accumulo.server.ServerContext; + +public class AsyncConditionalTabletsMutatorImpl implements Ample.AsyncConditionalTabletsMutator { + private final BiConsumer<KeyExtent,Ample.ConditionalResult> resultsConsumer; + private final ExecutorService executor; + private Future<Map<KeyExtent,Ample.ConditionalResult>> backgroundProcessing = null; + private ConditionalTabletsMutatorImpl bufferingMutator; + private final ServerContext context; + private long mutatedTablets = 0; + + public static final int BATCH_SIZE = 1000; + + AsyncConditionalTabletsMutatorImpl(ServerContext context, + BiConsumer<KeyExtent,Ample.ConditionalResult> resultsConsumer) { + this.resultsConsumer = Objects.requireNonNull(resultsConsumer); + this.bufferingMutator = new ConditionalTabletsMutatorImpl(context); + this.context = context; + this.executor = Executors.newSingleThreadExecutor(); + + } + + @Override + public Ample.OperationRequirements mutateTablet(KeyExtent extent) { + if (mutatedTablets > BATCH_SIZE) { + if (backgroundProcessing != null) { + // a previous batch of mutations was submitted for processing so wait on it. + try { + backgroundProcessing.get().forEach(resultsConsumer); Review Comment: I'm wondering if we should add an optional timeout configuration for the Aync mutator to pass when waiting on the future? I'm not sure how you'd pick a good timeout value as operations could be long running so this might be the best option. ########## core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java: ########## @@ -292,24 +313,26 @@ enum Status { TabletMetadata readMetadata(); } - public interface ConditionalTabletsMutator extends AutoCloseable { - + interface AsyncConditionalTabletsMutator extends AutoCloseable { /** * @return A fluent interface to conditional mutating a tablet. Ensure you call * {@link ConditionalTabletMutator#submit(RejectionHandler)} when finished. */ OperationRequirements mutateTablet(KeyExtent extent); + @Override + void close(); Review Comment: You should be able to remove this method as `close()` is already being inherited from `AutoCloseable` ########## server/base/src/main/java/org/apache/accumulo/server/metadata/AsyncConditionalTabletsMutatorImpl.java: ########## @@ -0,0 +1,95 @@ +/* + * 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.server.metadata; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.function.BiConsumer; + +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.metadata.schema.Ample; +import org.apache.accumulo.server.ServerContext; + +public class AsyncConditionalTabletsMutatorImpl implements Ample.AsyncConditionalTabletsMutator { + private final BiConsumer<KeyExtent,Ample.ConditionalResult> resultsConsumer; + private final ExecutorService executor; + private Future<Map<KeyExtent,Ample.ConditionalResult>> backgroundProcessing = null; + private ConditionalTabletsMutatorImpl bufferingMutator; + private final ServerContext context; + private long mutatedTablets = 0; + + public static final int BATCH_SIZE = 1000; + + AsyncConditionalTabletsMutatorImpl(ServerContext context, + BiConsumer<KeyExtent,Ample.ConditionalResult> resultsConsumer) { + this.resultsConsumer = Objects.requireNonNull(resultsConsumer); + this.bufferingMutator = new ConditionalTabletsMutatorImpl(context); + this.context = context; + this.executor = Executors.newSingleThreadExecutor(); + + } + + @Override + public Ample.OperationRequirements mutateTablet(KeyExtent extent) { + if (mutatedTablets > BATCH_SIZE) { + if (backgroundProcessing != null) { + // a previous batch of mutations was submitted for processing so wait on it. + try { + backgroundProcessing.get().forEach(resultsConsumer); + } catch (InterruptedException | ExecutionException e) { + throw new IllegalStateException(e); + } + } + + // Spin up processing of the mutations submitted so far in a background thread. Must copy the + // reference for the background thread because a new one is about to be created. + var bufferingMutatorRef = bufferingMutator; + backgroundProcessing = executor.submit(() -> { + var result = bufferingMutatorRef.process(); + bufferingMutatorRef.close(); + return result; + }); + + bufferingMutator = new ConditionalTabletsMutatorImpl(context); + mutatedTablets = 0; + } + mutatedTablets++; + return bufferingMutator.mutateTablet(extent); + } + + @Override + public void close() { + if (backgroundProcessing != null) { Review Comment: I was trying to go through and think if there's any case where we could end up calling `backgroundProcessing.get().forEach(resultsConsumer);` twice (once in mutateTablet and then on close) but I don't think so as long as this object is only ever used from one thread which seems like that is the intention. ########## server/base/src/main/java/org/apache/accumulo/server/metadata/AsyncConditionalTabletsMutatorImpl.java: ########## @@ -0,0 +1,95 @@ +/* + * 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.server.metadata; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.function.BiConsumer; + +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.metadata.schema.Ample; +import org.apache.accumulo.server.ServerContext; + +public class AsyncConditionalTabletsMutatorImpl implements Ample.AsyncConditionalTabletsMutator { + private final BiConsumer<KeyExtent,Ample.ConditionalResult> resultsConsumer; + private final ExecutorService executor; + private Future<Map<KeyExtent,Ample.ConditionalResult>> backgroundProcessing = null; + private ConditionalTabletsMutatorImpl bufferingMutator; + private final ServerContext context; + private long mutatedTablets = 0; + + public static final int BATCH_SIZE = 1000; + + AsyncConditionalTabletsMutatorImpl(ServerContext context, + BiConsumer<KeyExtent,Ample.ConditionalResult> resultsConsumer) { + this.resultsConsumer = Objects.requireNonNull(resultsConsumer); + this.bufferingMutator = new ConditionalTabletsMutatorImpl(context); + this.context = context; + this.executor = Executors.newSingleThreadExecutor(); Review Comment: We could use a thread pool and reuse executors vs having to create a new one each time. We could use a cached pool or a bounded pool if we wanted to limit the number of concurrent fate operations. (I'm not sure if the number of operations is already bound elsewhere). Something else that could be done if using a pool would be to allow executing more than one background thread at once which could speed up processing so we are not blocking and waiting for the previous background future to finish. The downside would be increased complexity by needing to track multiple submitted background tasks instead of 1 and also would mean ordering would be lost as you don't know which thread would finish first but that probably wouldn't matter in a lot of cases. (Ie in this case you don't really care which updates finish first you just want the op id set on all tablets) -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
