cshannon commented on code in PR #4317: URL: https://github.com/apache/accumulo/pull/4317#discussion_r1518076210
########## test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java: ########## @@ -242,6 +257,138 @@ 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()); + + // Bump max end row size and verify split occurs and unsplittable column is cleaned up + client.tableOperations().setProperty(tableName, Property.TABLE_MAX_END_ROW_SIZE.getKey(), + "500"); + + // Wait for splits to occur + assertTrue(client.tableOperations().listSplits(tableName).isEmpty()); + Wait.waitFor(() -> !client.tableOperations().listSplits(tableName).isEmpty(), + Wait.MAX_WAIT_MILLIS, 100); + + // Verify all tablets have no unsplittable metadata column + Wait.waitFor(() -> { + try (var tabletsMetadata = + getServerContext().getAmple().readTablets().forTable(tableId).build()) { + return tabletsMetadata.stream() + .allMatch(tabletMetadata -> tabletMetadata.getUnSplittable() == null); + } + }, Wait.MAX_WAIT_MILLIS, 100); + } + } + + // Test the unsplittable column is cleaned up if a previously marked unsplittable tablet + // no longer needs to be split + @Test + @Timeout(60) + public void testUnsplittableCleanup() throws Exception { + log.info("Unsplittable Column Cleanup"); + 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); + } Review Comment: @keith-turner - the latest commit efe30445ad1e69151bdac4e93ac57f05d05d8e4e updates this test by writing the same row (that is not too large to split) as suggested and writes enough data so we end up splitting the default tablet and then we end up with a tablet that is unsplittable due to this condition. The test then still bumps the threshold and verifies all the tablets get cleared of the marker and no more splits happen. See what you think..we could also keep the old test and add this as a 3rd test too. -- 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