|
| 1 | +/* |
| 2 | + * Copyright DataStax, Inc. |
| 3 | + * |
| 4 | + * Licensed under the Apache License, Version 2.0 (the "License"); |
| 5 | + * you may not use this file except in compliance with the License. |
| 6 | + * You may obtain a copy of the License at |
| 7 | + * |
| 8 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 9 | + * |
| 10 | + * Unless required by applicable law or agreed to in writing, software |
| 11 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 12 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 13 | + * See the License for the specific language governing permissions and |
| 14 | + * limitations under the License. |
| 15 | + */ |
| 16 | + |
| 17 | +package org.apache.cassandra.io.sstable; |
| 18 | + |
| 19 | +import java.util.Set; |
| 20 | + |
| 21 | +import org.junit.After; |
| 22 | +import org.junit.BeforeClass; |
| 23 | +import org.junit.Test; |
| 24 | +import org.junit.runner.RunWith; |
| 25 | + |
| 26 | +import org.apache.cassandra.SchemaLoader; |
| 27 | +import org.apache.cassandra.config.CassandraRelevantProperties; |
| 28 | +import org.apache.cassandra.db.ColumnFamilyStore; |
| 29 | +import org.apache.cassandra.db.DecoratedKey; |
| 30 | +import org.apache.cassandra.db.Directories; |
| 31 | +import org.apache.cassandra.db.Keyspace; |
| 32 | +import org.apache.cassandra.db.RowUpdateBuilder; |
| 33 | +import org.apache.cassandra.db.lifecycle.Tracker; |
| 34 | +import org.apache.cassandra.io.sstable.format.SSTableReader; |
| 35 | +import org.apache.cassandra.io.sstable.metadata.StatsMetadata; |
| 36 | +import org.apache.cassandra.schema.KeyspaceParams; |
| 37 | +import org.apache.cassandra.schema.TableMetadataRef; |
| 38 | +import org.apache.cassandra.utils.ByteBufferUtil; |
| 39 | +import org.jboss.byteman.contrib.bmunit.BMRule; |
| 40 | +import org.jboss.byteman.contrib.bmunit.BMRules; |
| 41 | +import org.jboss.byteman.contrib.bmunit.BMUnitRunner; |
| 42 | + |
| 43 | +import static org.junit.Assert.assertEquals; |
| 44 | +import static org.junit.Assert.assertThrows; |
| 45 | +import static org.junit.Assert.assertTrue; |
| 46 | + |
| 47 | +@RunWith(BMUnitRunner.class) |
| 48 | +public class StorageHandlerTest |
| 49 | +{ |
| 50 | + private static final String KS = "StorageHandlerTest"; |
| 51 | + private static final String TABLE = "testTable"; |
| 52 | + |
| 53 | + private static ColumnFamilyStore store; |
| 54 | + |
| 55 | + private static volatile OnOpeningWrittenSSTableFailure onOpeningWrittenSSTableFailureInterceptor; |
| 56 | + |
| 57 | + @BeforeClass |
| 58 | + public static void setup() |
| 59 | + { |
| 60 | + CassandraRelevantProperties.REMOTE_STORAGE_HANDLER_FACTORY.setString(TestStorageHandlerFactory.class.getName()); |
| 61 | + |
| 62 | + SchemaLoader.prepareServer(); |
| 63 | + SchemaLoader.createKeyspace(KS, |
| 64 | + KeyspaceParams.simple(1), |
| 65 | + SchemaLoader.standardCFMD(KS, TABLE)); |
| 66 | + |
| 67 | + store = Keyspace.open(KS).getColumnFamilyStore(TABLE); |
| 68 | + } |
| 69 | + |
| 70 | + @After |
| 71 | + public void afterTest() |
| 72 | + { |
| 73 | + store.truncateBlocking(); |
| 74 | + } |
| 75 | + |
| 76 | + private void addDataAndFlush(ColumnFamilyStore cfs, int numKeys) |
| 77 | + { |
| 78 | + for (int i = 0; i < numKeys; i ++) { |
| 79 | + new RowUpdateBuilder(cfs.metadata(), i, String.valueOf(i)) |
| 80 | + .clustering("0") |
| 81 | + .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) |
| 82 | + .build() |
| 83 | + .applyUnsafe(); |
| 84 | + } |
| 85 | + cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.USER_FORCED); |
| 86 | + } |
| 87 | + |
| 88 | + @Test |
| 89 | + @BMRules(rules = { |
| 90 | + @BMRule(name = "Fail opening reader", |
| 91 | + targetClass = "org.apache.cassandra.io.sstable.format.SSTableWriter$TransactionalProxy", |
| 92 | + targetMethod = "openResultInternal", |
| 93 | + targetLocation = "AT INVOKE openFinal", |
| 94 | + action = "throw new RuntimeException(\"Problem reading\")") |
| 95 | + } |
| 96 | + ) |
| 97 | + public void testOnOpeningWrittenSSTableFailure() |
| 98 | + { |
| 99 | + int numKeys = 10; |
| 100 | + |
| 101 | + onOpeningWrittenSSTableFailureInterceptor = (reason, descriptor, components, compressedSize, uncompressedSize, stats, firstKey, lastKey, estimatedKeys, throwable) -> { |
| 102 | + assertEquals(SSTableReader.OpenReason.NORMAL, reason); |
| 103 | + assertEquals(TABLE, descriptor.cfname); |
| 104 | + assertEquals("Problem reading", throwable.getMessage()); |
| 105 | + assertTrue(compressedSize > 0); |
| 106 | + assertTrue(uncompressedSize > 0); |
| 107 | + // Note that the update uses the iteration count sa timestamp. |
| 108 | + assertEquals(0, stats.minTimestamp); |
| 109 | + assertEquals(numKeys - 1, stats.maxTimestamp); |
| 110 | + // Note that tests uses the byter-order partitioner. |
| 111 | + assertEquals("0", ByteBufferUtil.string(firstKey.getKey())); |
| 112 | + assertEquals("9", ByteBufferUtil.string(lastKey.getKey())); |
| 113 | + }; |
| 114 | + |
| 115 | + assertThrows("Problem reading", RuntimeException.class, () -> addDataAndFlush(store, numKeys)); |
| 116 | + } |
| 117 | + |
| 118 | + private static class TestStorageHandler extends DefaultStorageHandler |
| 119 | + { |
| 120 | + private TestStorageHandler(SSTable.Owner owner, TableMetadataRef metadata, Directories directories, Tracker dataTracker) |
| 121 | + { |
| 122 | + super(owner, metadata, directories, dataTracker); |
| 123 | + } |
| 124 | + |
| 125 | + @Override |
| 126 | + public SSTableReader onOpeningWrittenSSTableFailure(SSTableReader.OpenReason reason, Descriptor descriptor, Set<Component> components, long compressedSize, long uncompressedSize, StatsMetadata stats, DecoratedKey firstKey, DecoratedKey lastKey, long estimatedKeys, Throwable throwable) |
| 127 | + { |
| 128 | + if (onOpeningWrittenSSTableFailureInterceptor != null) |
| 129 | + { |
| 130 | + try |
| 131 | + { |
| 132 | + onOpeningWrittenSSTableFailureInterceptor.call(reason, descriptor, components, compressedSize, uncompressedSize, stats, firstKey, lastKey, estimatedKeys, throwable); |
| 133 | + } |
| 134 | + catch (Exception e) |
| 135 | + { |
| 136 | + throw new RuntimeException(e); |
| 137 | + } |
| 138 | + } |
| 139 | + return super.onOpeningWrittenSSTableFailure(reason, descriptor, components, compressedSize, uncompressedSize, stats, firstKey, lastKey, estimatedKeys, throwable); |
| 140 | + } |
| 141 | + } |
| 142 | + |
| 143 | + @FunctionalInterface |
| 144 | + private interface OnOpeningWrittenSSTableFailure |
| 145 | + { |
| 146 | + void call(SSTableReader.OpenReason reason, Descriptor descriptor, Set<Component> components, long compressedSize, long uncompressedSize, StatsMetadata stats, DecoratedKey firstKey, DecoratedKey lastKey, long estimatedKeys, Throwable throwable) throws Exception; |
| 147 | + } |
| 148 | + |
| 149 | + public static class TestStorageHandlerFactory implements StorageHandlerFactory |
| 150 | + { |
| 151 | + @Override |
| 152 | + public StorageHandler create(SSTable.Owner owner, TableMetadataRef metadata, Directories directories, Tracker dataTracker) |
| 153 | + { |
| 154 | + return new TestStorageHandler(owner, metadata, directories, dataTracker); |
| 155 | + } |
| 156 | + } |
| 157 | +} |
0 commit comments