Skip to content

Commit

Permalink
Merge branch 'cassandra-3.11' into trunk
Browse files Browse the repository at this point in the history
  • Loading branch information
pauloricardomg committed Jun 6, 2018
2 parents 843a5fd + 02e9ddf commit d3b6a67
Show file tree
Hide file tree
Showing 9 changed files with 195 additions and 236 deletions.
18 changes: 9 additions & 9 deletions src/java/org/apache/cassandra/db/SystemKeyspace.java
Original file line number Diff line number Diff line change
Expand Up @@ -113,15 +113,15 @@ private SystemKeyspace()

public static final TableMetadata Batches =
parse(BATCHES,
"batches awaiting replay",
"CREATE TABLE %s ("
+ "id timeuuid,"
+ "mutations list<blob>,"
+ "version int,"
+ "PRIMARY KEY ((id)))")
.partitioner(new LocalPartitioner(TimeUUIDType.instance))
.compaction(CompactionParams.scts(singletonMap("min_threshold", "2")))
.build();
"batches awaiting replay",
"CREATE TABLE %s ("
+ "id timeuuid,"
+ "mutations list<blob>,"
+ "version int,"
+ "PRIMARY KEY ((id)))")
.partitioner(new LocalPartitioner(TimeUUIDType.instance))
.compaction(CompactionParams.stcs(singletonMap("min_threshold", "2")))
.build();

private static final TableMetadata Paxos =
parse(PAXOS,
Expand Down
2 changes: 1 addition & 1 deletion src/java/org/apache/cassandra/schema/CompactionParams.java
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ public static CompactionParams create(Class<? extends AbstractCompactionStrategy
return new CompactionParams(klass, allOptions, isEnabled, tombstoneOption);
}

public static CompactionParams scts(Map<String, String> options)
public static CompactionParams stcs(Map<String, String> options)
{
return create(SizeTieredCompactionStrategy.class, options);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ public static void defineSchema() throws ConfigurationException
SchemaLoader.createKeyspace(KEYSPACE1,
KeyspaceParams.simple(1),
SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD)
.compaction(CompactionParams.scts(compactionOptions)));
.compaction(CompactionParams.stcs(compactionOptions)));
}

@Before
Expand Down
2 changes: 1 addition & 1 deletion test/unit/org/apache/cassandra/SchemaLoader.java
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ public static void schemaDefinition(String testName) throws ConfigurationExcepti
KeyspaceParams.simple(1),
Tables.of(
// Column Families
standardCFMD(ks1, "Standard1").compaction(CompactionParams.scts(compactionOptions)).build(),
standardCFMD(ks1, "Standard1").compaction(CompactionParams.stcs(compactionOptions)).build(),
standardCFMD(ks1, "Standard2").build(),
standardCFMD(ks1, "Standard3").build(),
standardCFMD(ks1, "Standard4").build(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ public static void loadData() throws ConfigurationException
SchemaLoader.standardCFMD(KEYSPACE1, LCS_TABLE)
.compaction(CompactionParams.lcs(Collections.emptyMap())),
SchemaLoader.standardCFMD(KEYSPACE1, STCS_TABLE)
.compaction(CompactionParams.scts(Collections.emptyMap())),
.compaction(CompactionParams.stcs(Collections.emptyMap())),
SchemaLoader.standardCFMD(KEYSPACE1, DTCS_TABLE)
.compaction(CompactionParams.create(DateTieredCompactionStrategy.class, Collections.emptyMap())),
SchemaLoader.standardCFMD(KEYSPACE1, TWCS_TABLE)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.After;
import org.junit.Ignore;
import org.junit.Test;

import org.apache.cassandra.dht.Murmur3Partitioner;
Expand Down Expand Up @@ -64,6 +63,7 @@
import org.apache.cassandra.UpdateBuilder;
import org.apache.cassandra.utils.concurrent.Transactional;

import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
import static org.apache.cassandra.service.ActiveRepairService.UNREPAIRED_SSTABLE;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertEquals;
Expand All @@ -72,8 +72,6 @@
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;

import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;


public class AntiCompactionTest
{
Expand Down Expand Up @@ -176,7 +174,6 @@ public void antiCompactOnePendingRepair() throws Exception
antiCompactOne(UNREPAIRED_SSTABLE, UUIDGen.getTimeUUID());
}

@Ignore
@Test
public void antiCompactionSizeTest() throws InterruptedException, IOException
{
Expand All @@ -186,12 +183,14 @@ public void antiCompactionSizeTest() throws InterruptedException, IOException
SSTableReader s = writeFile(cfs, 1000);
cfs.addSSTable(s);
Range<Token> range = new Range<Token>(new BytesToken(ByteBufferUtil.bytes(0)), new BytesToken(ByteBufferUtil.bytes(500)));
List<Range<Token>> ranges = Arrays.asList(range);
Collection<SSTableReader> sstables = cfs.getLiveSSTables();
UUID parentRepairSession = UUID.randomUUID();
registerParentRepairSession(parentRepairSession, ranges, UNREPAIRED_SSTABLE, UUIDGen.getTimeUUID());
try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
Refs<SSTableReader> refs = Refs.ref(sstables))
{
CompactionManager.instance.performAnticompaction(cfs, Arrays.asList(range), refs, txn, 12345, NO_PENDING_REPAIR, parentRepairSession);
CompactionManager.instance.performAnticompaction(cfs, ranges, refs, txn, 12345, NO_PENDING_REPAIR, parentRepairSession);
}
long sum = 0;
long rows = 0;
Expand Down Expand Up @@ -241,18 +240,7 @@ public void generateSStable(ColumnFamilyStore store, String Suffix)
}

@Test
public void antiCompactTenSTC() throws InterruptedException, IOException
{
antiCompactTen("SizeTieredCompactionStrategy");
}

@Test
public void antiCompactTenLC() throws InterruptedException, IOException
{
antiCompactTen("LeveledCompactionStrategy");
}

public void antiCompactTen(String compactionStrategy) throws InterruptedException, IOException
public void antiCompactTen() throws InterruptedException, IOException
{
Keyspace keyspace = Keyspace.open(KEYSPACE1);
ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ public static void beforeClass()
SchemaLoader.createKeyspace(KS_PREFIX,
KeyspaceParams.simple(1),
SchemaLoader.standardCFMD(KS_PREFIX, TABLE_PREFIX)
.compaction(CompactionParams.scts(Collections.emptyMap())));
.compaction(CompactionParams.stcs(Collections.emptyMap())));
}

@AfterClass
Expand Down

0 comments on commit d3b6a67

Please sign in to comment.