yifan-c commented on code in PR #183:
URL:
https://github.com/apache/cassandra-analytics/pull/183#discussion_r2957989943
##########
cassandra-analytics-core/build.gradle:
##########
@@ -217,8 +217,9 @@ tasks.register('testSequential', Test) {
test {
systemProperty "cassandra.analytics.bridges.sstable_format",
System.getProperty("cassandra.analytics.bridges.sstable_format", "big")
minHeapSize = '1024m'
- maxHeapSize = '3072m'
- maxParallelForks = Math.max(Runtime.runtime.availableProcessors() * 2, 8)
+ maxHeapSize = System.getenv('CORE_TEST_MAX_HEAP_SIZE') ?: '3072m'
+ maxParallelForks = System.getenv('CORE_MAX_PARALLEL_FORKS')?.toInteger()
+ ?: Math.max(Runtime.runtime.availableProcessors() * 2,
8)
Review Comment:
Please try out the Sequential test tag first.
##########
cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/reader/AbstractStreamScanner.java:
##########
@@ -443,7 +444,8 @@ public void consume()
Cell<?> cell = cells.next();
// Re: isLive vs. isTombstone - isLive considers TTL so
that if a cell is expiring soon,
// it is handled as tombstone
- if (cell.isLive(timeProvider.referenceEpochInSeconds()))
+ // C* 4.0 Cell.isLive requires int for nowInSec; checked
cast will throw after Y2038
+ if
(cell.isLive(Ints.checkedCast(timeProvider.referenceEpochInSeconds())))
Review Comment:
Since the `referenceEpochInSeconds` value is static, can you check it just
once on time provider initialization? Instead of cast and check in every loop
here. The code here is the hot path.
##########
.circleci/config.yml:
##########
@@ -64,10 +64,13 @@ commands:
JDK_VERSION: "<<parameters.jdk>>"
INTEGRATION_MAX_PARALLEL_FORKS: 1
INTEGRATION_MAX_HEAP_SIZE: "1500M"
+ CORE_MAX_PARALLEL_FORKS: 2
+ CORE_TEST_MAX_HEAP_SIZE: "2048m"
CASSANDRA_USE_JDK11: <<parameters.use_jdk11>>
command: |
+ export GRADLE_OPTS="-Xmx2g -Dorg.gradle.jvmargs=-Xmx2g"
# Run compile/unit tests, skipping integration tests
- ./gradlew --stacktrace clean assemble check -x
cassandra-analytics-integration-tests:test
-Dcassandra.analytics.bridges.sstable_format=<<parameters.sstable_format>>
+ ./gradlew --no-daemon --max-workers=2 --stacktrace clean assemble
check -x cassandra-analytics-integration-tests:test
-Dcassandra.analytics.bridges.sstable_format=<<parameters.sstable_format>>
Review Comment:
Instead of making CI pipeline changes, can you try to tag the relevant tests
with `@Tag("Sequential")`?
`--no-daemon` flag makes sense.
##########
cassandra-analytics-cdc/src/test/java/org/apache/cassandra/cdc/AvroGenericRecordTransformerTest.java:
##########
@@ -0,0 +1,502 @@
+/*
+ * 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
+ *
+ * http://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.cassandra.cdc;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.bridge.CdcBridgeFactory;
+import org.apache.cassandra.cdc.api.KeyspaceTypeKey;
+import org.apache.cassandra.cdc.avro.AvroConstants;
+import org.apache.cassandra.cdc.avro.AvroGenericRecordTransformer;
+import org.apache.cassandra.cdc.avro.AvroSchemas;
+import org.apache.cassandra.cdc.avro.CdcEventUtils;
+import org.apache.cassandra.cdc.avro.CqlToAvroSchemaConverter;
+import org.apache.cassandra.cdc.avro.TestSchemaStore;
+import org.apache.cassandra.cdc.kafka.AvroGenericRecordSerializer;
+import org.apache.cassandra.cdc.msg.CdcEvent;
+import org.apache.cassandra.cdc.test.CdcTestBase;
+import org.apache.cassandra.cdc.test.CdcTester;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+
+import static
org.apache.cassandra.cdc.test.CdcTester.newUniquePartitionDeletion;
+import static org.apache.cassandra.cdc.test.CdcTester.testWith;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests that exercise the full CDC-to-Avro pipeline:
+ * write mutations -> read CDC events from commit logs -> convert to Avro
GenericRecord -> validate.
+ */
+@SuppressWarnings("DataFlowIssue")
+public class AvroGenericRecordTransformerTest extends CdcTestBase
+{
+ private static final int NUM_ROWS = 50;
+
+ private CqlToAvroSchemaConverter getConverter(CassandraVersion version)
+ {
+ CqlToAvroSchemaConverter converter =
CdcBridgeFactory.getCqlToAvroSchemaConverter(version);
+ assertThat(converter).isNotNull();
+ return converter;
+ }
+
+ /**
+ * Build the Avro transformer by converting the CQL table schema (already
registered by CdcTester)
+ * into an Avro schema and registering it in the test schema store.
+ */
+ private AvroGenericRecordTransformer
buildTransformer(CqlToAvroSchemaConverter converter,
+ CqlTable cqlTable)
+ {
+ TestSchemaStore schemaStore = new TestSchemaStore();
+ Schema avroSchema = converter.convert(cqlTable);
+ String namespace = cqlTable.keyspace() + "." + cqlTable.table();
+ schemaStore.registerSchema(namespace, avroSchema);
+
+ Function<KeyspaceTypeKey, CqlField.CqlType> typeLookup = key ->
bridge.parseType(key.type);
+ return new AvroGenericRecordTransformer(schemaStore, typeLookup, "");
+ }
+
+ @ParameterizedTest
+
@MethodSource("org.apache.cassandra.cdc.test.TestVersionSupplier#testVersions")
+ public void testBasicInsertAvroEncoding(CassandraVersion version)
Review Comment:
The file adds a few irrelevant test cases. Improving test coverage is in
general a good thing. So I am fine with those test cases in this file.
##########
cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/db/DbUtils.java:
##########
@@ -33,23 +33,27 @@ private DbUtils()
throw new IllegalStateException(getClass() + " is static utility class
and shall not be instantiated");
}
- public static DeletionTime deletionTime(long markedForDeleteAt, int
localDeletionTime)
+ // C* 4.0 DeletionTime constructor requires int for localDeletionTime;
checked cast will throw after Y2038
+ public static DeletionTime deletionTime(long markedForDeleteAt, long
localDeletionTime)
Review Comment:
The comment looks scary, but the methods (this and the others in this file)
are indeed for test only. Let's add the annotation `@VisibleForTesting` to all
methods. The test should ensure passing the valid value.
##########
cassandra-analytics-cdc-codec/src/main/java/org/apache/cassandra/cdc/avro/CdcEventUtils.java:
##########
@@ -281,14 +281,14 @@ public static GenericData.Record getTTLAvro(CdcEvent
event, Schema ttlSchema)
return ttlRecord;
}
- public static Map<String, Integer> getTTL(CdcEvent event)
+ public static Map<String, Long> getTTL(CdcEvent event)
{
CdcEvent.TimeToLive ttl = event.getTtl();
if (ttl == null)
{
return null;
}
- return mapOf(AvroConstants.TTL_KEY, ttl.ttlInSec,
AvroConstants.DELETED_AT_KEY, ttl.expirationTimeInSec);
+ return mapOf(AvroConstants.TTL_KEY, (long) ttl.ttlInSec,
AvroConstants.DELETED_AT_KEY, ttl.expirationTimeInSec);
Review Comment:
Why converting TTL to `long`, meanwhile the schema has it as `int`?
The change is probably unnecessary.
##########
CHANGES.txt:
##########
@@ -1,5 +1,6 @@
0.4.0
-----
+ * Fix year 2038 problem using long for absolute times and support C* 5.0
extended localDeletionTime
Review Comment:
The year 2038 problem sounds cool, but I feel it shades the actual change in
this patch.
I would revise the change log entry to "Support extended deletion time in
CDC for Cassandra 5.0" and update the JIRA title too.
##########
cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/data/complex/CqlMap.java:
##########
@@ -120,15 +120,15 @@ public void addCell(final
org.apache.cassandra.db.rows.Row.Builder rowBuilder,
ColumnMetadata cd,
long timestamp,
int ttl,
- int now,
+ long now,
Object value)
{
for (Map.Entry<?, ?> entry : ((Map<?, ?>) value).entrySet())
{
if (ttl != NO_TTL)
{
- rowBuilder.addCell(BufferCell.expiring(cd, timestamp, ttl,
now, valueType().serialize(entry.getValue()),
-
CellPath.create(keyType().serialize(entry.getKey()))));
+ rowBuilder.addCell(CqlType.expiring(cd, timestamp, ttl, now,
valueType().serialize(entry.getValue()),
+
CellPath.create(keyType().serialize(entry.getKey()))));
Review Comment:
can be reverted too
##########
cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkWriteComplexTypeTtlTest.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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
+ *
+ * http://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.cassandra.analytics;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.Test;
+
+import org.apache.cassandra.distributed.shared.Uninterruptibles;
+import org.apache.cassandra.sidecar.testing.QualifiedName;
+import org.apache.cassandra.spark.bulkwriter.TTLOption;
+import org.apache.cassandra.spark.bulkwriter.WriterOptions;
+import org.apache.cassandra.testing.ClusterBuilderConfiguration;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import static org.apache.cassandra.testing.TestUtils.DC1_RF3;
+import static org.apache.cassandra.testing.TestUtils.TEST_KEYSPACE;
+import static org.apache.spark.sql.types.DataTypes.IntegerType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+import static org.apache.spark.sql.types.DataTypes.createArrayType;
+import static org.apache.spark.sql.types.DataTypes.createMapType;
+import static org.apache.spark.sql.types.DataTypes.createStructType;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests that bulk writes with TTL work correctly for complex types (list,
set, map, UDT).
+ * This ensures the expiring cell path is invoked correctly for collection and
composite types.
+ * TODO: add tuple ttl test after adding support for writing tuples
+ */
+class BulkWriteComplexTypeTtlTest extends SharedClusterSparkIntegrationTestBase
+{
+ static final int ROW_COUNT = 100;
+
+ static final QualifiedName LIST_TTL_TABLE = new
QualifiedName(TEST_KEYSPACE, "test_list_ttl");
+ static final QualifiedName SET_TTL_TABLE = new
QualifiedName(TEST_KEYSPACE, "test_set_ttl");
+ static final QualifiedName MAP_TTL_TABLE = new
QualifiedName(TEST_KEYSPACE, "test_map_ttl");
+ static final QualifiedName UDT_TTL_TABLE = new
QualifiedName(TEST_KEYSPACE, "test_udt_ttl");
+
+ static final String SIMPLE_UDT_NAME = "simple_udt";
+
+ @Test
+ void testListWithTtl()
+ {
+ SparkSession spark = getOrCreateSparkSession();
+ StructType schema = new StructType()
+ .add("id", IntegerType, false)
+ .add("listdata", createArrayType(IntegerType),
false);
+
+ List<Row> rows = IntStream.range(0, ROW_COUNT)
+ .mapToObj(i -> RowFactory.create(i,
Arrays.asList(i, i + 1)))
+ .collect(Collectors.toList());
+ Dataset<Row> df = spark.createDataFrame(rows, schema);
+
+ bulkWriterDataFrameWriter(df,
LIST_TTL_TABLE).option(WriterOptions.TTL.name(), TTLOption.constant(80))
+ .save();
+
+ Dataset<Row> preExpiry = bulkReaderDataFrame(LIST_TTL_TABLE).load();
+ assertThat(preExpiry.collectAsList()).hasSize(ROW_COUNT);
+
+ Uninterruptibles.sleepUninterruptibly(80, TimeUnit.SECONDS);
+ Dataset<Row> postExpiry = bulkReaderDataFrame(LIST_TTL_TABLE).load();
+ assertThat(postExpiry.collectAsList()).isEmpty();
+ }
+
+ @Test
+ void testSetWithTtl()
+ {
+ SparkSession spark = getOrCreateSparkSession();
+ StructType schema = new StructType()
+ .add("id", IntegerType, false)
+ .add("setdata", createArrayType(StringType),
false);
+
+ List<Row> rows = IntStream.range(0, ROW_COUNT)
+ .mapToObj(i -> RowFactory.create(i,
ImmutableSet.of("item" + i)))
+ .collect(Collectors.toList());
+ Dataset<Row> df = spark.createDataFrame(rows, schema);
+
+ bulkWriterDataFrameWriter(df,
SET_TTL_TABLE).option(WriterOptions.TTL.name(), TTLOption.constant(80))
+ .save();
+
+ Dataset<Row> preExpiry = bulkReaderDataFrame(SET_TTL_TABLE).load();
+ assertThat(preExpiry.collectAsList()).hasSize(ROW_COUNT);
+
+ Uninterruptibles.sleepUninterruptibly(80, TimeUnit.SECONDS);
+ Dataset<Row> postExpiry = bulkReaderDataFrame(SET_TTL_TABLE).load();
+ assertThat(postExpiry.collectAsList()).isEmpty();
+ }
+
+ @Test
+ void testMapWithTtl()
+ {
+ SparkSession spark = getOrCreateSparkSession();
+ StructType schema = new StructType()
+ .add("id", IntegerType, false)
+ .add("mapdata", createMapType(StringType,
IntegerType), false);
+
+ List<Row> rows = IntStream.range(0, ROW_COUNT)
+ .mapToObj(i -> RowFactory.create(i,
ImmutableMap.of("key" + i, i)))
+ .collect(Collectors.toList());
+ Dataset<Row> df = spark.createDataFrame(rows, schema);
+
+ bulkWriterDataFrameWriter(df,
MAP_TTL_TABLE).option(WriterOptions.TTL.name(), TTLOption.constant(80))
+ .save();
+
+ Dataset<Row> preExpiry = bulkReaderDataFrame(MAP_TTL_TABLE).load();
+ assertThat(preExpiry.collectAsList()).hasSize(ROW_COUNT);
+
+ Uninterruptibles.sleepUninterruptibly(80, TimeUnit.SECONDS);
+ Dataset<Row> postExpiry = bulkReaderDataFrame(MAP_TTL_TABLE).load();
+ assertThat(postExpiry.collectAsList()).isEmpty();
+ }
+
+ @Test
+ void testUdtWithTtl()
+ {
+ SparkSession spark = getOrCreateSparkSession();
+ StructType udtType = createStructType(new StructField[]{
+ new StructField("f1", StringType, true, Metadata.empty()),
+ new StructField("f2", IntegerType, true, Metadata.empty())
+ });
+ StructType schema = new StructType()
+ .add("id", IntegerType, false)
+ .add("udtfield", udtType, false);
+
+ List<Row> rows = IntStream.range(0, ROW_COUNT)
+ .mapToObj(i -> RowFactory.create(i,
RowFactory.create("course" + i, i)))
+ .collect(Collectors.toList());
+ Dataset<Row> df = spark.createDataFrame(rows, schema);
+
+ bulkWriterDataFrameWriter(df,
UDT_TTL_TABLE).option(WriterOptions.TTL.name(), TTLOption.constant(80))
+ .save();
+
+ Dataset<Row> preExpiry = bulkReaderDataFrame(UDT_TTL_TABLE).load();
+ assertThat(preExpiry.collectAsList()).hasSize(ROW_COUNT);
+
+ Uninterruptibles.sleepUninterruptibly(80, TimeUnit.SECONDS);
Review Comment:
Please find a different way and avoid sleep the test for 80 seconds. We
cannot have the CI sleep for 80 seconds.
If there is no other way, please delete all the tests in this file. They are
irrelevant to the patch anyway.
##########
cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/reader/CompactionStreamScanner.java:
##########
@@ -117,15 +118,16 @@ protected void handleCellTombstoneInComplex(BigInteger
token, Cell<?> cell)
@Override
UnfilteredPartitionIterator initializePartitions()
{
- int nowInSec = timeProvider.referenceEpochInSeconds();
+ long nowInSec = timeProvider.referenceEpochInSeconds();
Keyspace keyspace = Keyspace.openWithoutSSTables(metadata.keyspace);
ColumnFamilyStore cfStore =
keyspace.getColumnFamilyStore(metadata.name);
controller = new PurgingCompactionController(cfStore,
CompactionParams.TombstoneOption.NONE);
List<ISSTableScanner> scannerList = toCompact.stream()
.map(Scannable::scanner)
.collect(Collectors.toList());
scanners = new AbstractCompactionStrategy.ScannerList(scannerList);
- ci = new CompactionIterator(OperationType.COMPACTION,
scanners.scanners, controller, nowInSec, taskId);
+ // C* 4.0 CompactionIterator requires int for nowInSec; checked cast
will throw after Y2038
+ ci = new CompactionIterator(OperationType.COMPACTION,
scanners.scanners, controller, Ints.checkedCast(nowInSec), taskId);
Review Comment:
again, the checkedCast is probably unnecessary.
##########
cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/data/complex/AbstractCqlList.java:
##########
@@ -105,15 +105,15 @@ public void addCell(final
org.apache.cassandra.db.rows.Row.Builder rowBuilder,
ColumnMetadata cd,
long timestamp,
int ttl,
- int now,
+ long now,
Object value)
{
for (Object o : (List<?>) value)
{
if (ttl != NO_TTL)
{
- rowBuilder.addCell(BufferCell.expiring(cd, timestamp, ttl,
now, type().serialize(o),
-
CellPath.create(ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes()))));
+ rowBuilder.addCell(CqlType.expiring(cd, timestamp, ttl, now,
type().serialize(o),
+
CellPath.create(ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes()))));
}
Review Comment:
I think this change can be reverted once you drop `CqlTypeY2038Test` and
revert the change in `CqlType`
##########
cassandra-four-zero-bridge/src/test/java/org/apache/cassandra/spark/data/CqlTypeY2038Test.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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
+ *
+ * http://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.cassandra.spark.data;
+
+import java.nio.ByteBuffer;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.Test;
+
+import org.apache.cassandra.db.rows.BufferCell;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.reader.SchemaBuilder;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Tests verifying Y2038 boundary behavior for {@link CqlType#tombstone} and
{@link CqlType#expiring}
+ * in Cassandra 4.0.
+ */
+class CqlTypeY2038Test
Review Comment:
I do not think this test is valuable. Analytics/CDC does not _set_ TTL, it
only _reads_ the TTL values from Cassandra. The TTL values are guaranteed to be
valid already; otherwise the cql write fails already. Please delete this file.
##########
cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/data/complex/CqlSet.java:
##########
@@ -104,15 +104,15 @@ public void addCell(final
org.apache.cassandra.db.rows.Row.Builder rowBuilder,
ColumnMetadata cd,
long timestamp,
int ttl,
- int now,
+ long now,
Object value)
{
for (Object o : (Set<?>) value)
{
if (ttl != NO_TTL)
{
- rowBuilder.addCell(BufferCell.expiring(cd, timestamp, ttl,
now, ByteBufferUtil.EMPTY_BYTE_BUFFER,
-
CellPath.create(type().serialize(o))));
+ rowBuilder.addCell(CqlType.expiring(cd, timestamp, ttl, now,
ByteBufferUtil.EMPTY_BYTE_BUFFER,
+
CellPath.create(type().serialize(o))));
Review Comment:
can revert
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]