|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.cassandra.cql3; |
| 20 | + |
| 21 | +import java.util.ArrayList; |
| 22 | +import java.util.List; |
| 23 | +import java.util.Random; |
| 24 | +import java.util.concurrent.ExecutorService; |
| 25 | +import java.util.concurrent.Executors; |
| 26 | +import java.util.concurrent.Future; |
| 27 | +import java.util.concurrent.TimeUnit; |
| 28 | +import java.util.concurrent.atomic.AtomicBoolean; |
| 29 | +import java.util.concurrent.atomic.AtomicInteger; |
| 30 | + |
| 31 | +import org.junit.After; |
| 32 | +import org.junit.Assert; |
| 33 | +import org.junit.Test; |
| 34 | + |
| 35 | +import org.apache.cassandra.config.DatabaseDescriptor; |
| 36 | +import org.apache.cassandra.db.ColumnFamilyStore; |
| 37 | +import org.hamcrest.Matchers; |
| 38 | + |
| 39 | +import static org.junit.Assert.assertEquals; |
| 40 | +import static org.junit.Assert.assertTrue; |
| 41 | + |
| 42 | +public class EarlyOpenCompactionTest extends CQLTester |
| 43 | +{ |
| 44 | + private static final int NUM_PARTITIONS = 1000; |
| 45 | + private static final int NUM_ROWS_PER_PARTITION = 100; |
| 46 | + private static final int VALUE_SIZE = 1000; // ~1KB per row |
| 47 | + private static final int VERIFICATION_THREADS = 4; |
| 48 | + |
| 49 | + private final AtomicBoolean stopVerification = new AtomicBoolean(false); |
| 50 | + private final AtomicInteger verificationErrors = new AtomicInteger(0); |
| 51 | + private final Random random = new Random(); |
| 52 | + private ExecutorService executor; |
| 53 | + |
| 54 | + @After |
| 55 | + public void cleanupAfter() throws Throwable |
| 56 | + { |
| 57 | + stopVerification.set(true); |
| 58 | + if (executor != null) |
| 59 | + { |
| 60 | + executor.shutdownNow(); |
| 61 | + executor.awaitTermination(1, TimeUnit.MINUTES); |
| 62 | + } |
| 63 | + DatabaseDescriptor.setSSTablePreemptiveOpenIntervalInMiB(50); |
| 64 | + } |
| 65 | + |
| 66 | + @Test |
| 67 | + public void testEarlyOpenDuringCompaction() throws Throwable |
| 68 | + { |
| 69 | + // Create a table with a simple schema |
| 70 | + createTable("CREATE TABLE %s (" + |
| 71 | + "pk int, " + |
| 72 | + "ck int, " + |
| 73 | + "data text, " + |
| 74 | + "PRIMARY KEY (pk, ck)" + |
| 75 | + ")"); |
| 76 | + |
| 77 | + ColumnFamilyStore cfs = getCurrentColumnFamilyStore(); |
| 78 | + disableCompaction(); |
| 79 | + |
| 80 | + // Insert data to create multiple SSTables |
| 81 | + System.out.println("Inserting test data..."); |
| 82 | + for (int i = 0; i < NUM_PARTITIONS; i++) |
| 83 | + { |
| 84 | + for (int j = 0; j < NUM_ROWS_PER_PARTITION; j++) |
| 85 | + { |
| 86 | + String value = randomString(VALUE_SIZE); |
| 87 | + execute("INSERT INTO %s (pk, ck, data) VALUES (?, ?, ?)", i, j, value); |
| 88 | + } |
| 89 | + |
| 90 | + // Flush from time to time to get 10 sstables |
| 91 | + if (i > 0 && i % Math.max(1, NUM_PARTITIONS / 10) == 0) |
| 92 | + { |
| 93 | + flush(); |
| 94 | + } |
| 95 | + } |
| 96 | + |
| 97 | + // Final flush to ensure all data is written |
| 98 | + flush(); |
| 99 | + |
| 100 | + // Verify we have multiple SSTables |
| 101 | + int sstableCount = cfs.getLiveSSTables().size(); |
| 102 | + assertTrue("Expected multiple SSTables, got: " + sstableCount, sstableCount > 1); |
| 103 | + |
| 104 | + // Start verification threads |
| 105 | + System.out.println("Starting verification threads..."); |
| 106 | + executor = Executors.newFixedThreadPool(VERIFICATION_THREADS); |
| 107 | + List<Future<?>> futures = new ArrayList<>(); |
| 108 | + |
| 109 | + for (int i = 0; i < VERIFICATION_THREADS; i++) |
| 110 | + { |
| 111 | + futures.add(executor.submit(new VerificationTask())); |
| 112 | + } |
| 113 | + |
| 114 | + // Wait a bit to ensure verification is running |
| 115 | + Thread.sleep(1000); |
| 116 | + |
| 117 | + // Set early open interval to 1MiB to trigger early open during compaction |
| 118 | + System.out.println("Setting early open interval to 1MiB..."); |
| 119 | + DatabaseDescriptor.setSSTablePreemptiveOpenIntervalInMiB(1); |
| 120 | + // Slow down compaction to give the verifier time to fail. |
| 121 | + DatabaseDescriptor.setCompactionThroughputMebibytesPerSec(10); |
| 122 | + |
| 123 | + // Trigger compaction and await its completion |
| 124 | + System.out.println("Starting compaction..."); |
| 125 | + cfs.enableAutoCompaction(true); |
| 126 | + |
| 127 | + // Let verification run for a while during and after compaction |
| 128 | + System.out.println("Verifying data during and after compaction..."); |
| 129 | + Thread.sleep(1000); |
| 130 | + |
| 131 | + // Stop verification |
| 132 | + stopVerification.set(true); |
| 133 | + |
| 134 | + // Wait for verification to complete |
| 135 | + for (Future<?> future : futures) |
| 136 | + { |
| 137 | + try |
| 138 | + { |
| 139 | + future.get(10, TimeUnit.SECONDS); |
| 140 | + } |
| 141 | + catch (Exception e) |
| 142 | + { |
| 143 | + System.err.println("Verification task failed: " + e); |
| 144 | + e.printStackTrace(); |
| 145 | + } |
| 146 | + } |
| 147 | + |
| 148 | + // Verify no errors occurred during verification |
| 149 | + int errors = verificationErrors.get(); |
| 150 | + assertEquals("Found " + errors + " verification errors. Check logs for details.", 0, errors); |
| 151 | + |
| 152 | + System.out.println("Test completed successfully"); |
| 153 | + } |
| 154 | + |
| 155 | + private class VerificationTask implements Runnable |
| 156 | + { |
| 157 | + @Override |
| 158 | + public void run() |
| 159 | + { |
| 160 | + try |
| 161 | + { |
| 162 | + Random localRandom = new Random(Thread.currentThread().getId()); |
| 163 | + |
| 164 | + while (!stopVerification.get() && !Thread.currentThread().isInterrupted()) |
| 165 | + { |
| 166 | + // Randomly choose between point query and partition range query |
| 167 | + if (localRandom.nextBoolean()) |
| 168 | + { |
| 169 | + // Point query |
| 170 | + int pk = localRandom.nextInt(NUM_PARTITIONS * 110 / 100); // 10% chance outside |
| 171 | + int ck = localRandom.nextInt(NUM_ROWS_PER_PARTITION * 110 / 100); // 10% chance outside |
| 172 | + |
| 173 | + try |
| 174 | + { |
| 175 | + Assert.assertEquals(pk < NUM_PARTITIONS && ck < NUM_ROWS_PER_PARTITION ? 1 : 0, |
| 176 | + execute("SELECT data FROM %s WHERE pk = ? AND ck = ?", pk, ck).size()); |
| 177 | + } |
| 178 | + catch (Throwable t) |
| 179 | + { |
| 180 | + verificationErrors.incrementAndGet(); |
| 181 | + System.err.println("Point query failed for pk=" + pk + ", ck=" + ck + ": " + t); |
| 182 | + t.printStackTrace(); |
| 183 | + } |
| 184 | + } |
| 185 | + else |
| 186 | + { |
| 187 | + // Partition range query |
| 188 | + int pk = localRandom.nextInt(NUM_PARTITIONS); |
| 189 | + |
| 190 | + try |
| 191 | + { |
| 192 | + Assert.assertThat(execute("SELECT data FROM %s WHERE token(pk) <= token(?) AND token(pk) >= token(?)", pk, pk).size(), |
| 193 | + Matchers.greaterThanOrEqualTo(NUM_ROWS_PER_PARTITION)); |
| 194 | + } |
| 195 | + catch (Throwable t) |
| 196 | + { |
| 197 | + verificationErrors.incrementAndGet(); |
| 198 | + System.err.println("Range query failed for pk in (" + pk + ", " + (pk + 1) + ", " + (pk + 2) + "): " + t); |
| 199 | + t.printStackTrace(); |
| 200 | + } |
| 201 | + } |
| 202 | + |
| 203 | + // Add a small delay to prevent overwhelming the system |
| 204 | + Thread.yield(); |
| 205 | + } |
| 206 | + } |
| 207 | + catch (Throwable t) |
| 208 | + { |
| 209 | + verificationErrors.incrementAndGet(); |
| 210 | + System.err.println("Verification task failed: " + t); |
| 211 | + t.printStackTrace(); |
| 212 | + } |
| 213 | + } |
| 214 | + } |
| 215 | + |
| 216 | + private String randomString(int length) |
| 217 | + { |
| 218 | + StringBuilder sb = new StringBuilder(length); |
| 219 | + for (int i = 0; i < length; i++) |
| 220 | + { |
| 221 | + sb.append((char)('a' + random.nextInt(26))); |
| 222 | + } |
| 223 | + return sb.toString(); |
| 224 | + } |
| 225 | +} |
0 commit comments