keith-turner commented on code in PR #4317:
URL: https://github.com/apache/accumulo/pull/4317#discussion_r1518191977


##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/FindSplits.java:
##########
@@ -73,15 +95,59 @@ public Repo<Manager> call(FateId fateId, Manager manager) 
throws Exception {
     }
 
     if (splits.isEmpty()) {
-      log.info("Tablet {} needs to split, but no split points could be found.",
-          tabletMetadata.getExtent());
-      // ELASTICITY_TODO record the fact that tablet is un-splittable in 
metadata table in a new
-      // column. Record the config used to reach this decision and a hash of 
the file. The tablet
-      // mgmt iterator can inspect this column and only try to split the 
tablet when something has
-      // changed.
+      Consumer<ConditionalResult> resultConsumer = result -> {
+        if (result.getStatus() == Status.REJECTED) {
+          log.debug("{} unsplittable metadata update for {} was rejected ", 
fateId,
+              result.getExtent());
+        }
+      };
+
+      try (var tabletsMutator = 
ample.conditionallyMutateTablets(resultConsumer)) {
+        // No split points were found, so we need to check if the tablet still
+        // needs to be split but is unsplittable, or if a split is not needed
+
+        // Case 1: If a split is needed then set the unsplittable marker as no 
split
+        // points could be found so that we don't keep trying again until the
+        // split metadata is changed
+        if (SplitUtils.needsSplit(manager.getContext(), tabletMetadata)) {
+          log.info("Tablet {} needs to split, but no split points could be 
found.",
+              tabletMetadata.getExtent());
+          var unSplittableMeta = computedUnsplittable
+              .orElseGet(() -> SplitUtils.toUnSplittable(manager.getContext(), 
tabletMetadata));
+
+          // With the current design we don't need to require the files to be 
the same
+          // for correctness as the TabletManagementIterator will detect the 
difference
+          // when computing the hash and retry a new split operation if there 
is not a match.
+          // But if we already know there's a change now, it would be more 
efficient to fail and
+          // retry the current fate op vs completing and having the iterator 
submit a new one.
+          log.debug("Setting unsplittable metadata on tablet {}. hashCode: {}",
+              tabletMetadata.getExtent(), unSplittableMeta);
+          var mutator = 
tabletsMutator.mutateTablet(extent).requireAbsentOperation()
+              .requireSame(tabletMetadata, 
FILES).setUnSplittable(unSplittableMeta);
+          mutator.submit(tm -> unSplittableMeta.equals(tm.getUnSplittable()));
+
+          // Case 2: If the unsplittable marker has already been previously 
set, but we do not need
+          // to split then clear the marker. This could happen in some 
scenarios such as
+          // a compaction that shrinks a previously unsplittable tablet below 
the threshold
+          // or if the threshold has been raised higher because the tablet 
management iterator
+          // will try and split any time the computed metadata changes.
+        } else if (tabletMetadata.getUnSplittable() != null) {
+          log.info("Tablet {} no longer needs to split, deleting unsplittable 
marker.",
+              tabletMetadata.getExtent());
+          var mutator = 
tabletsMutator.mutateTablet(extent).requireAbsentOperation()
+              .requireSame(tabletMetadata, FILES).deleteUnSplittable();
+          mutator.submit(tm -> tm.getUnSplittable() == null);
+          // Case 3: The table config and/or set of files changed since the 
tablet mgmt iterator
+          // examined this tablet.
+        } else {
+          log.info("Tablet {} no longer needs to split, ignoring it.", 
tabletMetadata.getExtent());

Review Comment:
   I would lower the log level on this, unless there is a bug with splitting 
its not useful.
   
   
   ```suggestion
             log.debug("Tablet {} no longer needs to split, ignoring it.", 
tabletMetadata.getExtent());
   ```



##########
server/manager/src/test/java/org/apache/accumulo/manager/tableOps/split/UpdateTabletsTest.java:
##########
@@ -261,6 +261,7 @@ public void testManyColumns() throws Exception {
     
EasyMock.expect(tabletMeta.getHostingRequested()).andReturn(true).atLeastOnce();
     
EasyMock.expect(tabletMeta.getSuspend()).andReturn(suspendingTServer).atLeastOnce();
     
EasyMock.expect(tabletMeta.getLast()).andReturn(lastLocation).atLeastOnce();
+    
EasyMock.expect(tabletMeta.getUnSplittable()).andReturn(null).atLeastOnce();

Review Comment:
   Would be good to test the case where its present to ensure its deleted.
   
   ```suggestion
       UnSplittableMetadata usm = 
UnSplittableMetadata.toUnSplittable(origExtent, 1000, 1001, 1002, 
tabletFiles.keySet());
       
EasyMock.expect(tabletMeta.getUnSplittable()).andReturn(usm).atLeastOnce();
   ```
   
   Then later in the code where it checks what methods were called on the 
mutator could do the following.
   
   ```java
   
EasyMock.expect(tablet3Mutator.deleteUnSplittable()).andReturn(tablet3Mutator);
   ```



##########
test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java:
##########
@@ -242,6 +257,148 @@ public void automaticSplitLater() throws Exception {
     }
   }
 
+  @Test
+  @Timeout(60)
+  public void testUnsplittableColumn() throws Exception {
+    log.info("Unsplittable Column Test");
+    try (AccumuloClient client = 
Accumulo.newClient().from(getClientProperties()).build()) {
+      // make a table and lower the configuration properties
+      // @formatter:off
+      var maxEndRow = 100;
+      Map<String,String> props = Map.of(
+          Property.TABLE_SPLIT_THRESHOLD.getKey(), "1K",
+          Property.TABLE_FILE_COMPRESSION_TYPE.getKey(), "none",
+          Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey(), "64",
+          Property.TABLE_MAX_END_ROW_SIZE.getKey(), "" + maxEndRow,
+          Property.TABLE_MAJC_RATIO.getKey(), "9999"
+      );
+      // @formatter:on
+
+      final String tableName = getUniqueNames(1)[0];
+      client.tableOperations().create(tableName, new 
NewTableConfiguration().setProperties(props));
+
+      // Create a key for a table entry that is longer than the allowed size 
for an
+      // end row and fill this key with all m's except the last spot
+      byte[] data = new byte[maxEndRow + 1];
+      Arrays.fill(data, 0, data.length - 2, (byte) 'm');
+
+      final int numOfMutations = 20;
+      try (BatchWriter batchWriter = client.createBatchWriter(tableName)) {
+        // Make the last place in the key different for every entry added to 
the table
+        for (int i = 0; i < numOfMutations; i++) {
+          data[data.length - 1] = (byte) i;
+          Mutation m = new Mutation(data);
+          m.put("cf", "cq", "value");
+          batchWriter.addMutation(m);
+        }
+      }
+      // Flush the BatchWriter and table
+      client.tableOperations().flush(tableName, null, null, true);
+
+      // Wait for the tablets to be marked as unsplittable due to the system 
split running
+      TableId tableId = 
TableId.of(client.tableOperations().tableIdMap().get(tableName));
+      Wait.waitFor(() -> getServerContext().getAmple()
+          .readTablet(new KeyExtent(tableId, null, null)).getUnSplittable() != 
null,
+          Wait.MAX_WAIT_MILLIS, 100);
+
+      // Verify that the unsplittable column is read correctly
+      TabletMetadata tm =
+          getServerContext().getAmple().readTablet(new KeyExtent(tableId, 
null, null));
+      assertEquals(tm.getUnSplittable(), 
SplitUtils.toUnSplittable(getServerContext(), tm));
+
+      // Make sure no splits occurred in the table
+      assertTrue(client.tableOperations().listSplits(tableName).isEmpty());
+

Review Comment:
   Would be nice to squeeze the following in here.
   
    * record the current unsplittable value
    * set Property.TABLE_MAX_END_ROW_SIZE to 101
    * Wait for the unsplittable value to change (assuming the tablet still 
needs to split but can not)
    * set Property.TABLE_MAX_END_ROW_SIZE to 500 and continue with rest of test



##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/UnSplittableMetadata.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.core.metadata.schema;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Base64;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+
+import com.google.common.base.Preconditions;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.Hashing;
+
+public class UnSplittableMetadata {
+
+  private final HashCode hashOfSplitParameters;
+
+  private UnSplittableMetadata(KeyExtent keyExtent, long splitThreshold, long 
maxEndRowSize,
+      int maxFilesToOpen, Set<StoredTabletFile> files) {
+    this(calculateSplitParamsHash(keyExtent, splitThreshold, maxEndRowSize, 
maxFilesToOpen, files));
+  }
+
+  private UnSplittableMetadata(HashCode hashOfSplitParameters) {
+    this.hashOfSplitParameters = Objects.requireNonNull(hashOfSplitParameters);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    UnSplittableMetadata that = (UnSplittableMetadata) o;
+    return Objects.equals(hashOfSplitParameters, that.hashOfSplitParameters);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(hashOfSplitParameters);
+  }
+
+  @Override
+  public String toString() {
+    return toBase64();
+  }
+
+  public String toBase64() {
+    return Base64.getEncoder().encodeToString(hashOfSplitParameters.asBytes());
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  private static HashCode calculateSplitParamsHash(KeyExtent keyExtent, long 
splitThreshold,
+      long maxEndRowSize, int maxFilesToOpen, Set<StoredTabletFile> files) {
+    Preconditions.checkArgument(splitThreshold > 0, "splitThreshold must be 
greater than 0");
+    Preconditions.checkArgument(maxEndRowSize > 0, "maxEndRowSize must be 
greater than 0");
+    Preconditions.checkArgument(maxFilesToOpen > 0, "maxFilesToOpen must be 
greater than 0");
+
+    // Use static call to murmur3_128() so the seed is always the same
+    // Hashing.goodFastHash will seed with the current time, and we need the 
seed to be
+    // the same across restarts and instances
+    var hasher = Hashing.murmur3_128().newHasher();
+    
hasher.putBytes(serializeKeyExtent(keyExtent)).putLong(splitThreshold).putLong(maxEndRowSize)
+        .putInt(maxFilesToOpen);
+    files.stream().map(StoredTabletFile::getNormalizedPathStr).sorted()

Review Comment:
   Would the following also include the range associated with the file?  If so 
that may be good to include.
   
   ```suggestion
       files.stream().map(StoredTabletFile::getMetadata).sorted()
   ```
   
   If this is changed, then could update the unit test to have some files with 
ranges.



-- 
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-unsubscr...@accumulo.apache.org

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

Reply via email to