You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/11/25 13:50:09 UTC

[GitHub] [flink-connector-cassandra] echauchot opened a new pull request, #3: [FLINK-26822] Add Cassandra Source

echauchot opened a new pull request, #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3

   This PR is the move of https://github.com/apache/flink/pull/21073 to the cassandra dedicated repo


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1055351423


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;

Review Comment:
   :+1: yes, thanks for catching it



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1055410141


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Sets;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            // needs to be mutable

Review Comment:
   I just figured out that the set of ringRanges is passed by reference from the `CassandraSplit` to the `CassandraSplitState`. So when the mutable `CassandraSplitState` updates its `ringRanges` it modifies `CassandraSplit` ones in reality. `CassandraSplit` is supposed to be immutable. I'll make it indeed immutable



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101578673


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorState.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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Objects;
+import java.util.Queue;
+
+/** Sate for {@link CassandraSplitEnumerator} to track the splits yet to assign. */
+public class CassandraEnumeratorState {
+    private final Queue<CassandraSplit> unassignedSplits;
+
+    public CassandraEnumeratorState() {
+        this.unassignedSplits = new ArrayDeque<>();
+    }
+
+    public CassandraEnumeratorState(Queue<CassandraSplit> unassignedSplits) {
+        this.unassignedSplits = unassignedSplits;

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
zentol commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1385506324

   This was a first pass; haven't looked at the tests yet.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1143203775


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This class prepares the generation of {@link CassandraSplit}s based on Cassandra cluster
+ * partitioner and cluster statistics. It estimates the total size of the table using Cassandra
+ * system table system.size_estimates.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    @VisibleForTesting public Long minSplitMemorySize = 10_000_000L; // 10 MB
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Prepare the {@param CassandraEnumeratorState} for lazy generation of {@link CassandraSplit}s:
+     * calculate {@code numSplitsToGenerate} based on estimated target table size and provided
+     * {@code maxSplitMemorySize} and calculate {@code increment} which is the size of a split in
+     * tokens.
+     */
+    public CassandraEnumeratorState prepareSplits() {
+        final long numSplitsToGenerate = decideOnNumSplits();
+        final BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplitsToGenerate)));
+        final BigInteger startToken = partitioner.minToken;
+        return new CassandraEnumeratorState(
+                numSplitsToGenerate,
+                increment,
+                startToken,
+                partitioner.maxToken,
+                new ArrayDeque<>());
+    }
+
+    /**
+     * Determine {@code numSplits} based on the estimation of the target table size and user defined
+     * {@code maxSplitMemorySize}. Provide fallbacks when table size is unavailable, too few or too
+     * many splits are calculated.
+     */
+    private long decideOnNumSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            checkState(
+                    maxSplitMemorySize >= minSplitMemorySize,
+                    "Defined maxSplitMemorySize (%s) is below minimum (%s)",
+                    maxSplitMemorySize,
+                    minSplitMemorySize);
+            final long estimateTableSize = estimateTableSize();
+            if (estimateTableSize == 0) { // size estimates unavailable
+                LOG.info(
+                        "Cassandra size estimates are not available for {}.{} table. Creating as many splits as parallelism ({})",
+                        keyspace,
+                        table,
+                        parallelism);
+                numSplits = parallelism;
+            } else {
+                LOG.debug(
+                        "Estimated size for {}.{} table is {} bytes",
+                        keyspace,
+                        table,
+                        estimateTableSize);
+                numSplits =
+                        estimateTableSize / maxSplitMemorySize == 0
+                                ? parallelism // estimateTableSize under sizes maxSplitMemorySize
+                                // creating as many splits as parallelism

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1086438935


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraRow.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+
+import com.datastax.driver.core.ExecutionInfo;
+import com.datastax.driver.core.Row;
+
+/**
+ * Wrapper for Cassandra {@link Row} that stores associated {@link RingRange} to be able to update
+ * split states. It also stores {@link ExecutionInfo} Cassandra statistics about the query execution
+ * that produced this row.
+ */
+public class CassandraRow {
+
+    private final Row row;
+    private final RingRange associatedRingRange;
+    private final ExecutionInfo executionInfo;

Review Comment:
   Yes otherwise the mapper fails with an exception when we try to map



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089211568


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits, enumeratorContext.currentParallelism());
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitioner = clusterMetadata.getPartitioner();
+        final SplitsGenerator splitsGenerator = new SplitsGenerator(partitioner);
+        if (MURMUR3PARTITIONER.equals(partitioner)) {
+            LOG.info("Murmur3Partitioner detected, splitting");
+            List<BigInteger> tokens =
+                    clusterMetadata.getTokenRanges().stream()
+                            .map(
+                                    tokenRange ->
+                                            new BigInteger(
+                                                    tokenRange.getEnd().getValue().toString()))
+                            .collect(Collectors.toList());
+            return splitsGenerator.generateSplits(numberOfSplits, tokens);
+        } else {
+            // Murmur3Partitioner is the default and recommended partitioner for Cassandra 1.2+
+            // see
+            // https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerAbout.html
+            LOG.warn(
+                    "The current Cassandra partitioner is {}, only Murmur3Partitioner is supported "
+                            + "for splitting, using an single split",
+                    partitioner);
+            return splitsGenerator.generateSplits(1, Collections.emptyList());
+        }
+    }
+
+    @Override
+    public void addSplitsBack(List<CassandraSplit> splits, int subtaskId) {
+        LOG.info("Add {} splits back to CassandraSplitEnumerator.", splits.size());
+        state.addNewSplits(splits, enumeratorContext.currentParallelism());
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        LOG.info("Adding reader {} to CassandraSplitEnumerator.", subtaskId);
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    private void assignUnprocessedSplitsToReader(int readerId) {
+        checkReaderRegistered(readerId);
+
+        final Set<CassandraSplit> splitsForReader = state.getSplitsForReader(readerId);

Review Comment:
   https://issues.apache.org/jira/browse/FLINK-30805



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1105631447


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorState.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Objects;
+import java.util.Queue;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** State for {@link CassandraSplitEnumerator} to track the splits yet to assign. */
+public class CassandraEnumeratorState {
+
+    private final Queue<CassandraSplit> unassignedSplits;
+
+    CassandraEnumeratorState() {
+        this.unassignedSplits = new ArrayDeque<>();
+    }
+
+    CassandraEnumeratorState(Queue<CassandraSplit> unassignedSplits) {
+        checkNotNull(unassignedSplits);
+        this.unassignedSplits = unassignedSplits;
+    }
+
+    public Queue<CassandraSplit> getUnassignedSplits() {
+        return unassignedSplits;
+    }
+
+    public void addNewSplits(Collection<CassandraSplit> newSplits) {
+        unassignedSplits.addAll(newSplits);
+    }
+
+    public @Nullable CassandraSplit getASplit() {
+        return unassignedSplits.poll();
+    }
+
+    boolean hasMoreSplits() {
+        return unassignedSplits.size() != 0;

Review Comment:
   :+1:



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1105851298


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -31,84 +32,35 @@
 /** tests for query generation and query sanity checks. */
 class CassandraQueryTest {
 
+    private static final Pattern PATTERN = Pattern.compile(CassandraSplitReader.SELECT_REGEXP);
+
     @Test
     public void testKeySpaceTableExtractionRegexp() {
-        final Pattern pattern = Pattern.compile(CassandraSplitReader.SELECT_REGEXP);
-        Matcher matcher;
-        matcher = pattern.matcher("SELECT field FROM keyspace.table where field = value;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
-
-        matcher = pattern.matcher("SELECT * FROM keyspace.table;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
 
-        matcher = pattern.matcher("select field1, field2 from keyspace.table;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
+        assertQueryFormatCorrect("SELECT field FROM keyspace.table where field = value;");
+        assertQueryFormatCorrect("SELECT * FROM keyspace.table;");
+        assertQueryFormatCorrect("select field1, field2 from keyspace.table;");
+        assertQueryFormatCorrect("select field1, field2 from keyspace.table LIMIT(1000);");
+        assertQueryFormatCorrect("select field1 from keyspace.table ;");
+        assertQueryFormatCorrect("select field1 from keyspace.table where field1=1;");
 
-        matcher = pattern.matcher("select field1, field2 from keyspace.table LIMIT(1000);");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
-
-        matcher = pattern.matcher("select field1 from keyspace.table ;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
-
-        matcher = pattern.matcher("select field1 from keyspace.table where field1=1;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
-
-        matcher = pattern.matcher("select field1 from table;"); // missing keyspace
-        assertThat(matcher.matches()).isFalse();
-
-        matcher = pattern.matcher("select field1 from keyspace.table"); // missing ";"
-        assertThat(matcher.matches()).isFalse();
+        assertQueryFormatIncorrect("select field1 from table;"); // missing keyspace
+        assertQueryFormatIncorrect("select field1 from keyspace.table"); // missing ";"
     }
 
     @Test
     public void testProhibitedClauses() {
-        assertThatThrownBy(
-                        () ->
-                                CassandraSource.checkQueryValidity(
-                                        "SELECT COUNT(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () -> CassandraSource.checkQueryValidity("SELECT AVG(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-
-        assertThatThrownBy(
-                        () -> CassandraSource.checkQueryValidity("SELECT MIN(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () -> CassandraSource.checkQueryValidity("SELECT MAX(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () -> CassandraSource.checkQueryValidity("SELECT SUM(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () ->
-                                CassandraSource.checkQueryValidity(
-                                        "SELECT field1, field2 from flink.table ORDER BY field1;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () ->
-                                CassandraSource.checkQueryValidity(
-                                        "SELECT field1, field2 from flink.table GROUP BY field1;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
+        Arrays.stream(
+                        new String[] {

Review Comment:
   Sketchy thing, not ready for review



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1135384036


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.cassandra.service.StorageServiceMBean;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableList;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableMap;
+
+import javax.management.JMX;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.rmi.server.RMISocketFactory;
+import java.util.Map;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final String JMX_URL = "service:jmx:rmi:///jndi/rmi://%s:%d/jmxrmi";
+    private static final String STORAGE_SERVICE_MBEAN =
+            "org.apache.cassandra.db:type=StorageService";
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+    private final String dockerHostIp;
+    private final int jmxPort;
+
+    private Cluster cluster;
+    private Session session;
+    private ClusterBuilder clusterBuilder;
+
+    public CassandraTestEnvironment() {
+        try (ServerSocket s = new ServerSocket(0)) {
+            // use fixed free random port for JMX
+            jmxPort = s.getLocalPort();
+            // resolve IP of docker host to establish JMX connection
+            dockerHostIp = InetAddress.getLocalHost().getHostAddress();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+
+        cassandraContainer = new CassandraContainer(DOCKER_CASSANDRA_IMAGE);
+        // use a fixed port mapping for JMX, it doesn't work well with mapped ports
+        cassandraContainer.setPortBindings(ImmutableList.of(jmxPort + ":" + jmxPort));
+        // more generous timeouts and remote JMX configuration
+        addJavaOpts(
+                cassandraContainer,
+                "-Dcassandra.request_timeout_in_ms=30000",
+                "-Dcassandra.read_request_timeout_in_ms=15000",
+                "-Dcassandra.write_request_timeout_in_ms=6000",
+                "-Dcassandra.jmx.remote.port=" + jmxPort,
+                "-Dcom.sun.management.jmxremote.rmi.port=" + jmxPort,
+                "-Djava.rmi.server.hostname=" + dockerHostIp,
+                "-Dcom.sun.management.jmxremote.host=" + dockerHostIp);

Review Comment:
   ```suggestion
                   "-Dcassandra.write_request_timeout_in_ms=6000");
   ```
   The JMX properties aren't required when using nodetool.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138354478


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.net.InetSocketAddress;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+
+    private Cluster cluster;
+    private Session session;
+    private ClusterBuilder clusterBuilder;
+
+    public CassandraTestEnvironment() {
+        cassandraContainer = new CassandraContainer(DOCKER_CASSANDRA_IMAGE);
+        // more generous timeouts
+        addJavaOpts(
+                cassandraContainer,
+                "-Dcassandra.request_timeout_in_ms=30000",
+                "-Dcassandra.read_request_timeout_in_ms=15000",
+                "-Dcassandra.write_request_timeout_in_ms=6000");
+    }
+
+    @Override
+    public void startUp() throws Exception {
+        startEnv();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        stopEnv();
+    }
+
+    private static void addJavaOpts(GenericContainer<?> container, String... opts) {
+        String jvmOpts = container.getEnvMap().getOrDefault("JVM_OPTS", "");
+        container.withEnv("JVM_OPTS", jvmOpts + " " + StringUtils.join(opts, " "));
+    }
+
+    private void startEnv() throws Exception {
+        // configure container start to wait until cassandra is ready to receive queries
+        cassandraContainer.waitingFor(new CassandraQueryWaitStrategy());
+        // start with retrials
+        cassandraContainer.start();
+        cassandraContainer.followOutput(
+                new Slf4jLogConsumer(LOG),
+                OutputFrame.OutputType.END,
+                OutputFrame.OutputType.STDERR,
+                OutputFrame.OutputType.STDOUT);
+
+        cluster = cassandraContainer.getCluster();
+        clusterBuilder =
+                createBuilderWithConsistencyLevel(
+                        ConsistencyLevel.ONE,
+                        cassandraContainer.getHost(),
+                        cassandraContainer.getMappedPort(CQL_PORT));
+
+        session = cluster.connect();
+        session.execute(requestWithTimeout(CREATE_KEYSPACE_QUERY));
+        // create a dedicated table for split size tests (to avoid having to flush with each test)
+        insertTestDataForSplitSizeTests();
+    }
+
+    private void insertTestDataForSplitSizeTests() throws Exception {
+        session.execute(requestWithTimeout(CREATE_SPLITS_TABLE_QUERY));
+        for (int i = 0; i < NB_SPLITS_RECORDS; i++) {
+            session.execute(requestWithTimeout(String.format(INSERT_INTO_FLINK_SPLITS, i, i)));
+        }
+        flushMemTables(SPLITS_TABLE);
+    }
+
+    private void stopEnv() {
+
+        if (session != null) {
+            session.close();
+        }
+        if (cluster != null) {
+            cluster.close();
+        }
+        cassandraContainer.stop();
+    }
+
+    private ClusterBuilder createBuilderWithConsistencyLevel(
+            ConsistencyLevel consistencyLevel, String host, int port) {
+        return new ClusterBuilder() {
+            @Override
+            protected Cluster buildCluster(Cluster.Builder builder) {
+                return builder.addContactPointsWithPorts(new InetSocketAddress(host, port))
+                        .withQueryOptions(
+                                new QueryOptions()
+                                        .setConsistencyLevel(consistencyLevel)
+                                        .setSerialConsistencyLevel(ConsistencyLevel.LOCAL_SERIAL))
+                        .withSocketOptions(
+                                new SocketOptions()
+                                        // default timeout x 3
+                                        .setConnectTimeoutMillis(15000)
+                                        // default timeout x3 and higher than
+                                        // request_timeout_in_ms at the cluster level
+                                        .setReadTimeoutMillis(READ_TIMEOUT_MILLIS))
+                        .withoutJMXReporting()
+                        .withoutMetrics()
+                        .build();
+            }
+        };
+    }
+
+    /**
+     * Force the flush of cassandra memTables to SSTables in order to update size_estimates. It is
+     * needed for the tests because we just inserted records, we need to force cassandra to update
+     * size_estimates system table.
+     */
+    void flushMemTables(String table) throws Exception {
+        cassandraContainer.execInContainer("nodetool", "flush", KEYSPACE, table);

Review Comment:
   no problem, yo where right to double check



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.net.InetSocketAddress;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+
+    private Cluster cluster;
+    private Session session;
+    private ClusterBuilder clusterBuilder;
+
+    public CassandraTestEnvironment() {
+        cassandraContainer = new CassandraContainer(DOCKER_CASSANDRA_IMAGE);
+        // more generous timeouts
+        addJavaOpts(
+                cassandraContainer,
+                "-Dcassandra.request_timeout_in_ms=30000",
+                "-Dcassandra.read_request_timeout_in_ms=15000",
+                "-Dcassandra.write_request_timeout_in_ms=6000");
+    }
+
+    @Override
+    public void startUp() throws Exception {
+        startEnv();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        stopEnv();
+    }
+
+    private static void addJavaOpts(GenericContainer<?> container, String... opts) {
+        String jvmOpts = container.getEnvMap().getOrDefault("JVM_OPTS", "");
+        container.withEnv("JVM_OPTS", jvmOpts + " " + StringUtils.join(opts, " "));
+    }
+
+    private void startEnv() throws Exception {
+        // configure container start to wait until cassandra is ready to receive queries
+        cassandraContainer.waitingFor(new CassandraQueryWaitStrategy());
+        // start with retrials
+        cassandraContainer.start();
+        cassandraContainer.followOutput(
+                new Slf4jLogConsumer(LOG),
+                OutputFrame.OutputType.END,
+                OutputFrame.OutputType.STDERR,
+                OutputFrame.OutputType.STDOUT);
+
+        cluster = cassandraContainer.getCluster();
+        clusterBuilder =
+                createBuilderWithConsistencyLevel(
+                        ConsistencyLevel.ONE,
+                        cassandraContainer.getHost(),
+                        cassandraContainer.getMappedPort(CQL_PORT));
+
+        session = cluster.connect();
+        session.execute(requestWithTimeout(CREATE_KEYSPACE_QUERY));
+        // create a dedicated table for split size tests (to avoid having to flush with each test)
+        insertTestDataForSplitSizeTests();
+    }
+
+    private void insertTestDataForSplitSizeTests() throws Exception {
+        session.execute(requestWithTimeout(CREATE_SPLITS_TABLE_QUERY));
+        for (int i = 0; i < NB_SPLITS_RECORDS; i++) {
+            session.execute(requestWithTimeout(String.format(INSERT_INTO_FLINK_SPLITS, i, i)));
+        }
+        flushMemTables(SPLITS_TABLE);
+    }
+
+    private void stopEnv() {
+
+        if (session != null) {
+            session.close();
+        }
+        if (cluster != null) {
+            cluster.close();
+        }
+        cassandraContainer.stop();
+    }
+
+    private ClusterBuilder createBuilderWithConsistencyLevel(
+            ConsistencyLevel consistencyLevel, String host, int port) {
+        return new ClusterBuilder() {
+            @Override
+            protected Cluster buildCluster(Cluster.Builder builder) {
+                return builder.addContactPointsWithPorts(new InetSocketAddress(host, port))
+                        .withQueryOptions(
+                                new QueryOptions()
+                                        .setConsistencyLevel(consistencyLevel)
+                                        .setSerialConsistencyLevel(ConsistencyLevel.LOCAL_SERIAL))
+                        .withSocketOptions(
+                                new SocketOptions()
+                                        // default timeout x 3
+                                        .setConnectTimeoutMillis(15000)
+                                        // default timeout x3 and higher than
+                                        // request_timeout_in_ms at the cluster level
+                                        .setReadTimeoutMillis(READ_TIMEOUT_MILLIS))
+                        .withoutJMXReporting()
+                        .withoutMetrics()
+                        .build();
+            }
+        };
+    }
+
+    /**
+     * Force the flush of cassandra memTables to SSTables in order to update size_estimates. It is
+     * needed for the tests because we just inserted records, we need to force cassandra to update
+     * size_estimates system table.
+     */
+    void flushMemTables(String table) throws Exception {
+        cassandraContainer.execInContainer("nodetool", "flush", KEYSPACE, table);

Review Comment:
   no problem, yo were right to double check



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138363561


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high
+                LOG.info(
+                        "maxSplitMemorySize set value leads to {} splits with a task parallelism of {}. Creating as many splits as parallelism",
+                        numSplits,
+                        parallelism);
+                numSplits = parallelism;
+            }
+        } else { // not defined
+            LOG.info("maxSplitMemorySize not set. Creating as many splits as parallelism");
+            numSplits = parallelism;
+        }
+
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken = partitioner.minToken;
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            BigInteger endToken = startToken.add(increment);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken;
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+            startToken = endToken;
+        }
+        LOG.debug("Generated {} splits : {}", splits.size(), splits);
+        return splits;
+    }
+
+    /**
+     * Estimates the size of the table in bytes. Cassandra size estimates can be 0 if the data was
+     * just inserted and the amount of data in the table was small. This is very common situation
+     * during tests.
+     */
+    @VisibleForTesting
+    public long estimateTableSize() {
+        List<TokenRange> tokenRanges = getTokenRangesOfTable();
+        long size = 0L;
+        for (TokenRange tokenRange : tokenRanges) {
+            size += tokenRange.meanPartitionSize * tokenRange.partitionCount;
+        }
+        return Math.round(size / getRingFraction(tokenRanges));
+    }
+
+    /**
+     * The values that we get from system.size_estimates are for one node. We need to extrapolate to
+     * the whole cluster. This method estimates the percentage, the node represents in the cluster.
+     *
+     * @param tokenRanges The list of {@link TokenRange} to estimate
+     * @return The percentage the node represent in the whole cluster
+     */
+    private float getRingFraction(List<TokenRange> tokenRanges) {
+        BigInteger addressedTokens = BigInteger.ZERO;
+        for (TokenRange tokenRange : tokenRanges) {
+            addressedTokens =
+                    addressedTokens.add(distance(tokenRange.rangeStart, tokenRange.rangeEnd));
+        }
+        // it is < 1 because it is a percentage
+        return Float.valueOf(addressedTokens.divide(partitioner.ringSize).toString());

Review Comment:
   Indeed, you're right if the `size_estimates` table is empty, then calculated `ringFraction` is 0 leading to division by 0. Thanks for catching it ! In that case I'll return a 0 estimated table size, leading to numsplits = parallelism (cf your suggestion in a comment above).



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137185252


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 20;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        100_000_000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // tableSize / maxSplitMemorySize is too little compared to parallelism falling back to
+        // number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too small split size set")
+    public void testGenerateSplitsWithTooSmallSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        1L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // tableSize / maxSplitMemorySize is too big compared to parallelism falling back to
+        // number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+    }
+
+    // overridden to use unordered checks
+    @Override
+    protected void checkResultWithSemantic(
+            CloseableIterator<Pojo> resultIterator,
+            List<List<Pojo>> testData,
+            CheckpointingMode semantic,
+            Integer limit) {
+        if (limit != null) {
+            Runnable runnable =
+                    () ->
+                            CollectIteratorAssertions.assertUnordered(resultIterator)
+                                    .withNumRecordsLimit(limit)
+                                    .matchesRecordsFromSource(testData, semantic);
+
+            assertThat(runAsync(runnable)).succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT);

Review Comment:
   because it was already like this  in `SourceTestSuiteBase` if just took the same impl and replaced `assertThat` by `assertUnordered`



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137146721


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.net.InetSocketAddress;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087939852


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits, enumeratorContext.currentParallelism());
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitioner = clusterMetadata.getPartitioner();
+        final SplitsGenerator splitsGenerator = new SplitsGenerator(partitioner);
+        if (MURMUR3PARTITIONER.equals(partitioner)) {
+            LOG.info("Murmur3Partitioner detected, splitting");
+            List<BigInteger> tokens =
+                    clusterMetadata.getTokenRanges().stream()
+                            .map(
+                                    tokenRange ->
+                                            new BigInteger(
+                                                    tokenRange.getEnd().getValue().toString()))
+                            .collect(Collectors.toList());
+            return splitsGenerator.generateSplits(numberOfSplits, tokens);
+        } else {
+            // Murmur3Partitioner is the default and recommended partitioner for Cassandra 1.2+
+            // see
+            // https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerAbout.html
+            LOG.warn(
+                    "The current Cassandra partitioner is {}, only Murmur3Partitioner is supported "
+                            + "for splitting, using an single split",
+                    partitioner);
+            return splitsGenerator.generateSplits(1, Collections.emptyList());
+        }
+    }
+
+    @Override
+    public void addSplitsBack(List<CassandraSplit> splits, int subtaskId) {
+        LOG.info("Add {} splits back to CassandraSplitEnumerator.", splits.size());
+        state.addNewSplits(splits, enumeratorContext.currentParallelism());
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        LOG.info("Adding reader {} to CassandraSplitEnumerator.", subtaskId);
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    private void assignUnprocessedSplitsToReader(int readerId) {
+        checkReaderRegistered(readerId);
+
+        final Set<CassandraSplit> splitsForReader = state.getSplitsForReader(readerId);

Review Comment:
   It looks like the fetching of additional splits isn't implemented generically. The FileSourceReader explicitly requests requests whenever it has finished a split, and I don't see anything where this happens automatically.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089034416


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraRow.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+
+import com.datastax.driver.core.ExecutionInfo;
+import com.datastax.driver.core.Row;
+
+/**
+ * Wrapper for Cassandra {@link Row} that stores associated {@link RingRange} to be able to update
+ * split states. It also stores {@link ExecutionInfo} Cassandra statistics about the query execution
+ * that produced this row.
+ */
+public class CassandraRow {
+
+    private final Row row;
+    private final RingRange associatedRingRange;
+    private final ExecutionInfo executionInfo;

Review Comment:
   Now that we store the `ResultSet` in `CassandraSplitState` to keep track of the fetch state, we no more need this `CassandraRow` wrapper as will have the `ExecutionInfo` in the `Resultset` with the call to `RecordEmitter#emitRecord()`



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1091859558


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/split/SplitsGeneratorTest.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link SplitsGenerator}. */
+public final class SplitsGeneratorTest {

Review Comment:
   ```suggestion
   final class SplitsGeneratorTest {
   ```



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.CassandraSource;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** tests for query generation and query sanity checks. */
+class CassandraQueryTest {
+
+    @Test
+    public void testKeySpaceTableExtractionRegexp() {
+        final Pattern pattern = Pattern.compile(CassandraSplitReader.SELECT_REGEXP);
+        Matcher matcher;
+        matcher = pattern.matcher("SELECT field FROM keyspace.table where field = value;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("SELECT * FROM keyspace.table;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1, field2 from keyspace.table;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1, field2 from keyspace.table LIMIT(1000);");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1 from keyspace.table ;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1 from keyspace.table where field1=1;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1 from table;"); // missing keyspace
+        assertThat(matcher.matches()).isFalse();
+
+        matcher = pattern.matcher("select field1 from keyspace.table"); // missing ";"
+        assertThat(matcher.matches()).isFalse();
+    }
+
+    @Test
+    public void testProhibitedClauses() {
+        assertThatThrownBy(
+                        () ->
+                                CassandraSource.checkQueryValidity(
+                                        "SELECT COUNT(*) from flink.table;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");
+        assertThatThrownBy(
+                        () -> CassandraSource.checkQueryValidity("SELECT AVG(*) from flink.table;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");
+
+        assertThatThrownBy(
+                        () -> CassandraSource.checkQueryValidity("SELECT MIN(*) from flink.table;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");
+        assertThatThrownBy(
+                        () -> CassandraSource.checkQueryValidity("SELECT MAX(*) from flink.table;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");
+        assertThatThrownBy(
+                        () -> CassandraSource.checkQueryValidity("SELECT SUM(*) from flink.table;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");
+        assertThatThrownBy(
+                        () ->
+                                CassandraSource.checkQueryValidity(
+                                        "SELECT field1, field2 from flink.table ORDER BY field1;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");
+        assertThatThrownBy(
+                        () ->
+                                CassandraSource.checkQueryValidity(
+                                        "SELECT field1, field2 from flink.table GROUP BY field1;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");

Review Comment:
   You could make this a bit more readable by having adding another method that does the assertions, and in here you do something like:
   ```
   Arrays.stream(
       "SELECT COUNT(*) from flink.table;", 
       ...)
     .forEach(CassandraQueryTest::assertProhibitedClauseRejected).
   ```



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -41,139 +41,31 @@ public SplitsGenerator(CassandraPartitioner partitioner) {
     }
 
     /**
-     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
-     * splits. Each split can contain several token ranges in order to reduce the overhead of
-     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
-     * share the same replicas.
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the ring.
      *
-     * @param totalSplitCount requested total amount of splits. This function may generate more
-     *     splits.
-     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
-     *     order.
-     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     * @param numSplits requested number of splits
+     * @return list containing {@code numSplits} CassandraSplits.
      */
-    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
-        if (totalSplitCount == 1) {
-            RingRange totalRingRange = RingRange.of(partitioner.min(), partitioner.max());
+    public List<CassandraSplit> generateSplits(long numSplits) {

Review Comment:
   AFAICT this could be a static method; there isn't really a benefit in having a `SplitsGenerator` object. It's only created, used exactly once, and then discarded right away, all happening in the same method.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorState.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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Objects;
+import java.util.Queue;
+
+/** Sate for {@link CassandraSplitEnumerator} to track the splits yet to assign. */
+public class CassandraEnumeratorState {
+    private final Queue<CassandraSplit> unassignedSplits;
+
+    public CassandraEnumeratorState() {
+        this.unassignedSplits = new ArrayDeque<>();
+    }
+
+    public CassandraEnumeratorState(Queue<CassandraSplit> unassignedSplits) {
+        this.unassignedSplits = unassignedSplits;
+    }
+
+    public void addNewSplits(Collection<CassandraSplit> newSplits) {
+        unassignedSplits.addAll(newSplits);
+    }
+
+    public CassandraSplit getASplit() {

Review Comment:
   add `@Nullable`



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorState.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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Objects;
+import java.util.Queue;
+
+/** Sate for {@link CassandraSplitEnumerator} to track the splits yet to assign. */
+public class CassandraEnumeratorState {
+    private final Queue<CassandraSplit> unassignedSplits;
+
+    public CassandraEnumeratorState() {
+        this.unassignedSplits = new ArrayDeque<>();
+    }
+
+    public CassandraEnumeratorState(Queue<CassandraSplit> unassignedSplits) {
+        this.unassignedSplits = unassignedSplits;

Review Comment:
   add `checkNotNull`



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitState.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import com.datastax.driver.core.ResultSet;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+
+/**
+ * Mutable {@link CassandraSplit} that keeps track of the reading process of the associated split.
+ */
+public class CassandraSplitState {
+    private final CassandraSplit cassandraSplit;
+    // Cassandra ResultSet is paginated, a new page is read only if all the records of the previous
+    // one were consumed. fetch() can be interrupted so we use the resultSet to keep track of the
+    // reading process.
+    // It is null when reading has not started (before fetch is called on the split).
+    @Nullable private ResultSet resultSet;

Review Comment:
   reminder: Curious how this is meant to persist interrupted fetches with checkpoints in-between fetches. Risk of duplicating input.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   > I added MAX_RECORDS_PER_SPLIT conf parameter as proposed above because I did not know if such parameter existed already. Feel
   
   I'll look around. The naming seems off though; it's not a maximum records for a split (which would be more relevant for the `SplitGenerator`), but more of a `batch size`.
   
   > So, I could store in the CassandraSplitState a reference to the ResultSet to resume the output of remaining records on a later fetch().
   
   You also have to consider how this information can be stored in a checkpoint. A checkpoint may happen between 2 fetches for 1 split (== first half of emitted will be remembered by Flink), and then the job is restarted. As-is you'd read the first half again, because your split state isn't written to checkpoints.
   
   Not sure how to solve that. If cassandra has a deterministic order in which it returns records (it probably doesnt :see_no_evil: ) then you could remember the count and skip so many records or adjust the start token of the split.
   
   Alternatively we could consider handling this in the SplitGenerator, generating as many splits as necessary to keep at most N records in memory in each split reader. (We'd never have the case that 1 splits contains the entire table)



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitSerializerTest.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.math.BigInteger;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link CassandraSplitSerializer}. */
+public class CassandraSplitSerializerTest {

Review Comment:
   ```suggestion
   class CassandraSplitSerializerTest {
   ```



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializerTest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
+
+import org.junit.jupiter.api.Test;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link CassandraEnumeratorStateSerializer}. */
+public class CassandraEnumeratorStateSerializerTest {

Review Comment:
   ```suggestion
   class CassandraEnumeratorStateSerializerTest {
   ```



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.CassandraSource;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** tests for query generation and query sanity checks. */
+class CassandraQueryTest {
+
+    @Test
+    public void testKeySpaceTableExtractionRegexp() {
+        final Pattern pattern = Pattern.compile(CassandraSplitReader.SELECT_REGEXP);
+        Matcher matcher;
+        matcher = pattern.matcher("SELECT field FROM keyspace.table where field = value;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("SELECT * FROM keyspace.table;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1, field2 from keyspace.table;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1, field2 from keyspace.table LIMIT(1000);");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1 from keyspace.table ;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1 from keyspace.table where field1=1;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1 from table;"); // missing keyspace
+        assertThat(matcher.matches()).isFalse();
+
+        matcher = pattern.matcher("select field1 from keyspace.table"); // missing ";"
+        assertThat(matcher.matches()).isFalse();

Review Comment:
   Similar to the below test, there are basically only 2 cases here, so we could categorize inputs into accepted/rejected and call new assertion methods.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorState.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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Objects;
+import java.util.Queue;
+
+/** Sate for {@link CassandraSplitEnumerator} to track the splits yet to assign. */
+public class CassandraEnumeratorState {
+    private final Queue<CassandraSplit> unassignedSplits;
+
+    public CassandraEnumeratorState() {
+        this.unassignedSplits = new ArrayDeque<>();
+    }
+
+    public CassandraEnumeratorState(Queue<CassandraSplit> unassignedSplits) {

Review Comment:
   ```suggestion
       private CassandraEnumeratorState(Queue<CassandraSplit> unassignedSplits) {
   ```
   Would this be possible?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorState.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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Objects;
+import java.util.Queue;
+
+/** Sate for {@link CassandraSplitEnumerator} to track the splits yet to assign. */

Review Comment:
   ```suggestion
   /** State for {@link CassandraSplitEnumerator} to track the splits yet to assign. */
   ```



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and Flink
+ * source parallelism.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final CassandraPartitioner partitioner;
+
+    public SplitsGenerator(CassandraPartitioner partitioner) {
+        this.partitioner = partitioner;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the ring.
+     *
+     * @param numSplits requested number of splits
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long numSplits) {

Review Comment:
   ```suggestion
       public List<CassandraSplit> generateSplits(int numSplits) {
   ```



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and Flink
+ * source parallelism.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final CassandraPartitioner partitioner;
+
+    public SplitsGenerator(CassandraPartitioner partitioner) {
+        this.partitioner = partitioner;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the ring.
+     *
+     * @param numSplits requested number of splits
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long numSplits) {
+        if (numSplits == 1) {
+            return Collections.singletonList(
+                    new CassandraSplit(partitioner.minToken(), partitioner.maxToken()));
+        }
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger splitSize =
+                (partitioner.ringSize()).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken, endToken = partitioner.minToken();
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            startToken = endToken;
+            endToken = startToken.add(splitSize);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken();
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+        }

Review Comment:
   ```suggestion
           BigInteger startToken = partitioner.minToken();
           for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
               BigInteger endToken = startToken.add(splitSize);
               if (splitCount == numSplits) {
                   endToken = partitioner.maxToken();
               }
               splits.add(new CassandraSplit(startToken, endToken));
   
               startToken = endToken;
           }
   ```
   
   To me this feels more intuitive/readable. It's strange to assign minToken to endToken at the start, as is setting up the next iteration at the start of the current iteration.
   
   Personally I'd also move the `splitCount == numSplits` out of the loop. It is trivially true when a "simple" loop from 0 to `< numSplits` terminates.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1404820642

   > > I don't think it is an ASF rule but fair enough,
   > 
   > FYI; By and large it should be viewed as a legal requirement. By copying code from cassandra you have to adhere to their licensing, which among other state that you must have prominent notices for changes to a file.
   
   You're right cf ASF v2 4.d:
   `If the Work includes a "NOTICE" text file as part of its distribution, then any Derivative Works that You distribute must include a readable copy of the attribution notices contained within such NOTICE file, excluding those notices that do not pertain to any part of the Derivative Works, in at least one of the following places: within a NOTICE text file distributed as part of the Derivative Works; within the Source form or documentation, if provided along with the Derivative Works; or, within a display generated by the Derivative Works, if and wherever such third-party notices normally appear. The contents of the NOTICE file are for informational purposes only and do not modify the License. You may add Your own attribution notices within Derivative Works that You distribute, alongside or as an addendum to the NOTICE text from the Work, provided that such additional attribution notices cannot be construed as modifying the License.`
   
   Anyway, as we decided to replace the split code with something that was never merged to Beam, there is no need anymore.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1088816929


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   > from all splits into memory
   
   Well, in 99% of the cases, `unprocessedSplits` will contain only a single split as the enumerator now assigns only one split per reader. The remaining case I see is when the parallelism of the job is decreased (either by user or failover). In that case, as you know, upon restoration, a reader could be assigned the splits of another reader (as there are less readers after parallelism decrease).
   
   But still you're right, in case of parallelism=1 then, all the records would be in 1 split in 1 reader, there will be only a single query issued and a single resultset containing all the records. These records will indeed be stored in `recordsBySplit` to serve for constructing returned `RecordsBySplits`. That being said, as there will be a single resultset, we need to figure out a way of splitting it for that particular case. Luckily Cassandra `ResultSet` is paginated with results fetched only when all the records of the previous page were consumed. So, I could store in the `CassandraSplitState` a reference to the `ResultSet` to resume the output of remaining records on a later `fetch()`. My question is: what would be the condition to exit fetch() earlier? What about allowing the user to configure a `maxRecordPerSplit` ?
   
   It is true that fetch early exit should be mentioned in the docs. I'll open a ticket for that and do the PR later. 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089212568


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraRecordEmitter.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.api.connector.source.SourceOutput;
+import org.apache.flink.connector.base.source.reader.RecordEmitter;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.ExecutionInfo;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.mapping.Mapper;
+import com.datastax.driver.mapping.MappingManager;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * {@link RecordEmitter} that converts the {@link CassandraRow} read by the {@link
+ * CassandraSplitReader} to specified POJO and output it while updating splits state. This class
+ * uses the Cassandra driver mapper to map the row to the POJO.
+ *
+ * @param <OUT> type of POJO record to output
+ */
+public class CassandraRecordEmitter<OUT>
+        implements RecordEmitter<CassandraRow, OUT, CassandraSplitState> {
+
+    private final Mapper<OUT> mapper;
+
+    public CassandraRecordEmitter(
+            Class<OUT> pojoClass, ClusterBuilder clusterBuilder, MapperOptions mapperOptions) {
+        // session and cluster are managed at the SplitReader level. So we need to create one
+        // locally here just to me able to create the mapper.
+        final Cluster cluster = clusterBuilder.getCluster();
+        final Session session = cluster.connect();

Review Comment:
   I'll do as you suggested in the other comment



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101496694


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -48,6 +52,30 @@ public CassandraSplitState toSplitState() {
         return new CassandraSplitState(new HashSet<>(ringRanges), splitId());
     }
 
+    public void serialize(ObjectOutputStream objectOutputStream) throws IOException {

Review Comment:
   > For symmetry I'd prefer this method to be static and accept a CassandraSplit instance 
   
   I made the serializer take an OutputStream to avoid creating an OutputStream for each split in the EnumeratorState serialization which would create a stream header for each split



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101583261


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorState.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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Objects;
+import java.util.Queue;
+
+/** Sate for {@link CassandraSplitEnumerator} to track the splits yet to assign. */
+public class CassandraEnumeratorState {
+    private final Queue<CassandraSplit> unassignedSplits;
+
+    public CassandraEnumeratorState() {
+        this.unassignedSplits = new ArrayDeque<>();
+    }
+
+    public CassandraEnumeratorState(Queue<CassandraSplit> unassignedSplits) {

Review Comment:
   no as now the serialization code of `CassandraEnumeratorState` which uses this method is in `CassandraEnumeratorStateSerializer`. But both constructors can be lowered to package local visibility



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1448409889

   @zentol I addressed all your comments and changed the splits architecture:
   I Introduced a table size estimation (based on Cassandra statistical size estimates). I added an optional user conf to specify max split memory size. If set, the source generates splits of `maxSplitMemorySize` with protection measures (in comparison to task parallelism) for number of splits. 
   I now read a split as a whole (no state needed).
   I added the related splits and size tests. They require to use JMX to force mem tables flush on the cassandra cluster so that the system size estimates can be updated (as we just wrote test data). The official Cassandra image deactivates jmx, to enable it we need to provide authentication and modify cassandra-env.sh so I had to create my own image (!)
   Also the flush is very long (30s) so for all split tests I wrote and flush only once (contrary to other tests that write test data for each test)
   
   PTAL. I hope it will be last round of review as I changed a lot and spent so much time on that.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] RyanSkraba commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
RyanSkraba commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1054648476


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Sets;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            // needs to be mutable

Review Comment:
   ```suggestion
               // The set containing the single, inclusive ring range needs to be mutable
   ```
   Just a suggestion for clarity.  This can be bit tricky to follow.



##########
flink-connector-cassandra/src/test/resources/log4j2-test.properties:
##########
@@ -18,7 +18,7 @@
 
 # Set root logger level to OFF to not flood build logs
 # set manually to INFO for debugging purposes
-rootLogger.level = OFF

Review Comment:
   I found this useful while reviewing, but it might be OK to turn off for CI.



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;

Review Comment:
   ```suggestion
   import static org.assertj.core.api.Assertions.assertThat;
   ```
   This is probably the one you want.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.api.connector.source.SourceSplit;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * {@link SourceSplit} for Cassandra source. A Cassandra split is just a set of {@link RingRange}s
+ * (a range between 2 tokens). Tokens are spread across the Cassandra cluster with each node
+ * managing a share of the token ring. Each split can contain several token ranges in order to
+ * reduce the overhead on Cassandra vnodes.
+ */
+public class CassandraSplit implements SourceSplit, Serializable {
+
+    private final Set<RingRange> ringRanges;
+
+    public CassandraSplit(Set<RingRange> ringRanges) {
+        this.ringRanges = ringRanges;
+    }
+
+    public Set<RingRange> getRingRanges() {
+        return ringRanges;
+    }
+
+    @Override
+    public String splitId() {
+        return ringRanges.toString();

Review Comment:
   I noticed this is mutable since ringRanges are mutable while the split is being read -- do you think this will pose a problem if the splitId changes?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorState.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+/** Sate for {@link CassandraSplitEnumerator} to track the splits yet to assign. */
+public class CassandraEnumeratorState implements Serializable {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraEnumeratorState.class);
+
+    // map readerId to splits
+    private final Map<Integer, Set<CassandraSplit>> unassignedSplits = new HashMap<>();
+
+    public void addNewSplits(Collection<CassandraSplit> newSplits, int numReaders) {
+        for (CassandraSplit split : newSplits) {
+            int ownerReader = getOwnerReader(numReaders, split);
+            unassignedSplits.computeIfAbsent(ownerReader, r -> new HashSet<>()).add(split);
+        }
+    }
+
+    private int getOwnerReader(int numReaders, CassandraSplit split) {
+        // readerId == subTaksId == 0 or 1 if numReaders == 2 so  modulo is fine for ownerReader

Review Comment:
   ```suggestion
           // readerId == subTaskId == 0 or 1 if numReaders == 2 so  modulo is fine for ownerReader
   ```



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138384098


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 20;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        100_000_000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // tableSize / maxSplitMemorySize is too little compared to parallelism falling back to
+        // number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too small split size set")
+    public void testGenerateSplitsWithTooSmallSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        1L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // tableSize / maxSplitMemorySize is too big compared to parallelism falling back to

Review Comment:
   cf https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138381568



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1468388485

   > Instead of this whole JMX business we could use nodetool directly.
   > 
   > (Note: This review is specifically about the JMX parts; nothing else)
   
   Indeed, this binary is included in the offical cassandra image ! Way simpler. Thanks for the suggestion, I removed all the JMX related code.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1086442791


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorStateSerializer;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraSplitEnumerator;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSourceReader;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSplitReader;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A bounded source to read from Cassandra and return a collection of entities as {@code
+ * DataStream<Entity>}. An entity is built by Cassandra mapper ({@code
+ * com.datastax.driver.mapping.EntityMapper}) based on a POJO containing annotations (as described
+ * in <a
+ * href="https://docs.datastax.com/en/developer/java-driver/3.11/manual/object_mapper/creating/">
+ * Cassandra object mapper</a>).
+ *
+ * <p>To use it, do the following:
+ *
+ * <pre>{@code
+ * ClusterBuilder clusterBuilder = new ClusterBuilder() {
+ *   @Override
+ *   protected Cluster buildCluster(Cluster.Builder builder) {
+ *     return builder.addContactPointsWithPorts(new InetSocketAddress(HOST,PORT))
+ *                   .withQueryOptions(new QueryOptions().setConsistencyLevel(CL))
+ *                   .withSocketOptions(new SocketOptions()
+ *                   .setConnectTimeoutMillis(CONNECT_TIMEOUT)
+ *                   .setReadTimeoutMillis(READ_TIMEOUT))
+ *                   .build();
+ *   }
+ * };
+ * Source cassandraSource = new CassandraSource(clusterBuilder,
+ *                                              Pojo.class,
+ *                                              "select ... from KEYSPACE.TABLE ...;",
+ *                                              () -> new Mapper.Option[] {Mapper.Option.saveNullFields(true)});
+ *
+ * DataStream<Pojo> stream = env.fromSource(cassandraSource, WatermarkStrategy.noWatermarks(),
+ * "CassandraSource");
+ * }</pre>
+ */
+@PublicEvolving
+public class CassandraSource<OUT>
+        implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
+
+    public static final String CQL_PROHIBITTED_CLAUSES_REGEXP =
+            "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    private static final long serialVersionUID = 7773196541275567433L;
+
+    private final ClusterBuilder clusterBuilder;
+    private final Class<OUT> pojoClass;
+    private final String query;
+    private final MapperOptions mapperOptions;
+
+    public CassandraSource(
+            ClusterBuilder clusterBuilder,
+            Class<OUT> pojoClass,
+            String query,
+            MapperOptions mapperOptions) {
+        checkNotNull(clusterBuilder, "ClusterBuilder required but not provided");
+        checkNotNull(pojoClass, "POJO class required but not provided");
+        checkQueryValidity(query);
+        this.clusterBuilder = clusterBuilder;
+        ClosureCleaner.clean(clusterBuilder, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+        this.pojoClass = pojoClass;
+        this.query = query;
+        this.mapperOptions = mapperOptions;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return Boundedness.BOUNDED;
+    }
+
+    @Internal
+    @Override
+    public SourceReader<OUT, CassandraSplit> createReader(SourceReaderContext readerContext) {
+        return new CassandraSourceReader<>(
+                readerContext, clusterBuilder, pojoClass, query, mapperOptions);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> createEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext) {
+        return new CassandraSplitEnumerator(enumContext, null, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> restoreEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext,
+            CassandraEnumeratorState enumCheckpoint) {
+        return new CassandraSplitEnumerator(enumContext, enumCheckpoint, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraSplit> getSplitSerializer() {
+        return CassandraSplitSerializer.INSTANCE;
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraEnumeratorState> getEnumeratorCheckpointSerializer() {
+        return CassandraEnumeratorStateSerializer.INSTANCE;
+    }
+
+    @Override
+    public TypeInformation<OUT> getProducedType() {
+        return TypeInformation.of(pojoClass);
+    }
+
+    @VisibleForTesting
+    public static void checkQueryValidity(String query) {
+        checkNotNull(query, "query required but not provided");
+        checkState(
+                query.matches(CassandraSplitReader.SELECT_REGEXP),
+                "query must be of the form select ... from keyspace.table ...;");
+        checkState(
+                !query.matches(CQL_PROHIBITTED_CLAUSES_REGEXP),
+                "query must not contain aggregate or order clauses because they will be done per split. "
+                        + "So they will be incorrect after merging the splits");

Review Comment:
   I don't get what you mean. What I meant here is that any oder or grouping SQL operation will be done at the split level so they will be incorrect at the DataStream level. So I chose to disable them in the source and let the user use the regular methods in DataStream instead.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1085417186


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {
+            // allow to interrupt the reading of splits as requested in the API
+            if (wakeup.get()) {
+                break;
+            }
+            if (!cassandraSplitState.isEmpty()) {
+                try {
+                    final Set<RingRange> ringRanges =
+                            cassandraSplitState.getUnprocessedRingRanges();
+                    final String cassandraSplitId = cassandraSplitState.getSplitId();
+
+                    for (RingRange ringRange : ringRanges) {
+                        Token startToken =
+                                clusterMetadata.newToken(ringRange.getStart().toString());
+                        Token endToken = clusterMetadata.newToken(ringRange.getEnd().toString());
+                        if (ringRange.isWrapping()) {
+                            // A wrapping range is one that overlaps from the end of the partitioner
+                            // range and its
+                            // start (ie : when the start token of the split is greater than the end
+                            // token)
+                            // We need to generate two queries here : one that goes from the start
+                            // token to the end
+                            // of
+                            // the partitioner range, and the other from the start of the
+                            // partitioner range to the
+                            // end token of the split.
+
+                            addRecordsToOutput(
+                                    session.execute(
+                                            getLowestSplitQuery(
+                                                    query, partitionKey, ringRange.getEnd())),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                            addRecordsToOutput(
+                                    session.execute(
+                                            getHighestSplitQuery(
+                                                    query, partitionKey, ringRange.getStart())),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                        } else {
+                            addRecordsToOutput(
+                                    session.execute(
+                                            preparedStatement
+                                                    .bind()
+                                                    .setToken(0, startToken)
+                                                    .setToken(1, endToken)),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                        }
+                        cassandraSplitState.markRingRangeAsFinished(ringRange);
+                    }
+                    // put the already read split to finished splits
+                    finishedSplits.add(cassandraSplitState.getSplitId());
+                    // for reentrant calls: if fetch is woken up,
+                    // do not reprocess the already processed splits
+                    unprocessedSplits.remove(cassandraSplitState);
+                } catch (Exception ex) {
+                    LOG.error("Error while reading split ", ex);
+                }
+            } else {
+                finishedSplits.add(cassandraSplitState.getSplitId());
+            }
+        }
+        return new RecordsBySplits<>(recordsBySplit, finishedSplits);
+    }
+
+    private String getPartitionKey(Metadata clusterMetadata) {
+        Matcher queryMatcher = Pattern.compile(SELECT_REGEXP).matcher(query);
+        if (!queryMatcher.matches()) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Failed to extract keyspace and table out of the provided query: %s",
+                            query));
+        }
+        String keyspace = queryMatcher.group(1);
+        String table = queryMatcher.group(2);
+        return clusterMetadata.getKeyspace(keyspace).getTable(table).getPartitionKey().stream()
+                .map(ColumnMetadata::getName)
+                .collect(Collectors.joining(","));
+    }
+
+    @Override
+    public void wakeUp() {
+        wakeup.compareAndSet(false, true);
+    }
+
+    @Override
+    public void handleSplitsChanges(SplitsChange<CassandraSplit> splitsChanges) {
+        for (CassandraSplit cassandraSplit : splitsChanges.splits()) {
+            unprocessedSplits.add(cassandraSplit.toSplitState());
+        }
+    }
+
+    @VisibleForTesting
+    static String getHighestSplitQuery(String query, String partitionKey, BigInteger highest) {
+        return generateQuery(
+                query, partitionKey, highest, " (token(%s) >= %d) AND", " WHERE (token(%s) >= %d)");
+    }
+
+    @VisibleForTesting
+    static String getLowestSplitQuery(String query, String partitionKey, BigInteger lowest) {
+        return generateQuery(
+                query, partitionKey, lowest, " (token(%s) < %d) AND", " WHERE (token(%s) < %d)");
+    }
+
+    @VisibleForTesting
+    static String generateRangeQuery(String query, String partitionKey) {
+        return generateQuery(
+                query,
+                partitionKey,
+                null,
+                " (token(%s) >= ?) AND (token(%s) < ?) AND",
+                " WHERE (token(%s) >= ?) AND (token(%s) < ?)");
+    }
+
+    private static String generateQuery(

Review Comment:
   Agree, it needs some doc. I'll add a javadoc.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1327516095

   R: @zentol 
   R: @RyanSkraba you mentioned that you already started to review this PR (when it was in the main flink repo) car you send your review comments ?


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1105819550


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   > it should be a MemorySize
   
   Yes that is what I meant.
   
   > Sounds good. We do something similar in the [mongodb connector](https://github.com/apache/flink-connector-mongodb/blob/main/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.java).
   
   Yes split from Beam 2017 did something similar: it generates CassandraSplits based on Cassandra cluster partitioner and cluster statistics. It estimates the total size of the table using Cassandra system table system.size_estimates and generates estimateTableSize() / maxSplitMemorySize splits with token boundaries. But there is no way to estimate the size of the data with the optional SQL filters without reading the data. So the splits can be smaller than maxSplitMemorySize.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1429539810

   @zentol the PR is not ready for review yet as I changed a lot (split size estimation and fetch state management) it is still under coding, the tests don't pass yet. But still thanks for the confirmation about the general architecture about splits and memory


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1033560949


##########
flink-connector-cassandra/pom.xml:
##########
@@ -78,6 +78,12 @@ under the License.
 			<scope>provided</scope>
 		</dependency>
 
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-base</artifactId>

Review Comment:
   forgot that one :+1: 
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1055349889


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Sets;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            // needs to be mutable

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073637429


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorStateSerializer;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraSplitEnumerator;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSourceReader;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSplitReader;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A bounded source to read from Cassandra and return a collection of entities as {@code
+ * DataStream<Entity>}. An entity is built by Cassandra mapper ({@code
+ * com.datastax.driver.mapping.EntityMapper}) based on a POJO containing annotations (as described
+ * in <a
+ * href="https://docs.datastax.com/en/developer/java-driver/3.11/manual/object_mapper/creating/">
+ * Cassandra object mapper</a>).
+ *
+ * <p>To use it, do the following:
+ *
+ * <pre>{@code
+ * ClusterBuilder clusterBuilder = new ClusterBuilder() {
+ *   @Override
+ *   protected Cluster buildCluster(Cluster.Builder builder) {
+ *     return builder.addContactPointsWithPorts(new InetSocketAddress(HOST,PORT))
+ *                   .withQueryOptions(new QueryOptions().setConsistencyLevel(CL))
+ *                   .withSocketOptions(new SocketOptions()
+ *                   .setConnectTimeoutMillis(CONNECT_TIMEOUT)
+ *                   .setReadTimeoutMillis(READ_TIMEOUT))
+ *                   .build();
+ *   }
+ * };
+ * Source cassandraSource = new CassandraSource(clusterBuilder,
+ *                                              Pojo.class,
+ *                                              "select ... from KEYSPACE.TABLE ...;",
+ *                                              () -> new Mapper.Option[] {Mapper.Option.saveNullFields(true)});
+ *
+ * DataStream<Pojo> stream = env.fromSource(cassandraSource, WatermarkStrategy.noWatermarks(),
+ * "CassandraSource");
+ * }</pre>
+ */
+@PublicEvolving
+public class CassandraSource<OUT>
+        implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
+
+    public static final String CQL_PROHIBITTED_CLAUSES_REGEXP =
+            "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    private static final long serialVersionUID = 7773196541275567433L;

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1098953393


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   > I'll look around. The naming seems off though; it's not a maximum records for a split (which would be more relevant for the SplitGenerator), but more of a batch size.
   
   Agree, it is a batch size with the current code
   
   > You also have to consider how this information can be stored in a checkpoint
   
   Yes, I did not have time to address this point before my vacation but I wanted to validate with you the general approach of these 3 big architecture changes. Now regarding this point: indeed, the ResultSet is just a handler on the fetching process, so storing it as part of the checkpoint does not help in knowing what was already processed after restoration. That is sure that Cassandra does not provide deterministic order because ordering does not scale, so counting does not work either. I looked into `ResultSet#ExecutionInfo` and found no counter or processed Row ids etc... 
   
   So I guess that leaves the SplitGenerator solution you proposed. In that case the conf becomes a max_rows_per_split and not a batch_size anymore. Also, with this solution: 
   
   - We might get more splits than the current parallelism. So each split reader will read more than one split
   
   - As the aim here is to avoid storing too much in memory, I'd prefer that the user conf parameter is in bytes. and I can get my size estimates from Beam 2017 code so that the SplitGenerator provides splits of a given size. Then all the records of each split will be stored in memory to create the needed `RecordsBySplits`. 
   
   WDYT ?
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100135736


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -48,6 +52,30 @@ public CassandraSplitState toSplitState() {
         return new CassandraSplitState(new HashSet<>(ringRanges), splitId());
     }
 
+    public void serialize(ObjectOutputStream objectOutputStream) throws IOException {

Review Comment:
   it makes sense that this method takes a CassandraSplit instance only if it resides in the serializer. So I'll move the method and make it static
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100135736


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -48,6 +52,30 @@ public CassandraSplitState toSplitState() {
         return new CassandraSplitState(new HashSet<>(ringRanges), splitId());
     }
 
+    public void serialize(ObjectOutputStream objectOutputStream) throws IOException {

Review Comment:
   it makes sense that this method takes a CassandraSplit instance only if it resides in the serializer. So I'll rather inline it into the serializer
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100252874


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        this.unprocessedSplits = new HashSet<>();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();

Review Comment:
   Yes I thought about that but the problem I had was when to close the session/cluster. I you prefer this solution, I can override `SourceReaderBase#close()` and close them there + create a factory to initialize cluster and session at the creation of SourceReader before the super(...) calll. Also, I like the design of passing a map function to the emitter from the source reader. 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087944134


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {
+            // allow to interrupt the reading of splits as requested in the API
+            if (wakeup.get()) {
+                break;
+            }
+            if (!cassandraSplitState.isEmpty()) {
+                try {
+                    final Set<RingRange> ringRanges =
+                            cassandraSplitState.getUnprocessedRingRanges();
+                    final String cassandraSplitId = cassandraSplitState.getSplitId();
+
+                    for (RingRange ringRange : ringRanges) {
+                        Token startToken =
+                                clusterMetadata.newToken(ringRange.getStart().toString());
+                        Token endToken = clusterMetadata.newToken(ringRange.getEnd().toString());
+                        if (ringRange.isWrapping()) {
+                            // A wrapping range is one that overlaps from the end of the partitioner
+                            // range and its
+                            // start (ie : when the start token of the split is greater than the end
+                            // token)
+                            // We need to generate two queries here : one that goes from the start
+                            // token to the end
+                            // of
+                            // the partitioner range, and the other from the start of the
+                            // partitioner range to the
+                            // end token of the split.
+
+                            addRecordsToOutput(
+                                    session.execute(
+                                            getLowestSplitQuery(
+                                                    query, partitionKey, ringRange.getEnd())),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                            addRecordsToOutput(
+                                    session.execute(
+                                            getHighestSplitQuery(
+                                                    query, partitionKey, ringRange.getStart())),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                        } else {
+                            addRecordsToOutput(
+                                    session.execute(
+                                            preparedStatement
+                                                    .bind()
+                                                    .setToken(0, startToken)
+                                                    .setToken(1, endToken)),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                        }
+                        cassandraSplitState.markRingRangeAsFinished(ringRange);
+                    }
+                    // put the already read split to finished splits
+                    finishedSplits.add(cassandraSplitState.getSplitId());
+                    // for reentrant calls: if fetch is woken up,
+                    // do not reprocess the already processed splits
+                    unprocessedSplits.remove(cassandraSplitState);
+                } catch (Exception ex) {
+                    LOG.error("Error while reading split ", ex);
+                }
+            } else {
+                finishedSplits.add(cassandraSplitState.getSplitId());
+            }
+        }
+        return new RecordsBySplits<>(recordsBySplit, finishedSplits);
+    }
+
+    private String getPartitionKey(Metadata clusterMetadata) {
+        Matcher queryMatcher = Pattern.compile(SELECT_REGEXP).matcher(query);
+        if (!queryMatcher.matches()) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Failed to extract keyspace and table out of the provided query: %s",
+                            query));
+        }
+        String keyspace = queryMatcher.group(1);
+        String table = queryMatcher.group(2);
+        return clusterMetadata.getKeyspace(keyspace).getTable(table).getPartitionKey().stream()
+                .map(ColumnMetadata::getName)
+                .collect(Collectors.joining(","));
+    }
+
+    @Override
+    public void wakeUp() {
+        wakeup.compareAndSet(false, true);
+    }
+
+    @Override
+    public void handleSplitsChanges(SplitsChange<CassandraSplit> splitsChanges) {
+        for (CassandraSplit cassandraSplit : splitsChanges.splits()) {
+            unprocessedSplits.add(cassandraSplit.toSplitState());
+        }
+    }
+
+    @VisibleForTesting
+    static String getHighestSplitQuery(String query, String partitionKey, BigInteger highest) {
+        return generateQuery(
+                query, partitionKey, highest, " (token(%s) >= %d) AND", " WHERE (token(%s) >= %d)");
+    }
+
+    @VisibleForTesting
+    static String getLowestSplitQuery(String query, String partitionKey, BigInteger lowest) {
+        return generateQuery(
+                query, partitionKey, lowest, " (token(%s) < %d) AND", " WHERE (token(%s) < %d)");
+    }
+
+    @VisibleForTesting
+    static String generateRangeQuery(String query, String partitionKey) {
+        return generateQuery(
+                query,
+                partitionKey,
+                null,
+                " (token(%s) >= ?) AND (token(%s) < ?) AND",
+                " WHERE (token(%s) >= ?) AND (token(%s) < ?)");
+    }
+
+    private static String generateQuery(
+            String query,
+            String partitionKey,
+            @Nullable BigInteger token,
+            String whereFilter,
+            String noWhereFilter) {
+        Matcher queryMatcher = Pattern.compile(SELECT_REGEXP).matcher(query);
+        if (!queryMatcher.matches()) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Failed to extract keyspace and table out of the provided query: %s",
+                            query));
+        }
+        final int whereIndex = query.toLowerCase().indexOf("where");
+        int insertionPoint;
+        String filter;
+        if (whereIndex != -1) {
+            insertionPoint = whereIndex + "where".length();
+            filter =
+                    (token == null)
+                            ? String.format(whereFilter, partitionKey, partitionKey)
+                            : String.format(whereFilter, partitionKey, token);
+        } else {
+            // end of keyspace.table
+            insertionPoint = queryMatcher.end(2);
+            filter =
+                    (token == null)
+                            ? String.format(noWhereFilter, partitionKey, partitionKey)
+                            : String.format(noWhereFilter, partitionKey, token);
+        }
+        return String.format(
+                "%s%s%s",
+                query.substring(0, insertionPoint), filter, query.substring(insertionPoint));
+    }
+
+    /**
+     * This method populates the {@code Map<String, Collection<CassandraRow>> recordsBySplit} map
+     * that is used to create the {@link RecordsBySplits} that are output by the fetch method. It
+     * modifies its {@code output} parameter.
+     */
+    private void addRecordsToOutput(
+            ResultSet resultSet,

Review Comment:
   Alright then.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        this.unprocessedSplits = new HashSet<>();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();

Review Comment:
   Have you considered reusing the same cluster/session across split readers? If they were initialized in the SourceReader we could easily share them with the record emitter (or hide the usage of the cluster object mapper behind a Function that is passed to the emitter).



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089146562


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorStateSerializer;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraSplitEnumerator;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSourceReader;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSplitReader;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A bounded source to read from Cassandra and return a collection of entities as {@code
+ * DataStream<Entity>}. An entity is built by Cassandra mapper ({@code
+ * com.datastax.driver.mapping.EntityMapper}) based on a POJO containing annotations (as described
+ * in <a
+ * href="https://docs.datastax.com/en/developer/java-driver/3.11/manual/object_mapper/creating/">
+ * Cassandra object mapper</a>).
+ *
+ * <p>To use it, do the following:
+ *
+ * <pre>{@code
+ * ClusterBuilder clusterBuilder = new ClusterBuilder() {
+ *   @Override
+ *   protected Cluster buildCluster(Cluster.Builder builder) {
+ *     return builder.addContactPointsWithPorts(new InetSocketAddress(HOST,PORT))
+ *                   .withQueryOptions(new QueryOptions().setConsistencyLevel(CL))
+ *                   .withSocketOptions(new SocketOptions()
+ *                   .setConnectTimeoutMillis(CONNECT_TIMEOUT)
+ *                   .setReadTimeoutMillis(READ_TIMEOUT))
+ *                   .build();
+ *   }
+ * };
+ * Source cassandraSource = new CassandraSource(clusterBuilder,
+ *                                              Pojo.class,
+ *                                              "select ... from KEYSPACE.TABLE ...;",
+ *                                              () -> new Mapper.Option[] {Mapper.Option.saveNullFields(true)});
+ *
+ * DataStream<Pojo> stream = env.fromSource(cassandraSource, WatermarkStrategy.noWatermarks(),
+ * "CassandraSource");
+ * }</pre>
+ */
+@PublicEvolving
+public class CassandraSource<OUT>
+        implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
+
+    public static final String CQL_PROHIBITTED_CLAUSES_REGEXP =
+            "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    private static final long serialVersionUID = 7773196541275567433L;
+
+    private final ClusterBuilder clusterBuilder;
+    private final Class<OUT> pojoClass;
+    private final String query;
+    private final MapperOptions mapperOptions;
+
+    public CassandraSource(
+            ClusterBuilder clusterBuilder,
+            Class<OUT> pojoClass,
+            String query,
+            MapperOptions mapperOptions) {
+        checkNotNull(clusterBuilder, "ClusterBuilder required but not provided");
+        checkNotNull(pojoClass, "POJO class required but not provided");
+        checkQueryValidity(query);
+        this.clusterBuilder = clusterBuilder;
+        ClosureCleaner.clean(clusterBuilder, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+        this.pojoClass = pojoClass;
+        this.query = query;
+        this.mapperOptions = mapperOptions;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return Boundedness.BOUNDED;
+    }
+
+    @Internal
+    @Override
+    public SourceReader<OUT, CassandraSplit> createReader(SourceReaderContext readerContext) {
+        return new CassandraSourceReader<>(
+                readerContext, clusterBuilder, pojoClass, query, mapperOptions);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> createEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext) {
+        return new CassandraSplitEnumerator(enumContext, null, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> restoreEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext,
+            CassandraEnumeratorState enumCheckpoint) {
+        return new CassandraSplitEnumerator(enumContext, enumCheckpoint, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraSplit> getSplitSerializer() {
+        return CassandraSplitSerializer.INSTANCE;
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraEnumeratorState> getEnumeratorCheckpointSerializer() {
+        return CassandraEnumeratorStateSerializer.INSTANCE;
+    }
+
+    @Override
+    public TypeInformation<OUT> getProducedType() {
+        return TypeInformation.of(pojoClass);
+    }
+
+    @VisibleForTesting
+    public static void checkQueryValidity(String query) {
+        checkNotNull(query, "query required but not provided");
+        checkState(
+                query.matches(CassandraSplitReader.SELECT_REGEXP),
+                "query must be of the form select ... from keyspace.table ...;");
+        checkState(
+                !query.matches(CQL_PROHIBITTED_CLAUSES_REGEXP),
+                "query must not contain aggregate or order clauses because they will be done per split. "
+                        + "So they will be incorrect after merging the splits");

Review Comment:
   Clearer indeed :+1:, I'll rephrase to your suggestion. 
   Regarding pushdown I agree, it will be needed in the future but is out of the scope of the current PR. So I suggest we keep aggregates unsupported in this PR and open a ticket to keep track of this future feature.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1086431597


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraRecordEmitter.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.api.connector.source.SourceOutput;
+import org.apache.flink.connector.base.source.reader.RecordEmitter;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.ExecutionInfo;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.mapping.Mapper;
+import com.datastax.driver.mapping.MappingManager;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * {@link RecordEmitter} that converts the {@link CassandraRow} read by the {@link
+ * CassandraSplitReader} to specified POJO and output it while updating splits state. This class
+ * uses the Cassandra driver mapper to map the row to the POJO.
+ *
+ * @param <OUT> type of POJO record to output
+ */
+public class CassandraRecordEmitter<OUT>
+        implements RecordEmitter<CassandraRow, OUT, CassandraSplitState> {
+
+    private final Mapper<OUT> mapper;
+
+    public CassandraRecordEmitter(
+            Class<OUT> pojoClass, ClusterBuilder clusterBuilder, MapperOptions mapperOptions) {
+        // session and cluster are managed at the SplitReader level. So we need to create one
+        // locally here just to me able to create the mapper.
+        final Cluster cluster = clusterBuilder.getCluster();
+        final Session session = cluster.connect();

Review Comment:
   Well, mapping from row to pojo is rather complex and is supported out of the box by Cassandra mapper. Would not it be easier for the user if we guide him in using the mapper rather than leaving him provide a potentially buggy mapping function ? I agree, creating the cluster and session here just for mapper creation is sketchy but I cannot get them in `CassandraRecordEmitter` from the `CassandraSplitReader`. WDYT ?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089015655


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   https://issues.apache.org/jira/browse/FLINK-30802. Can you assign it to me ?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1105819550


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   > it should be a MemorySize
   
   Yes that is what I meant.
   
   > Sounds good. We do something similar in the [mongodb connector](https://github.com/apache/flink-connector-mongodb/blob/main/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.java).
   
   Yes split from Beam 2017 did something similar: it generates CassandraSplits based on Cassandra cluster partitioner and cluster statistics. It estimates the total size of the table using Cassandra system table system.size_estimates and generates estimateTableSize() / maxSplitMemorySize splits with token boundaries. But there is no way to estimate the size of the data with the optional SQL filters without reading the data. So the splits can be smaller than maxSplitMemorySize. The code is still under adaptation to Flink, almost done, should push it soon



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1105868076


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+public class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138381568


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high

Review Comment:
   Well, off course this ratio is totally arbitrary. I just sketched a quick code to serve as a base for discussion about safeguards on the number of splits. I agree with your arguments on limit cases (freedom for the user and dangerousness of sudden threshold crossing). 
   
   Regarding enforcing a minimum size for user defined split size: sure it protects from having too many splits but what if the user defined value is way above the table size? We fallback to one split ?
   
   I'll implement the lazy split generation.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1106073628


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        checkReaderRegistered(subtaskId);
+        final CassandraSplit cassandraSplit = state.getASplit();
+        if (cassandraSplit != null) {
+            LOG.info("Assigning splits to reader {}", subtaskId);
+            enumeratorContext.assignSplit(cassandraSplit, subtaskId);
+        } else {
+            LOG.info(
+                    "No split assigned to reader {} because the enumerator has no unassigned split left",
+                    subtaskId);
+        }
+        if (!state.hasMoreSplits()) {
+            LOG.info(
+                    "No more CassandraSplits to assign. Sending NoMoreSplitsEvent to reader {}.",
+                    subtaskId);
+            enumeratorContext.signalNoMoreSplits(subtaskId);
+        }
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits);
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitionerName = clusterMetadata.getPartitioner();
+        final SplitsGenerator.CassandraPartitioner partitioner =
+                partitionerName.contains(MURMUR3PARTITIONER.className())
+                        ? MURMUR3PARTITIONER
+                        : RANDOMPARTITIONER;
+        return new SplitsGenerator(partitioner).generateSplits(numberOfSplits);

Review Comment:
   we could define safeguard fallbacks to avoid too many splits, too few splits etc...



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1142115352


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -75,54 +77,68 @@ public SplitsGenerator(
      * @return list containing {@code numSplits} CassandraSplits.
      */
     public List<CassandraSplit> generateSplits() {
+        long numSplits = decideOnNumSplits();
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken = partitioner.minToken;
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            BigInteger endToken = startToken.add(increment);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken;
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+            startToken = endToken;
+        }
+        LOG.debug("Generated {} splits : {}", splits.size(), splits);
+        return splits;
+    }
+
+    /**
+     * Determine {@code numSplits} based on the estimation of the target table size and user defined
+     * {@code maxSplitMemorySize}. Add fallbacks when table size is unavailable, too few or too many
+     * splits are calculated.
+     */
+    private long decideOnNumSplits() {
         long numSplits;
         if (maxSplitMemorySize != null) {
+            checkState(

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138381568


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high

Review Comment:
   Well, off course this ratio is totally arbitrary. I just sketched a quick code to serve as a base for discussion about safeguards on the number of splits. I agree with your arguments on limit cases (freedom for the user and dangerousness of sudden threshold crossing). 
   
   I'll implement the lazy split generation.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1363862897

   Also I figured out that the the test of the cassandra tokens split I got from Apache Beam tests only for `RandomPartitioner` (the tokens split method is compatible with both `RandomPartitioner` and `Murmur3Partitioner` as only tokens boundaries vary between the 2) whereas in the actual data split it is compatible only with `Murmur3Partitioner`. So it sould test the tokens split for this cassandra partitionner.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1055279343


##########
flink-connector-cassandra/src/test/resources/log4j2-test.properties:
##########
@@ -18,7 +18,7 @@
 
 # Set root logger level to OFF to not flood build logs
 # set manually to INFO for debugging purposes
-rootLogger.level = OFF

Review Comment:
   ok



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1055347991


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.api.connector.source.SourceSplit;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * {@link SourceSplit} for Cassandra source. A Cassandra split is just a set of {@link RingRange}s
+ * (a range between 2 tokens). Tokens are spread across the Cassandra cluster with each node
+ * managing a share of the token ring. Each split can contain several token ranges in order to
+ * reduce the overhead on Cassandra vnodes.
+ */
+public class CassandraSplit implements SourceSplit, Serializable {
+
+    private final Set<RingRange> ringRanges;
+
+    public CassandraSplit(Set<RingRange> ringRanges) {
+        this.ringRanges = ringRanges;
+    }
+
+    public Set<RingRange> getRingRanges() {
+        return ringRanges;
+    }
+
+    @Override
+    public String splitId() {
+        return ringRanges.toString();

Review Comment:
   Well, `ringRanges` are modified only in the `CassandraSplit` constructor so they are immutable. But the problem could be more on `CassandraSplitState` : its `unprocessedRingRanges` are mutated during the reading of splits. But `CassandraSplitState` is initialized from the `CassandraSplit`s received by the enumerator with `CassandraSplit`'s `splitId` .  `CassandraSplitState`  also serves to restore splits in case of failure and create the corresponding `CassandraSplit` reusing its `splitId`.  So I don't think there is any possible dynamic changing of a `splitId`. 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1142213149


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This class prepares the generation of {@link CassandraSplit}s based on Cassandra cluster
+ * partitioner and cluster statistics. It estimates the total size of the table using Cassandra
+ * system table system.size_estimates.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    @VisibleForTesting public Long minSplitMemorySize = 10_000_000L; // 10 MB

Review Comment:
   I did not this Flink utility. Thanks for the pointer.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1142213149


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This class prepares the generation of {@link CassandraSplit}s based on Cassandra cluster
+ * partitioner and cluster statistics. It estimates the total size of the table using Cassandra
+ * system table system.size_estimates.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    @VisibleForTesting public Long minSplitMemorySize = 10_000_000L; // 10 MB

Review Comment:
   I did not know this Flink utility. Thanks for the pointer.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100252874


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        this.unprocessedSplits = new HashSet<>();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();

Review Comment:
   Yes I thought about that but the problem I had was: who closes the session/cluster ? There is no callback in the SourceReader. That being said, I like the design of passing a map function to the emitter from the source reader. 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073329519


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {

Review Comment:
   Because of `CassandraEnumeratorStateSerializer` needed by `Source#getEnumeratorCheckpointSerializer()`.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073635144


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";

Review Comment:
   no, I'll define the CassandraPartitioner enum and use it here also



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1084247002


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializer.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+/** Serializer for {@link CassandraEnumeratorState}. */
+public class CassandraEnumeratorStateSerializer
+        implements SimpleVersionedSerializer<CassandraEnumeratorState> {
+
+    public static final CassandraEnumeratorStateSerializer INSTANCE =
+            new CassandraEnumeratorStateSerializer();
+    public static final int CURRENT_VERSION = 0;
+
+    private CassandraEnumeratorStateSerializer() { // singleton
+    }
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(CassandraEnumeratorState cassandraEnumeratorState) throws IOException {
+        try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                ObjectOutputStream objectOutputStream =
+                        new ObjectOutputStream(byteArrayOutputStream)) {
+            objectOutputStream.writeObject(cassandraEnumeratorState);

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1086442791


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorStateSerializer;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraSplitEnumerator;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSourceReader;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSplitReader;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A bounded source to read from Cassandra and return a collection of entities as {@code
+ * DataStream<Entity>}. An entity is built by Cassandra mapper ({@code
+ * com.datastax.driver.mapping.EntityMapper}) based on a POJO containing annotations (as described
+ * in <a
+ * href="https://docs.datastax.com/en/developer/java-driver/3.11/manual/object_mapper/creating/">
+ * Cassandra object mapper</a>).
+ *
+ * <p>To use it, do the following:
+ *
+ * <pre>{@code
+ * ClusterBuilder clusterBuilder = new ClusterBuilder() {
+ *   @Override
+ *   protected Cluster buildCluster(Cluster.Builder builder) {
+ *     return builder.addContactPointsWithPorts(new InetSocketAddress(HOST,PORT))
+ *                   .withQueryOptions(new QueryOptions().setConsistencyLevel(CL))
+ *                   .withSocketOptions(new SocketOptions()
+ *                   .setConnectTimeoutMillis(CONNECT_TIMEOUT)
+ *                   .setReadTimeoutMillis(READ_TIMEOUT))
+ *                   .build();
+ *   }
+ * };
+ * Source cassandraSource = new CassandraSource(clusterBuilder,
+ *                                              Pojo.class,
+ *                                              "select ... from KEYSPACE.TABLE ...;",
+ *                                              () -> new Mapper.Option[] {Mapper.Option.saveNullFields(true)});
+ *
+ * DataStream<Pojo> stream = env.fromSource(cassandraSource, WatermarkStrategy.noWatermarks(),
+ * "CassandraSource");
+ * }</pre>
+ */
+@PublicEvolving
+public class CassandraSource<OUT>
+        implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
+
+    public static final String CQL_PROHIBITTED_CLAUSES_REGEXP =
+            "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    private static final long serialVersionUID = 7773196541275567433L;
+
+    private final ClusterBuilder clusterBuilder;
+    private final Class<OUT> pojoClass;
+    private final String query;
+    private final MapperOptions mapperOptions;
+
+    public CassandraSource(
+            ClusterBuilder clusterBuilder,
+            Class<OUT> pojoClass,
+            String query,
+            MapperOptions mapperOptions) {
+        checkNotNull(clusterBuilder, "ClusterBuilder required but not provided");
+        checkNotNull(pojoClass, "POJO class required but not provided");
+        checkQueryValidity(query);
+        this.clusterBuilder = clusterBuilder;
+        ClosureCleaner.clean(clusterBuilder, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+        this.pojoClass = pojoClass;
+        this.query = query;
+        this.mapperOptions = mapperOptions;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return Boundedness.BOUNDED;
+    }
+
+    @Internal
+    @Override
+    public SourceReader<OUT, CassandraSplit> createReader(SourceReaderContext readerContext) {
+        return new CassandraSourceReader<>(
+                readerContext, clusterBuilder, pojoClass, query, mapperOptions);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> createEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext) {
+        return new CassandraSplitEnumerator(enumContext, null, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> restoreEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext,
+            CassandraEnumeratorState enumCheckpoint) {
+        return new CassandraSplitEnumerator(enumContext, enumCheckpoint, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraSplit> getSplitSerializer() {
+        return CassandraSplitSerializer.INSTANCE;
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraEnumeratorState> getEnumeratorCheckpointSerializer() {
+        return CassandraEnumeratorStateSerializer.INSTANCE;
+    }
+
+    @Override
+    public TypeInformation<OUT> getProducedType() {
+        return TypeInformation.of(pojoClass);
+    }
+
+    @VisibleForTesting
+    public static void checkQueryValidity(String query) {
+        checkNotNull(query, "query required but not provided");
+        checkState(
+                query.matches(CassandraSplitReader.SELECT_REGEXP),
+                "query must be of the form select ... from keyspace.table ...;");
+        checkState(
+                !query.matches(CQL_PROHIBITTED_CLAUSES_REGEXP),
+                "query must not contain aggregate or order clauses because they will be done per split. "
+                        + "So they will be incorrect after merging the splits");

Review Comment:
   I don't get what you mean. What I meant here is that any oder or grouping SQL operation will be done at the split level so they will lead to incorrect results at the DataStream level. So I chose to disable them in the source and let the user use the regular methods in DataStream instead.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorStateSerializer;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraSplitEnumerator;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSourceReader;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSplitReader;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A bounded source to read from Cassandra and return a collection of entities as {@code
+ * DataStream<Entity>}. An entity is built by Cassandra mapper ({@code
+ * com.datastax.driver.mapping.EntityMapper}) based on a POJO containing annotations (as described
+ * in <a
+ * href="https://docs.datastax.com/en/developer/java-driver/3.11/manual/object_mapper/creating/">
+ * Cassandra object mapper</a>).
+ *
+ * <p>To use it, do the following:
+ *
+ * <pre>{@code
+ * ClusterBuilder clusterBuilder = new ClusterBuilder() {
+ *   @Override
+ *   protected Cluster buildCluster(Cluster.Builder builder) {
+ *     return builder.addContactPointsWithPorts(new InetSocketAddress(HOST,PORT))
+ *                   .withQueryOptions(new QueryOptions().setConsistencyLevel(CL))
+ *                   .withSocketOptions(new SocketOptions()
+ *                   .setConnectTimeoutMillis(CONNECT_TIMEOUT)
+ *                   .setReadTimeoutMillis(READ_TIMEOUT))
+ *                   .build();
+ *   }
+ * };
+ * Source cassandraSource = new CassandraSource(clusterBuilder,
+ *                                              Pojo.class,
+ *                                              "select ... from KEYSPACE.TABLE ...;",
+ *                                              () -> new Mapper.Option[] {Mapper.Option.saveNullFields(true)});
+ *
+ * DataStream<Pojo> stream = env.fromSource(cassandraSource, WatermarkStrategy.noWatermarks(),
+ * "CassandraSource");
+ * }</pre>
+ */
+@PublicEvolving
+public class CassandraSource<OUT>
+        implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
+
+    public static final String CQL_PROHIBITTED_CLAUSES_REGEXP =
+            "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    private static final long serialVersionUID = 7773196541275567433L;
+
+    private final ClusterBuilder clusterBuilder;
+    private final Class<OUT> pojoClass;
+    private final String query;
+    private final MapperOptions mapperOptions;
+
+    public CassandraSource(
+            ClusterBuilder clusterBuilder,
+            Class<OUT> pojoClass,
+            String query,
+            MapperOptions mapperOptions) {
+        checkNotNull(clusterBuilder, "ClusterBuilder required but not provided");
+        checkNotNull(pojoClass, "POJO class required but not provided");
+        checkQueryValidity(query);
+        this.clusterBuilder = clusterBuilder;
+        ClosureCleaner.clean(clusterBuilder, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+        this.pojoClass = pojoClass;
+        this.query = query;
+        this.mapperOptions = mapperOptions;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return Boundedness.BOUNDED;
+    }
+
+    @Internal
+    @Override
+    public SourceReader<OUT, CassandraSplit> createReader(SourceReaderContext readerContext) {
+        return new CassandraSourceReader<>(
+                readerContext, clusterBuilder, pojoClass, query, mapperOptions);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> createEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext) {
+        return new CassandraSplitEnumerator(enumContext, null, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> restoreEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext,
+            CassandraEnumeratorState enumCheckpoint) {
+        return new CassandraSplitEnumerator(enumContext, enumCheckpoint, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraSplit> getSplitSerializer() {
+        return CassandraSplitSerializer.INSTANCE;
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraEnumeratorState> getEnumeratorCheckpointSerializer() {
+        return CassandraEnumeratorStateSerializer.INSTANCE;
+    }
+
+    @Override
+    public TypeInformation<OUT> getProducedType() {
+        return TypeInformation.of(pojoClass);
+    }
+
+    @VisibleForTesting
+    public static void checkQueryValidity(String query) {
+        checkNotNull(query, "query required but not provided");
+        checkState(
+                query.matches(CassandraSplitReader.SELECT_REGEXP),
+                "query must be of the form select ... from keyspace.table ...;");
+        checkState(
+                !query.matches(CQL_PROHIBITTED_CLAUSES_REGEXP),
+                "query must not contain aggregate or order clauses because they will be done per split. "
+                        + "So they will be incorrect after merging the splits");

Review Comment:
   I don't get what you mean. What I meant here is that any oder or grouping SQL operation will be done at the split level so they will lead to incorrect results at the DataStream level. So I chose to disable them in the source and let the user use the regular methods in DataStream downstream instead.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1105837021


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSourceReaderFactory.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.mapping.Mapper;
+import com.datastax.driver.mapping.MappingManager;
+
+/**
+ * Factory to create {@link CassandraSourceReader}s and allow creating the cluster and the session
+ * objects.

Review Comment:
   :+1:



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1106053839


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializer.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.core.memory.DataInputDeserializer;
+import org.apache.flink.core.memory.DataOutputSerializer;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+/** Serializer for {@link CassandraEnumeratorState}. */
+public class CassandraEnumeratorStateSerializer
+        implements SimpleVersionedSerializer<CassandraEnumeratorState> {
+
+    public static final CassandraEnumeratorStateSerializer INSTANCE =
+            new CassandraEnumeratorStateSerializer();
+    private static final ThreadLocal<DataOutputSerializer> SERIALIZER_CACHE =
+            ThreadLocal.withInitial(() -> new DataOutputSerializer(64));

Review Comment:
   ok. I'll not cache this outputStream. And thanks for the comment on the leak.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137154417


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(

Review Comment:
   :+1: I propose `testGenerateSplitsWithTableUnderSizesMaximumSplitSize` and `testGenerateSplitsWithTableOverSizesMaximumSplitSize` 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137367392


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.CassandraSource;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** tests for query generation and query sanity checks. */
+class CassandraQueryTest {
+
+    private static final Pattern SELECT_PATTERN = Pattern.compile(CassandraSource.SELECT_REGEXP);
+
+    @Test
+    public void testKeySpaceTableExtractionRegexp() {
+        Arrays.asList(
+                        "select field FROM keyspace.table where field = value;",
+                        "select * FROM keyspace.table;",
+                        "select field1, field2 from keyspace.table;",
+                        "select field1, field2 from keyspace.table LIMIT(1000);",
+                        "select field1 from keyspace.table ;",
+                        "select field1 from keyspace.table where field1=1;")
+                .forEach(CassandraQueryTest::assertQueryFormatCorrect);
+
+        Arrays.asList(
+                        "select field1 from table;", // missing keyspace
+                        "select field1 from keyspace.table" // missing ";"
+                        )

Review Comment:
   yes but `;` is kind of optional (tolerance) depending on the client CQL tool that sent the request so it is a common mistake
   
   But fair enough for the corner case, I guess an excess of protection is not bad.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138381568


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high

Review Comment:
   Well, off course this ratio is totally arbitrary. I just sketched a quick code to serve as a base for discussion about safeguards on the number of splits. I agree with your arguments on limit cases (freedom for the user and dangerousness of sudden threshold crossing). 
   
   I'll implement the lazy splits.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1135383543


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.cassandra.service.StorageServiceMBean;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableList;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableMap;
+
+import javax.management.JMX;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.rmi.server.RMISocketFactory;
+import java.util.Map;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final String JMX_URL = "service:jmx:rmi:///jndi/rmi://%s:%d/jmxrmi";
+    private static final String STORAGE_SERVICE_MBEAN =
+            "org.apache.cassandra.db:type=StorageService";
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+    private final String dockerHostIp;
+    private final int jmxPort;
+
+    private Cluster cluster;
+    private Session session;
+    private ClusterBuilder clusterBuilder;
+
+    public CassandraTestEnvironment() {
+        try (ServerSocket s = new ServerSocket(0)) {
+            // use fixed free random port for JMX
+            jmxPort = s.getLocalPort();
+            // resolve IP of docker host to establish JMX connection
+            dockerHostIp = InetAddress.getLocalHost().getHostAddress();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+
+        cassandraContainer = new CassandraContainer(DOCKER_CASSANDRA_IMAGE);
+        // use a fixed port mapping for JMX, it doesn't work well with mapped ports
+        cassandraContainer.setPortBindings(ImmutableList.of(jmxPort + ":" + jmxPort));
+        // more generous timeouts and remote JMX configuration
+        addJavaOpts(
+                cassandraContainer,
+                "-Dcassandra.request_timeout_in_ms=30000",
+                "-Dcassandra.read_request_timeout_in_ms=15000",
+                "-Dcassandra.write_request_timeout_in_ms=6000",
+                "-Dcassandra.jmx.remote.port=" + jmxPort,
+                "-Dcom.sun.management.jmxremote.rmi.port=" + jmxPort,
+                "-Djava.rmi.server.hostname=" + dockerHostIp,
+                "-Dcom.sun.management.jmxremote.host=" + dockerHostIp);
+    }
+
+    @Override
+    public void startUp() throws Exception {
+        startEnv();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        stopEnv();
+    }
+
+    private static void addJavaOpts(GenericContainer<?> container, String... opts) {
+        String jvmOpts = container.getEnvMap().getOrDefault("JVM_OPTS", "");
+        container.withEnv("JVM_OPTS", jvmOpts + " " + StringUtils.join(opts, " "));
+    }
+
+    private void startEnv() throws Exception {
+        // configure container start to wait until cassandra is ready to receive queries
+        cassandraContainer.waitingFor(new CassandraQueryWaitStrategy());
+        // start with retrials
+        cassandraContainer.start();
+        cassandraContainer.followOutput(
+                new Slf4jLogConsumer(LOG),
+                OutputFrame.OutputType.END,
+                OutputFrame.OutputType.STDERR,
+                OutputFrame.OutputType.STDOUT);
+
+        cluster = cassandraContainer.getCluster();
+        clusterBuilder =
+                createBuilderWithConsistencyLevel(
+                        ConsistencyLevel.ONE,
+                        cassandraContainer.getHost(),
+                        cassandraContainer.getMappedPort(CQL_PORT));
+
+        session = cluster.connect();
+        session.execute(requestWithTimeout(CREATE_KEYSPACE_QUERY));
+        // create a dedicated table for split size tests (to avoid having to flush with each test)
+        insertTestDataForSplitSizeTests();
+    }
+
+    private void insertTestDataForSplitSizeTests() throws Exception {
+        session.execute(requestWithTimeout(CREATE_SPLITS_TABLE_QUERY));
+        for (int i = 0; i < NB_SPLITS_RECORDS; i++) {
+            session.execute(requestWithTimeout(String.format(INSERT_INTO_FLINK_SPLITS, i, i)));
+        }
+        flushMemTables(SPLITS_TABLE);
+    }
+
+    private void stopEnv() {
+
+        if (session != null) {
+            session.close();
+        }
+        if (cluster != null) {
+            cluster.close();
+        }
+        cassandraContainer.stop();
+    }
+
+    private ClusterBuilder createBuilderWithConsistencyLevel(
+            ConsistencyLevel consistencyLevel, String host, int port) {
+        return new ClusterBuilder() {
+            @Override
+            protected Cluster buildCluster(Cluster.Builder builder) {
+                return builder.addContactPointsWithPorts(new InetSocketAddress(host, port))
+                        .withQueryOptions(
+                                new QueryOptions()
+                                        .setConsistencyLevel(consistencyLevel)
+                                        .setSerialConsistencyLevel(ConsistencyLevel.LOCAL_SERIAL))
+                        .withSocketOptions(
+                                new SocketOptions()
+                                        // default timeout x 3
+                                        .setConnectTimeoutMillis(15000)
+                                        // default timeout x3 and higher than
+                                        // request_timeout_in_ms at the cluster level
+                                        .setReadTimeoutMillis(READ_TIMEOUT_MILLIS))
+                        .withoutJMXReporting()
+                        .withoutMetrics()
+                        .build();
+            }
+        };
+    }
+
+    /**
+     * Force the flush of cassandra memTables to SSTables in order to update size_estimates. This
+     * flush method is what official Cassandra NoteTool does. It is needed for the tests because we
+     * just inserted records, we need to force cassandra to update size_estimates system table.
+     */

Review Comment:
   ```suggestion
   ```



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.cassandra.service.StorageServiceMBean;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableList;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableMap;
+
+import javax.management.JMX;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.rmi.server.RMISocketFactory;
+import java.util.Map;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final String JMX_URL = "service:jmx:rmi:///jndi/rmi://%s:%d/jmxrmi";
+    private static final String STORAGE_SERVICE_MBEAN =
+            "org.apache.cassandra.db:type=StorageService";
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+    private final String dockerHostIp;
+    private final int jmxPort;
+
+    private Cluster cluster;
+    private Session session;
+    private ClusterBuilder clusterBuilder;
+
+    public CassandraTestEnvironment() {
+        try (ServerSocket s = new ServerSocket(0)) {
+            // use fixed free random port for JMX
+            jmxPort = s.getLocalPort();
+            // resolve IP of docker host to establish JMX connection
+            dockerHostIp = InetAddress.getLocalHost().getHostAddress();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+

Review Comment:
   ```suggestion
   ```
   This wasn't safe anyway.



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.cassandra.service.StorageServiceMBean;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableList;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableMap;
+
+import javax.management.JMX;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.rmi.server.RMISocketFactory;
+import java.util.Map;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final String JMX_URL = "service:jmx:rmi:///jndi/rmi://%s:%d/jmxrmi";
+    private static final String STORAGE_SERVICE_MBEAN =
+            "org.apache.cassandra.db:type=StorageService";

Review Comment:
   ```suggestion
   ```



##########
flink-connector-cassandra/pom.xml:
##########
@@ -180,7 +188,17 @@ under the License.
 			<scope>provided</scope>
 		</dependency>
 
-		<!-- Test dependencies -->
+	<!-- Test dependencies -->
+
+		<!-- for JMX MBean invocation to flush mem tables.
+		Alternative is cassandra-all artifact which is huge.
+		palantir-cassandra-jmx-api is ASF v2 -->
+		<dependency>
+			<groupId>com.palantir.cassandra</groupId>
+			<artifactId>palantir-cassandra-jmx-api</artifactId>
+			<version>${palantir-cassandra-jmx-api.version}</version>
+			<scope>test</scope>
+		</dependency>

Review Comment:
   ```suggestion
   ```



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.cassandra.service.StorageServiceMBean;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableList;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableMap;
+
+import javax.management.JMX;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.rmi.server.RMISocketFactory;
+import java.util.Map;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final String JMX_URL = "service:jmx:rmi:///jndi/rmi://%s:%d/jmxrmi";
+    private static final String STORAGE_SERVICE_MBEAN =
+            "org.apache.cassandra.db:type=StorageService";
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+    private final String dockerHostIp;
+    private final int jmxPort;
+
+    private Cluster cluster;
+    private Session session;
+    private ClusterBuilder clusterBuilder;
+
+    public CassandraTestEnvironment() {
+        try (ServerSocket s = new ServerSocket(0)) {
+            // use fixed free random port for JMX
+            jmxPort = s.getLocalPort();
+            // resolve IP of docker host to establish JMX connection
+            dockerHostIp = InetAddress.getLocalHost().getHostAddress();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+
+        cassandraContainer = new CassandraContainer(DOCKER_CASSANDRA_IMAGE);
+        // use a fixed port mapping for JMX, it doesn't work well with mapped ports
+        cassandraContainer.setPortBindings(ImmutableList.of(jmxPort + ":" + jmxPort));
+        // more generous timeouts and remote JMX configuration
+        addJavaOpts(
+                cassandraContainer,
+                "-Dcassandra.request_timeout_in_ms=30000",
+                "-Dcassandra.read_request_timeout_in_ms=15000",
+                "-Dcassandra.write_request_timeout_in_ms=6000",
+                "-Dcassandra.jmx.remote.port=" + jmxPort,
+                "-Dcom.sun.management.jmxremote.rmi.port=" + jmxPort,
+                "-Djava.rmi.server.hostname=" + dockerHostIp,
+                "-Dcom.sun.management.jmxremote.host=" + dockerHostIp);
+    }
+
+    @Override
+    public void startUp() throws Exception {
+        startEnv();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        stopEnv();
+    }
+
+    private static void addJavaOpts(GenericContainer<?> container, String... opts) {
+        String jvmOpts = container.getEnvMap().getOrDefault("JVM_OPTS", "");
+        container.withEnv("JVM_OPTS", jvmOpts + " " + StringUtils.join(opts, " "));
+    }
+
+    private void startEnv() throws Exception {
+        // configure container start to wait until cassandra is ready to receive queries
+        cassandraContainer.waitingFor(new CassandraQueryWaitStrategy());
+        // start with retrials
+        cassandraContainer.start();
+        cassandraContainer.followOutput(
+                new Slf4jLogConsumer(LOG),
+                OutputFrame.OutputType.END,
+                OutputFrame.OutputType.STDERR,
+                OutputFrame.OutputType.STDOUT);
+
+        cluster = cassandraContainer.getCluster();
+        clusterBuilder =
+                createBuilderWithConsistencyLevel(
+                        ConsistencyLevel.ONE,
+                        cassandraContainer.getHost(),
+                        cassandraContainer.getMappedPort(CQL_PORT));
+
+        session = cluster.connect();
+        session.execute(requestWithTimeout(CREATE_KEYSPACE_QUERY));
+        // create a dedicated table for split size tests (to avoid having to flush with each test)
+        insertTestDataForSplitSizeTests();
+    }
+
+    private void insertTestDataForSplitSizeTests() throws Exception {
+        session.execute(requestWithTimeout(CREATE_SPLITS_TABLE_QUERY));
+        for (int i = 0; i < NB_SPLITS_RECORDS; i++) {
+            session.execute(requestWithTimeout(String.format(INSERT_INTO_FLINK_SPLITS, i, i)));
+        }
+        flushMemTables(SPLITS_TABLE);
+    }
+
+    private void stopEnv() {
+
+        if (session != null) {
+            session.close();
+        }
+        if (cluster != null) {
+            cluster.close();
+        }
+        cassandraContainer.stop();
+    }
+
+    private ClusterBuilder createBuilderWithConsistencyLevel(
+            ConsistencyLevel consistencyLevel, String host, int port) {
+        return new ClusterBuilder() {
+            @Override
+            protected Cluster buildCluster(Cluster.Builder builder) {
+                return builder.addContactPointsWithPorts(new InetSocketAddress(host, port))
+                        .withQueryOptions(
+                                new QueryOptions()
+                                        .setConsistencyLevel(consistencyLevel)
+                                        .setSerialConsistencyLevel(ConsistencyLevel.LOCAL_SERIAL))
+                        .withSocketOptions(
+                                new SocketOptions()
+                                        // default timeout x 3
+                                        .setConnectTimeoutMillis(15000)
+                                        // default timeout x3 and higher than
+                                        // request_timeout_in_ms at the cluster level
+                                        .setReadTimeoutMillis(READ_TIMEOUT_MILLIS))
+                        .withoutJMXReporting()
+                        .withoutMetrics()
+                        .build();
+            }
+        };
+    }
+
+    /**
+     * Force the flush of cassandra memTables to SSTables in order to update size_estimates. This
+     * flush method is what official Cassandra NoteTool does. It is needed for the tests because we
+     * just inserted records, we need to force cassandra to update size_estimates system table.
+     */
+    void flushMemTables(String table) throws Exception {
+        JMXServiceURL url = new JMXServiceURL(String.format(JMX_URL, dockerHostIp, jmxPort));
+        Map<String, Object> env =
+                ImmutableMap.of(
+                        "com.sun.jndi.rmi.factory.socket",
+                        RMISocketFactory.getDefaultSocketFactory()); // connection without ssl
+        try (JMXConnector jmxConnector = JMXConnectorFactory.connect(url, env)) {
+            MBeanServerConnection mBeanServerConnection = jmxConnector.getMBeanServerConnection();
+            ObjectName objectName = new ObjectName(STORAGE_SERVICE_MBEAN);
+            StorageServiceMBean mBeanProxy =
+                    JMX.newMBeanProxy(mBeanServerConnection, objectName, StorageServiceMBean.class);
+            mBeanProxy.forceKeyspaceFlush(KEYSPACE, table);
+        }

Review Comment:
   
   ```suggestion
           cassandraContainer.execInContainer("nodetool",
                   "flush",
                   KEYSPACE,
                   table);
   ```



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.cassandra.service.StorageServiceMBean;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableList;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableMap;
+
+import javax.management.JMX;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.rmi.server.RMISocketFactory;
+import java.util.Map;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final String JMX_URL = "service:jmx:rmi:///jndi/rmi://%s:%d/jmxrmi";
+    private static final String STORAGE_SERVICE_MBEAN =
+            "org.apache.cassandra.db:type=StorageService";
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+    private final String dockerHostIp;
+    private final int jmxPort;
+
+    private Cluster cluster;
+    private Session session;
+    private ClusterBuilder clusterBuilder;
+
+    public CassandraTestEnvironment() {
+        try (ServerSocket s = new ServerSocket(0)) {
+            // use fixed free random port for JMX
+            jmxPort = s.getLocalPort();
+            // resolve IP of docker host to establish JMX connection
+            dockerHostIp = InetAddress.getLocalHost().getHostAddress();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+
+        cassandraContainer = new CassandraContainer(DOCKER_CASSANDRA_IMAGE);
+        // use a fixed port mapping for JMX, it doesn't work well with mapped ports
+        cassandraContainer.setPortBindings(ImmutableList.of(jmxPort + ":" + jmxPort));
+        // more generous timeouts and remote JMX configuration
+        addJavaOpts(
+                cassandraContainer,
+                "-Dcassandra.request_timeout_in_ms=30000",
+                "-Dcassandra.read_request_timeout_in_ms=15000",
+                "-Dcassandra.write_request_timeout_in_ms=6000",
+                "-Dcassandra.jmx.remote.port=" + jmxPort,
+                "-Dcom.sun.management.jmxremote.rmi.port=" + jmxPort,
+                "-Djava.rmi.server.hostname=" + dockerHostIp,
+                "-Dcom.sun.management.jmxremote.host=" + dockerHostIp);

Review Comment:
   ```suggestion
                   "-Dcassandra.write_request_timeout_in_ms=6000");
   ```



##########
flink-connector-cassandra/pom.xml:
##########
@@ -53,6 +53,7 @@ under the License.
 		<!--driver 3.x works with 3.x and 4.x versions of Cassandra but driver 4.x is a complete refactoring with different API-->
 		<driver.version>3.11.2</driver.version>
 		<guava.version>19.0</guava.version>
+		<palantir-cassandra-jmx-api.version>2.2.18-1.108.0</palantir-cassandra-jmx-api.version>

Review Comment:
   ```suggestion
   ```



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.cassandra.service.StorageServiceMBean;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableList;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableMap;
+
+import javax.management.JMX;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.rmi.server.RMISocketFactory;
+import java.util.Map;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final String JMX_URL = "service:jmx:rmi:///jndi/rmi://%s:%d/jmxrmi";
+    private static final String STORAGE_SERVICE_MBEAN =
+            "org.apache.cassandra.db:type=StorageService";
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+    private final String dockerHostIp;
+    private final int jmxPort;

Review Comment:
   ```suggestion
   ```



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1406744295

   I have implified a lot SplitsGenerator: it is now based only on Cassandra min and max tokens per partitioner and generates (min,x][x, max) for 2 splits for example. It now supports all the Cassandra partitioners. Now a Cassandra split is just a ring range (a split of cassandra token range). This also simplifies a lot the range query generation.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089146562


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorStateSerializer;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraSplitEnumerator;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSourceReader;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSplitReader;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A bounded source to read from Cassandra and return a collection of entities as {@code
+ * DataStream<Entity>}. An entity is built by Cassandra mapper ({@code
+ * com.datastax.driver.mapping.EntityMapper}) based on a POJO containing annotations (as described
+ * in <a
+ * href="https://docs.datastax.com/en/developer/java-driver/3.11/manual/object_mapper/creating/">
+ * Cassandra object mapper</a>).
+ *
+ * <p>To use it, do the following:
+ *
+ * <pre>{@code
+ * ClusterBuilder clusterBuilder = new ClusterBuilder() {
+ *   @Override
+ *   protected Cluster buildCluster(Cluster.Builder builder) {
+ *     return builder.addContactPointsWithPorts(new InetSocketAddress(HOST,PORT))
+ *                   .withQueryOptions(new QueryOptions().setConsistencyLevel(CL))
+ *                   .withSocketOptions(new SocketOptions()
+ *                   .setConnectTimeoutMillis(CONNECT_TIMEOUT)
+ *                   .setReadTimeoutMillis(READ_TIMEOUT))
+ *                   .build();
+ *   }
+ * };
+ * Source cassandraSource = new CassandraSource(clusterBuilder,
+ *                                              Pojo.class,
+ *                                              "select ... from KEYSPACE.TABLE ...;",
+ *                                              () -> new Mapper.Option[] {Mapper.Option.saveNullFields(true)});
+ *
+ * DataStream<Pojo> stream = env.fromSource(cassandraSource, WatermarkStrategy.noWatermarks(),
+ * "CassandraSource");
+ * }</pre>
+ */
+@PublicEvolving
+public class CassandraSource<OUT>
+        implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
+
+    public static final String CQL_PROHIBITTED_CLAUSES_REGEXP =
+            "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    private static final long serialVersionUID = 7773196541275567433L;
+
+    private final ClusterBuilder clusterBuilder;
+    private final Class<OUT> pojoClass;
+    private final String query;
+    private final MapperOptions mapperOptions;
+
+    public CassandraSource(
+            ClusterBuilder clusterBuilder,
+            Class<OUT> pojoClass,
+            String query,
+            MapperOptions mapperOptions) {
+        checkNotNull(clusterBuilder, "ClusterBuilder required but not provided");
+        checkNotNull(pojoClass, "POJO class required but not provided");
+        checkQueryValidity(query);
+        this.clusterBuilder = clusterBuilder;
+        ClosureCleaner.clean(clusterBuilder, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+        this.pojoClass = pojoClass;
+        this.query = query;
+        this.mapperOptions = mapperOptions;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return Boundedness.BOUNDED;
+    }
+
+    @Internal
+    @Override
+    public SourceReader<OUT, CassandraSplit> createReader(SourceReaderContext readerContext) {
+        return new CassandraSourceReader<>(
+                readerContext, clusterBuilder, pojoClass, query, mapperOptions);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> createEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext) {
+        return new CassandraSplitEnumerator(enumContext, null, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> restoreEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext,
+            CassandraEnumeratorState enumCheckpoint) {
+        return new CassandraSplitEnumerator(enumContext, enumCheckpoint, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraSplit> getSplitSerializer() {
+        return CassandraSplitSerializer.INSTANCE;
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraEnumeratorState> getEnumeratorCheckpointSerializer() {
+        return CassandraEnumeratorStateSerializer.INSTANCE;
+    }
+
+    @Override
+    public TypeInformation<OUT> getProducedType() {
+        return TypeInformation.of(pojoClass);
+    }
+
+    @VisibleForTesting
+    public static void checkQueryValidity(String query) {
+        checkNotNull(query, "query required but not provided");
+        checkState(
+                query.matches(CassandraSplitReader.SELECT_REGEXP),
+                "query must be of the form select ... from keyspace.table ...;");
+        checkState(
+                !query.matches(CQL_PROHIBITTED_CLAUSES_REGEXP),
+                "query must not contain aggregate or order clauses because they will be done per split. "
+                        + "So they will be incorrect after merging the splits");

Review Comment:
   Ok I'll rephrase to your suggestion. 
   Regarding pushdown I agree, it will be needed in the future but is out of the scope of the current PR. So I suggest we keep aggregates unsupported in this PR and open a ticket to keep track of this future feature.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089162874


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits, enumeratorContext.currentParallelism());
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitioner = clusterMetadata.getPartitioner();
+        final SplitsGenerator splitsGenerator = new SplitsGenerator(partitioner);
+        if (MURMUR3PARTITIONER.equals(partitioner)) {
+            LOG.info("Murmur3Partitioner detected, splitting");
+            List<BigInteger> tokens =
+                    clusterMetadata.getTokenRanges().stream()
+                            .map(
+                                    tokenRange ->
+                                            new BigInteger(
+                                                    tokenRange.getEnd().getValue().toString()))
+                            .collect(Collectors.toList());
+            return splitsGenerator.generateSplits(numberOfSplits, tokens);
+        } else {
+            // Murmur3Partitioner is the default and recommended partitioner for Cassandra 1.2+
+            // see
+            // https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerAbout.html
+            LOG.warn(
+                    "The current Cassandra partitioner is {}, only Murmur3Partitioner is supported "
+                            + "for splitting, using an single split",
+                    partitioner);
+            return splitsGenerator.generateSplits(1, Collections.emptyList());
+        }
+    }
+
+    @Override
+    public void addSplitsBack(List<CassandraSplit> splits, int subtaskId) {
+        LOG.info("Add {} splits back to CassandraSplitEnumerator.", splits.size());
+        state.addNewSplits(splits, enumeratorContext.currentParallelism());
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        LOG.info("Adding reader {} to CassandraSplitEnumerator.", subtaskId);
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    private void assignUnprocessedSplitsToReader(int readerId) {
+        checkReaderRegistered(readerId);
+
+        final Set<CassandraSplit> splitsForReader = state.getSplitsForReader(readerId);

Review Comment:
   IMHO when I override a method such as `handleSplitRequest()` that a framework provides, I'm kind of expecting it to be called by the framework and not having to call it myself. The doc does not mention that we need to call it so I suggest we should open a ticket to improve the framework on this side. I'll open the ticket, feel free to comment it or close it if you disagree.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089214272


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        this.unprocessedSplits = new HashSet<>();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087607875


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set

Review Comment:
   I've pushed an impl that assigns only a single split to each reader and this removed the need for concurrent set



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089034416


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraRow.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+
+import com.datastax.driver.core.ExecutionInfo;
+import com.datastax.driver.core.Row;
+
+/**
+ * Wrapper for Cassandra {@link Row} that stores associated {@link RingRange} to be able to update
+ * split states. It also stores {@link ExecutionInfo} Cassandra statistics about the query execution
+ * that produced this row.
+ */
+public class CassandraRow {
+
+    private final Row row;
+    private final RingRange associatedRingRange;
+    private final ExecutionInfo executionInfo;

Review Comment:
   Now that we store the `ResultSet` in `CassandraSplitState` to keep track of the fetch state, we no more need this `CassandraRow` wrapper as will have the `ExecutionInfo` in the `Resultset` with the call to `RecordEmitter#emitRecord()`



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1033560949


##########
flink-connector-cassandra/pom.xml:
##########
@@ -78,6 +78,12 @@ under the License.
 			<scope>provided</scope>
 		</dependency>
 
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-base</artifactId>

Review Comment:
   :+1: 
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101496694


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -48,6 +52,30 @@ public CassandraSplitState toSplitState() {
         return new CassandraSplitState(new HashSet<>(ringRanges), splitId());
     }
 
+    public void serialize(ObjectOutputStream objectOutputStream) throws IOException {

Review Comment:
   > For symmetry I'd prefer this method to be static and accept a CassandraSplit instance 
   
   I made the serializer take an OutputStream to avoid creating an OutputStream for each split in the EnumeratorState serialization which would create a stream header for each split. I managed to solve this issue by caching the outputStream like it was done in the file source
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101601871


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and Flink
+ * source parallelism.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final CassandraPartitioner partitioner;
+
+    public SplitsGenerator(CassandraPartitioner partitioner) {
+        this.partitioner = partitioner;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the ring.
+     *
+     * @param numSplits requested number of splits
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long numSplits) {
+        if (numSplits == 1) {
+            return Collections.singletonList(
+                    new CassandraSplit(partitioner.minToken(), partitioner.maxToken()));
+        }
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger splitSize =
+                (partitioner.ringSize()).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken, endToken = partitioner.minToken();
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            startToken = endToken;
+            endToken = startToken.add(splitSize);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken();
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+        }

Review Comment:
   agree



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1098953393


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   > I'll look around. The naming seems off though; it's not a maximum records for a split (which would be more relevant for the SplitGenerator), but more of a batch size.
   
   Agree, it is a batch size with the current code
   
   > You also have to consider how this information can be stored in a checkpoint
   
   Yes, I did not have time to address this point before my vacation but I wanted to validate with you the general approach of these 3 big architecture changes. Now regarding this point: indeed, the ResultSet is just a handler on the fetching process, so storing it as part of the checkpoint does not help in knowing what was already processed after restoration. That is sure that Cassandra does not provide deterministic order because ordering does not scale, so counting does not work either. I looked into `ResultSet#ExecutionInfo` and found no counter or processed Row ids etc... 
   
   So I guess that leaves the SplitGenerator solution you proposed. In that case the conf becomes a max_rows_per_split and not a batch_size anymore. Also, with this solution: 
   
   - We might get more splits than the current parallelism. So each split reader will read more than one split
   
   - The aim here is to avoid storing too much in memory so I'd prefer that the user conf parameter is in bytes. and I can get my size estimates from Beam 2017 code so that the SplitGenerator provides splits of a given size. Then all the records of each split will be stored in memory to create the needed `RecordsBySplits`. 
   
   WDYT ?
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101701294


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitState.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import com.datastax.driver.core.ResultSet;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+
+/**
+ * Mutable {@link CassandraSplit} that keeps track of the reading process of the associated split.
+ */
+public class CassandraSplitState {
+    private final CassandraSplit cassandraSplit;
+    // Cassandra ResultSet is paginated, a new page is read only if all the records of the previous
+    // one were consumed. fetch() can be interrupted so we use the resultSet to keep track of the
+    // reading process.
+    // It is null when reading has not started (before fetch is called on the split).
+    @Nullable private ResultSet resultSet;

Review Comment:
   As discussed in the other comment ResultSet is just a handle so the status of the read will not be part of the checkpoint leading to a re-read of the already output data indeed. The only way is to manage the memory size of the split at the enumerator level and either output all the split or not a all. That way in case of interrupted fetch nothing will be output and the split could be read again from the beginning after recovery leading to no duplicates.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100135736


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -48,6 +52,30 @@ public CassandraSplitState toSplitState() {
         return new CassandraSplitState(new HashSet<>(ringRanges), splitId());
     }
 
+    public void serialize(ObjectOutputStream objectOutputStream) throws IOException {

Review Comment:
   it makes sense that this method takes a CassandraSplit instance only if it resides in the serializer. So rather inline it into the serializer
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1104204559


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.CassandraSource;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** tests for query generation and query sanity checks. */
+class CassandraQueryTest {
+
+    @Test
+    public void testKeySpaceTableExtractionRegexp() {
+        final Pattern pattern = Pattern.compile(CassandraSplitReader.SELECT_REGEXP);
+        Matcher matcher;
+        matcher = pattern.matcher("SELECT field FROM keyspace.table where field = value;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("SELECT * FROM keyspace.table;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1, field2 from keyspace.table;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1, field2 from keyspace.table LIMIT(1000);");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1 from keyspace.table ;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1 from keyspace.table where field1=1;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1 from table;"); // missing keyspace
+        assertThat(matcher.matches()).isFalse();
+
+        matcher = pattern.matcher("select field1 from keyspace.table"); // missing ";"
+        assertThat(matcher.matches()).isFalse();

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] RyanSkraba commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
RyanSkraba commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1344580035

   Hey @echauchot -- I've had some delays, but I'm still looking at this!  I'm bumping to the top of my to do list.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073332349


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {
+        return (start.compareTo(end) >= 0)

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073555595


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            return Collections.singletonList(
+                    new CassandraSplit(Collections.singleton(totalRingRange)));
+        }
+        int tokenRangeCount = ringTokens.size();
+
+        List<RingRange> ringRanges = new ArrayList<>();
+        for (int i = 0; i < tokenRangeCount; i++) {
+            BigInteger start = ringTokens.get(i);
+            BigInteger stop = ringTokens.get((i + 1) % tokenRangeCount);
+
+            if (isNotInRange(start) || isNotInRange(stop)) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s) not in range of %s", start, stop, partitioner));
+            }
+            if (start.equals(stop) && tokenRangeCount != 1) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s): two nodes have the same token", start, stop));
+            }
+
+            BigInteger rangeSize = stop.subtract(start);
+            if (rangeSize.compareTo(BigInteger.ZERO) <= 0) {
+                // wrap around case
+                rangeSize = rangeSize.add(this.rangeSize);
+            }
+
+            // the below, in essence, does this:
+            // splitCount = Maths.ceil((rangeSize / cluster range size) * totalSplitCount)
+            BigInteger[] splitCountAndRemainder =

Review Comment:
   See my general comments about Apache Beam Cassandra partitionner



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1081173791


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitSerializer.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+/** Serializer for {@link CassandraSplit}. */
+public class CassandraSplitSerializer implements SimpleVersionedSerializer<CassandraSplit> {
+
+    public static final CassandraSplitSerializer INSTANCE = new CassandraSplitSerializer();
+
+    public static final int CURRENT_VERSION = 0;
+
+    private CassandraSplitSerializer() {}
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(CassandraSplit cassandraSplit) throws IOException {
+        try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                ObjectOutputStream objectOutputStream =
+                        new ObjectOutputStream(byteArrayOutputStream)) {
+            objectOutputStream.writeObject(cassandraSplit);

Review Comment:
   Try to avoid it whenever possible. A lot of existing cases in Flink are just technical debt, others it just slipped through.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1081463284


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitSerializer.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+/** Serializer for {@link CassandraSplit}. */
+public class CassandraSplitSerializer implements SimpleVersionedSerializer<CassandraSplit> {
+
+    public static final CassandraSplitSerializer INSTANCE = new CassandraSplitSerializer();
+
+    public static final int CURRENT_VERSION = 0;
+
+    private CassandraSplitSerializer() {}
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(CassandraSplit cassandraSplit) throws IOException {
+        try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                ObjectOutputStream objectOutputStream =
+                        new ObjectOutputStream(byteArrayOutputStream)) {
+            objectOutputStream.writeObject(cassandraSplit);

Review Comment:
   Makes sense



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1086513855


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits, enumeratorContext.currentParallelism());
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitioner = clusterMetadata.getPartitioner();
+        final SplitsGenerator splitsGenerator = new SplitsGenerator(partitioner);
+        if (MURMUR3PARTITIONER.equals(partitioner)) {
+            LOG.info("Murmur3Partitioner detected, splitting");
+            List<BigInteger> tokens =
+                    clusterMetadata.getTokenRanges().stream()
+                            .map(
+                                    tokenRange ->
+                                            new BigInteger(
+                                                    tokenRange.getEnd().getValue().toString()))
+                            .collect(Collectors.toList());
+            return splitsGenerator.generateSplits(numberOfSplits, tokens);
+        } else {
+            // Murmur3Partitioner is the default and recommended partitioner for Cassandra 1.2+
+            // see
+            // https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerAbout.html
+            LOG.warn(
+                    "The current Cassandra partitioner is {}, only Murmur3Partitioner is supported "
+                            + "for splitting, using an single split",
+                    partitioner);
+            return splitsGenerator.generateSplits(1, Collections.emptyList());
+        }
+    }
+
+    @Override
+    public void addSplitsBack(List<CassandraSplit> splits, int subtaskId) {
+        LOG.info("Add {} splits back to CassandraSplitEnumerator.", splits.size());
+        state.addNewSplits(splits, enumeratorContext.currentParallelism());
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        LOG.info("Adding reader {} to CassandraSplitEnumerator.", subtaskId);
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    private void assignUnprocessedSplitsToReader(int readerId) {
+        checkReaderRegistered(readerId);
+
+        final Set<CassandraSplit> splitsForReader = state.getSplitsForReader(readerId);

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1088816929


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   > from all splits into memory
   
   Well, in 99% of the cases, `unprocessedSplits` will contain only a single split as the enumerator now assigns only one split per reader. The remaining case I see is when the parallelism of the job is decreased (either by user or failover). In that case, as you know, upon restoration, a reader could be assigned the splits of another reader (as there are less readers after parallelism decrease).
   
   But still you're right, in case of parallelism=1 then, all the records would be in 1 split in 1 reader, there will be only a single query issued and a single resultset containing all the records. These records will indeed be stored in `recordsBySplit` to serve for constructing returned `RecordsBySplits`. That being said, as there will be a single resultset, we need to figure out a way of splitting it for that particular case. Luckily Cassandra `ResultSet` is paginated with results fetched only when all the records of the previous page were consumed. So, I could store in the `CassandraSplitState` a reference to the `ResultSet` to resume the output of remaining records on a later `fetch()`. Question is: what would be the condition to exit fetch() earlier. What about allowing the user to configure a `maxRecordPerSplit` and also provide a default value?
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087950644


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {

Review Comment:
   class removed



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089140706


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   I have pushed an impl that keeps track of the state of the fetch with CassandraSplitState#resultSet. It supports fetch() of MAX_RECORDS_PER_SPLIT and fetch() resume. I added MAX_RECORDS_PER_SPLIT conf parameter as proposed above because I did not know if such parameter existed already. Feel free to point me to an existing one or to propose another parameter name.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137350771


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.net.InetSocketAddress;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+
+    private Cluster cluster;
+    private Session session;
+    private ClusterBuilder clusterBuilder;
+
+    public CassandraTestEnvironment() {
+        cassandraContainer = new CassandraContainer(DOCKER_CASSANDRA_IMAGE);
+        // more generous timeouts
+        addJavaOpts(
+                cassandraContainer,
+                "-Dcassandra.request_timeout_in_ms=30000",
+                "-Dcassandra.read_request_timeout_in_ms=15000",
+                "-Dcassandra.write_request_timeout_in_ms=6000");
+    }
+
+    @Override
+    public void startUp() throws Exception {
+        startEnv();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        stopEnv();
+    }
+
+    private static void addJavaOpts(GenericContainer<?> container, String... opts) {
+        String jvmOpts = container.getEnvMap().getOrDefault("JVM_OPTS", "");
+        container.withEnv("JVM_OPTS", jvmOpts + " " + StringUtils.join(opts, " "));
+    }
+
+    private void startEnv() throws Exception {
+        // configure container start to wait until cassandra is ready to receive queries
+        cassandraContainer.waitingFor(new CassandraQueryWaitStrategy());
+        // start with retrials
+        cassandraContainer.start();
+        cassandraContainer.followOutput(
+                new Slf4jLogConsumer(LOG),
+                OutputFrame.OutputType.END,
+                OutputFrame.OutputType.STDERR,
+                OutputFrame.OutputType.STDOUT);
+
+        cluster = cassandraContainer.getCluster();
+        clusterBuilder =
+                createBuilderWithConsistencyLevel(
+                        ConsistencyLevel.ONE,
+                        cassandraContainer.getHost(),
+                        cassandraContainer.getMappedPort(CQL_PORT));
+
+        session = cluster.connect();
+        session.execute(requestWithTimeout(CREATE_KEYSPACE_QUERY));
+        // create a dedicated table for split size tests (to avoid having to flush with each test)
+        insertTestDataForSplitSizeTests();
+    }
+
+    private void insertTestDataForSplitSizeTests() throws Exception {
+        session.execute(requestWithTimeout(CREATE_SPLITS_TABLE_QUERY));
+        for (int i = 0; i < NB_SPLITS_RECORDS; i++) {
+            session.execute(requestWithTimeout(String.format(INSERT_INTO_FLINK_SPLITS, i, i)));
+        }
+        flushMemTables(SPLITS_TABLE);
+    }
+
+    private void stopEnv() {
+
+        if (session != null) {
+            session.close();
+        }
+        if (cluster != null) {
+            cluster.close();
+        }
+        cassandraContainer.stop();
+    }
+
+    private ClusterBuilder createBuilderWithConsistencyLevel(
+            ConsistencyLevel consistencyLevel, String host, int port) {
+        return new ClusterBuilder() {
+            @Override
+            protected Cluster buildCluster(Cluster.Builder builder) {
+                return builder.addContactPointsWithPorts(new InetSocketAddress(host, port))
+                        .withQueryOptions(
+                                new QueryOptions()
+                                        .setConsistencyLevel(consistencyLevel)
+                                        .setSerialConsistencyLevel(ConsistencyLevel.LOCAL_SERIAL))
+                        .withSocketOptions(
+                                new SocketOptions()
+                                        // default timeout x 3
+                                        .setConnectTimeoutMillis(15000)
+                                        // default timeout x3 and higher than
+                                        // request_timeout_in_ms at the cluster level
+                                        .setReadTimeoutMillis(READ_TIMEOUT_MILLIS))
+                        .withoutJMXReporting()
+                        .withoutMetrics()
+                        .build();
+            }
+        };
+    }
+
+    /**
+     * Force the flush of cassandra memTables to SSTables in order to update size_estimates. It is
+     * needed for the tests because we just inserted records, we need to force cassandra to update
+     * size_estimates system table.
+     */
+    void flushMemTables(String table) throws Exception {
+        cassandraContainer.execInContainer("nodetool", "flush", KEYSPACE, table);

Review Comment:
   I tried it at first but it does not update all the statistics only the tokens are updated in `size_estimates` table. `partition_count` and `mean_partition_size` are still 0 leading to a table size of 0. 
   
   For the record, at the time I first wrote this code (2017) Spark Cassandra connector used the same flush method in the tests.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137340072


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 20;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        100_000_000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // tableSize / maxSplitMemorySize is too little compared to parallelism falling back to
+        // number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too small split size set")
+    public void testGenerateSplitsWithTooSmallSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        1L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // tableSize / maxSplitMemorySize is too big compared to parallelism falling back to
+        // number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+    }
+
+    // overridden to use unordered checks
+    @Override
+    protected void checkResultWithSemantic(
+            CloseableIterator<Pojo> resultIterator,
+            List<List<Pojo>> testData,
+            CheckpointingMode semantic,
+            Integer limit) {
+        if (limit != null) {
+            Runnable runnable =
+                    () ->
+                            CollectIteratorAssertions.assertUnordered(resultIterator)
+                                    .withNumRecordsLimit(limit)
+                                    .matchesRecordsFromSource(testData, semantic);
+
+            assertThat(runAsync(runnable)).succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT);

Review Comment:
   ok then. Do you happen to know why runAsync is used?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138769756


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high

Review Comment:
   > what if the user defined value is way above the table size? We fallback to one split ?
   
   Since it's only a _maximum_ and not a _target_ size, so long as we are below the maximum we can generate as many splits as we'd like (== parallelism I suppose).
   
   That's how I think about it at least.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137122610


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.CassandraSource;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** tests for query generation and query sanity checks. */
+class CassandraQueryTest {
+
+    private static final Pattern SELECT_PATTERN = Pattern.compile(CassandraSource.SELECT_REGEXP);
+
+    @Test
+    public void testKeySpaceTableExtractionRegexp() {
+        Arrays.asList(
+                        "select field FROM keyspace.table where field = value;",
+                        "select * FROM keyspace.table;",
+                        "select field1, field2 from keyspace.table;",
+                        "select field1, field2 from keyspace.table LIMIT(1000);",
+                        "select field1 from keyspace.table ;",
+                        "select field1 from keyspace.table where field1=1;")
+                .forEach(CassandraQueryTest::assertQueryFormatCorrect);
+
+        Arrays.asList(
+                        "select field1 from table;", // missing keyspace
+                        "select field1 from keyspace.table" // missing ";"
+                        )

Review Comment:
   Seems a corner case to me as these are not valid CQL queries and we should not test every possible typo. But fair enough



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137200743


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high
+                LOG.info(
+                        "maxSplitMemorySize set value leads to {} splits with a task parallelism of {}. Creating as many splits as parallelism",
+                        numSplits,
+                        parallelism);
+                numSplits = parallelism;
+            }
+        } else { // not defined
+            LOG.info("maxSplitMemorySize not set. Creating as many splits as parallelism");

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137317001


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 20;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        100_000_000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // tableSize / maxSplitMemorySize is too little compared to parallelism falling back to
+        // number of splits = parallelism

Review Comment:
   I asked myself the same but I concluded that having a parallelism > 1 is profitable compared to 1 (the reduced reading time compensates the added task synchronisation time) as long as the data is not too little. Of course everything depends of the threshold set for "too little'



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1055834384


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Sets;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            // needs to be mutable

Review Comment:
   done



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1337060717

   Any updates on this PR ? This subject has been open for a long time: all the other Cassandra PRs (https://github.com/apache/flink/pull/19586, https://github.com/apache/flink/pull/19680, https://github.com/apache/flink-connector-cassandra/pull/2) were just a preparation for this PR and this PR was submitted in October. 
   Is there anything I can do to help the review ?


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1386836888

   @zentol I did not author the `RingRange` and `SplitsGenerator` classes. I got them from the `Apache Beam` Cassandra connector. I agree, some notions need clarification. I'll add comments.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073523022


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {
+        return (start.compareTo(end) >= 0)
+                ? end.subtract(start).add(ringSize)
+                : end.subtract(start);
+    }
+
+    /** @return true if the ringRange overlaps. Note that if start == end, then wrapping is true */
+    public boolean isWrapping() {
+        return start.compareTo(end) >= 0;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("(%s,%s]", start.toString(), end.toString());
+    }
+
+    @Override
+    public boolean equals(@Nullable Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        RingRange ringRange = (RingRange) o;
+
+        if (getStart() != null
+                ? !getStart().equals(ringRange.getStart())
+                : ringRange.getStart() != null) {
+            return false;
+        }
+        return getEnd() != null ? getEnd().equals(ringRange.getEnd()) : ringRange.getEnd() == null;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getStart() != null ? getStart().hashCode() : 0;

Review Comment:
   no null indeed, will adapt hashcode/equals 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073553855


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();

Review Comment:
   From Cassandra partitionner specifications: https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerM3P.html



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087821967


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -48,6 +52,30 @@ public CassandraSplitState toSplitState() {
         return new CassandraSplitState(new HashSet<>(ringRanges), splitId());
     }
 
+    public void serialize(ObjectOutputStream objectOutputStream) throws IOException {

Review Comment:
   let's make this package private. Personally I'd move all of it into the serializer, but I can see why it could be useful to have it here.
   For symmetry I'd prefer this method to be static and accept a CassandraSplit instance.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -48,6 +52,30 @@ public CassandraSplitState toSplitState() {
         return new CassandraSplitState(new HashSet<>(ringRanges), splitId());
     }
 
+    public void serialize(ObjectOutputStream objectOutputStream) throws IOException {
+        objectOutputStream.writeInt(ringRanges.size());
+        for (RingRange ringRange : ringRanges) {
+            objectOutputStream.writeObject(ringRange.getStart());
+            objectOutputStream.writeObject(ringRange.getEnd());
+        }
+    }
+
+    public static CassandraSplit deserialize(ObjectInputStream objectInputStream)
+            throws IOException {
+        try {
+            final int nbRingRanges = objectInputStream.readInt();
+            Set<RingRange> ringRanges = new HashSet<>(nbRingRanges);

Review Comment:
   note: Passing npRingRanges won't have the desired effect of avoiding allocations, since the backing map is resized when the map is filled to a certain load. In fact you've guaranteed that the map will be resized at least once.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   Of course the splits are scoped to the reader.
   But say, if parallelism=1, then you're reading all records from all splits into memory.
   Even with a higher parallelism cases I don't see this working very well for larger tables.
   
   While the split reader API docs are a bit unclear on this, you _are_ allowed to exist the fetch call early. If you adjust the splits accordingly fetch() will be called again later on and you can continue where you left off.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087926802


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorStateSerializer;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraSplitEnumerator;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSourceReader;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSplitReader;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A bounded source to read from Cassandra and return a collection of entities as {@code
+ * DataStream<Entity>}. An entity is built by Cassandra mapper ({@code
+ * com.datastax.driver.mapping.EntityMapper}) based on a POJO containing annotations (as described
+ * in <a
+ * href="https://docs.datastax.com/en/developer/java-driver/3.11/manual/object_mapper/creating/">
+ * Cassandra object mapper</a>).
+ *
+ * <p>To use it, do the following:
+ *
+ * <pre>{@code
+ * ClusterBuilder clusterBuilder = new ClusterBuilder() {
+ *   @Override
+ *   protected Cluster buildCluster(Cluster.Builder builder) {
+ *     return builder.addContactPointsWithPorts(new InetSocketAddress(HOST,PORT))
+ *                   .withQueryOptions(new QueryOptions().setConsistencyLevel(CL))
+ *                   .withSocketOptions(new SocketOptions()
+ *                   .setConnectTimeoutMillis(CONNECT_TIMEOUT)
+ *                   .setReadTimeoutMillis(READ_TIMEOUT))
+ *                   .build();
+ *   }
+ * };
+ * Source cassandraSource = new CassandraSource(clusterBuilder,
+ *                                              Pojo.class,
+ *                                              "select ... from KEYSPACE.TABLE ...;",
+ *                                              () -> new Mapper.Option[] {Mapper.Option.saveNullFields(true)});
+ *
+ * DataStream<Pojo> stream = env.fromSource(cassandraSource, WatermarkStrategy.noWatermarks(),
+ * "CassandraSource");
+ * }</pre>
+ */
+@PublicEvolving
+public class CassandraSource<OUT>
+        implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
+
+    public static final String CQL_PROHIBITTED_CLAUSES_REGEXP =
+            "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    private static final long serialVersionUID = 7773196541275567433L;
+
+    private final ClusterBuilder clusterBuilder;
+    private final Class<OUT> pojoClass;
+    private final String query;
+    private final MapperOptions mapperOptions;
+
+    public CassandraSource(
+            ClusterBuilder clusterBuilder,
+            Class<OUT> pojoClass,
+            String query,
+            MapperOptions mapperOptions) {
+        checkNotNull(clusterBuilder, "ClusterBuilder required but not provided");
+        checkNotNull(pojoClass, "POJO class required but not provided");
+        checkQueryValidity(query);
+        this.clusterBuilder = clusterBuilder;
+        ClosureCleaner.clean(clusterBuilder, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+        this.pojoClass = pojoClass;
+        this.query = query;
+        this.mapperOptions = mapperOptions;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return Boundedness.BOUNDED;
+    }
+
+    @Internal
+    @Override
+    public SourceReader<OUT, CassandraSplit> createReader(SourceReaderContext readerContext) {
+        return new CassandraSourceReader<>(
+                readerContext, clusterBuilder, pojoClass, query, mapperOptions);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> createEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext) {
+        return new CassandraSplitEnumerator(enumContext, null, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> restoreEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext,
+            CassandraEnumeratorState enumCheckpoint) {
+        return new CassandraSplitEnumerator(enumContext, enumCheckpoint, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraSplit> getSplitSerializer() {
+        return CassandraSplitSerializer.INSTANCE;
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraEnumeratorState> getEnumeratorCheckpointSerializer() {
+        return CassandraEnumeratorStateSerializer.INSTANCE;
+    }
+
+    @Override
+    public TypeInformation<OUT> getProducedType() {
+        return TypeInformation.of(pojoClass);
+    }
+
+    @VisibleForTesting
+    public static void checkQueryValidity(String query) {
+        checkNotNull(query, "query required but not provided");
+        checkState(
+                query.matches(CassandraSplitReader.SELECT_REGEXP),
+                "query must be of the form select ... from keyspace.table ...;");
+        checkState(
+                !query.matches(CQL_PROHIBITTED_CLAUSES_REGEXP),
+                "query must not contain aggregate or order clauses because they will be done per split. "
+                        + "So they will be incorrect after merging the splits");

Review Comment:
   This is purely about phrasing.
   "Splits" is ultimately a source-internal concept that no one that hasn't written their own source is aware of.
   I'd suggest something like "Aggregations/OrderBy are not supported because the query is executed on subsets/partitions of the input table."
   
   Note that it _can_ still be useful to allow aggregation or order clauses. For example, pushing an aggregate into cassandra and applying a aggregate function in Flink _can_ be more efficient than fetching everything from Cassandra and doing all of the aggregation in Flink.
   
   If we want to support Flink SQL we'll need to revisit this, because it would be nice to to support pushdown aggregates. 
   
   On that note, how does groupBy interact with token ranges? I assume there's no guarantee that a particular subtask gets all values for a given key, because in practice the token ranges split the input into effectively random partitions?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1088816929


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   > from all splits into memory
   
   Well, in 99% of the cases, `unprocessedSplits` will contain only a single split as the enumerator now assigns only one split per reader. The remaining case I see is when the parallelism of the job is decreased (either by user or failover). In that case, as you know, upon restoration, a reader could be assigned the splits of another reader (as there are less readers after parallelism decrease).
   
   But still you're right, in case of parallelism=1 then, all the records would be in 1 split in 1 reader, there will be only a single query issued and a single resultset containing all the records. These records will indeed be stored in `recordsBySplit` to serve for constructing returned `RecordsBySplits`. That being said, as there will be a single resultset, we need to figure out a way of splitting it for that particular case. Luckily Cassandra `ResultSet` is paginated with results fetched only when all the records of the previous page were consumed. So, I could store in the `CassandraSplitState` a reference to the `ResultSet` to resume the output of remaining records on a later `fetch()`. My question is: what would be the condition to exit fetch() earlier? What about allowing the user to configure a `maxRecordPerSplit` and also provide a default value?
   
   It is true that fetch early exit should be mentioned in the docs. I'll open a ticket for that and do the PR later. 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1085595610


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set

Review Comment:
   `CassandraSourceReader` is  a `SingleThreadMultiplexSourceReaderBase` so each fetcher thread has one instance of `CassandraSplitReader` owing to the doc. but when we run `SourceTestSuiteBase#testIdleReader` (4 splits with parallelism = 5) accessing `CassandraSplitReader#unprocessedSplits`raises a `ConcurrentModificationException` with one of the 5 fetcher threads.  Am I right to state that for a batch source  the number of splits in a given source should always be equal to the parallelism? And thus that each SplitReader should only read a single split ?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
zentol commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1398165894

   > So I think I'll try to reuse this code, migrate it to Flink and update it to the latest Cassandra version and push it in this PR.
   
   Sounds good to me so far.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1085316765


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {
+            // allow to interrupt the reading of splits as requested in the API
+            if (wakeup.get()) {

Review Comment:
   none. It was just to interrupt the for loop on a timely manner (as the doc says it should not be blocking). If you think it is overkill, I can remove it



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1397142509

   > > I did not author the RingRange and SplitsGenerator classes. I got them from the Apache Beam Cassandra connector.
   > 
   > If you're copying stuff from other projects it please add some notice/attribution to the files and update the Flink source notice accordingly.
   
   ah yes I forgot this Flink guideline. I don't think it is an ASF rule but fair enough, I'll add it to the javadoc and notice.
   
   > 
   > > Back in 2017 I coded a [splitter for Cassandra Beam connector](https://github.com/echauchot/beam/blob/bfa33b85b6b310556ffa5c44c99bef50575b2c56/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java#L346) that works [with tokens](https://github.com/echauchot/beam/blob/BEAM-245-CassandraIO/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/DataSizeEstimates.java) also but that is simpler and supports all the Cassandra partitionners. Would you prefer that we use this other approach ?
   > 
   > Not sure? Why didn't it make it into Beam? Do you know why the Beam code is written the way it is?
   
   Actually, another splitting approach was opted in by the reviewer in 2017. But short after there was another author who changed the splitting to something similar to my 2017 token based code. So when I thought about coding the split for Flink connector I decided to take the version of the code that was merged to Beam master. But it is true that it is over complicated, redundant and not supporting the non-default Cassandra partitioner. The approach I had in 2017 was the same as the Cassandra Spark connector written by datastax (tokens + cassandra size estimates statistics). So I think I'll try to reuse this code, migrate it to Flink and update it to the latest Cassandra version and push it in this PR. 
   WDYT ?
   


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1085595610


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set

Review Comment:
   `CassandraSourceReader` is  a `SingleThreadMultiplexSourceReaderBase` so each fetcher thread has one instance of `CassandraSplitReader` owing to the doc. but when we run `SourceTestSuiteBase#testIdleReader` (4 splits with parallelism = 5) accessing `CassandraSplitReader#unprocessedSplits`raises a `ConcurrentModificationException` with one of the 5 fetcher threads.  unprocessedSplits is there because a given split reader might be assigned several splits in case the **strange splitGenerator** returns more splits than requested parallelism. With the new splitGenerator there will be one split per split reader so no need for the set



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1085585517


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   No. `CassandraSplits` are assigned to `CassandraSplitReader` (see `CassandraSplitEnumerator#assignUnprocessedSplitsToReader`). A `CassandraSplitReader` might be assigned several splits in case the **strange** splitGenerator returns more splits than requested parallelism. With the new `splitGenerator` there will be exactly as many splits as the parallelism.  There will be one split per split reader so no need for the set.
   
   Here we read all the records that belong to **the splits assigned to the reader***. After split generator change no more loop.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100252874


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        this.unprocessedSplits = new HashSet<>();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();

Review Comment:
   Yes I thought about that but the problem I had was when to close the session/cluster. In that case, I'll override `SourceReaderBase#close()`. Also, I like the design of passing a map function to the emitter from the source reader. 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137346895


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high
+                LOG.info(
+                        "maxSplitMemorySize set value leads to {} splits with a task parallelism of {}. Creating as many splits as parallelism",
+                        numSplits,
+                        parallelism);
+                numSplits = parallelism;
+            }
+        } else { // not defined
+            LOG.info("maxSplitMemorySize not set. Creating as many splits as parallelism");
+            numSplits = parallelism;
+        }
+
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken = partitioner.minToken;
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            BigInteger endToken = startToken.add(increment);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken;
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+            startToken = endToken;
+        }
+        LOG.debug("Generated {} splits : {}", splits.size(), splits);
+        return splits;
+    }
+
+    /**
+     * Estimates the size of the table in bytes. Cassandra size estimates can be 0 if the data was
+     * just inserted and the amount of data in the table was small. This is very common situation
+     * during tests.
+     */
+    @VisibleForTesting
+    public long estimateTableSize() {
+        List<TokenRange> tokenRanges = getTokenRangesOfTable();
+        long size = 0L;
+        for (TokenRange tokenRange : tokenRanges) {
+            size += tokenRange.meanPartitionSize * tokenRange.partitionCount;
+        }
+        return Math.round(size / getRingFraction(tokenRanges));
+    }
+
+    /**
+     * The values that we get from system.size_estimates are for one node. We need to extrapolate to
+     * the whole cluster. This method estimates the percentage, the node represents in the cluster.
+     *
+     * @param tokenRanges The list of {@link TokenRange} to estimate
+     * @return The percentage the node represent in the whole cluster
+     */
+    private float getRingFraction(List<TokenRange> tokenRanges) {
+        BigInteger addressedTokens = BigInteger.ZERO;
+        for (TokenRange tokenRange : tokenRanges) {
+            addressedTokens =
+                    addressedTokens.add(distance(tokenRange.rangeStart, tokenRange.rangeEnd));
+        }
+        // it is < 1 because it is a percentage
+        return Float.valueOf(addressedTokens.divide(partitioner.ringSize).toString());

Review Comment:
   Can this be a divide-by-zero if the size_estimates table was empty?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138350686


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 20;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        100_000_000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // tableSize / maxSplitMemorySize is too little compared to parallelism falling back to
+        // number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too small split size set")
+    public void testGenerateSplitsWithTooSmallSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        1L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // tableSize / maxSplitMemorySize is too big compared to parallelism falling back to
+        // number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+    }
+
+    // overridden to use unordered checks
+    @Override
+    protected void checkResultWithSemantic(
+            CloseableIterator<Pojo> resultIterator,
+            List<List<Pojo>> testData,
+            CheckpointingMode semantic,
+            Integer limit) {
+        if (limit != null) {
+            Runnable runnable =
+                    () ->
+                            CollectIteratorAssertions.assertUnordered(resultIterator)
+                                    .withNumRecordsLimit(limit)
+                                    .matchesRecordsFromSource(testData, semantic);
+
+            assertThat(runAsync(runnable)).succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT);

Review Comment:
   No, I find it strange as well as we assert asynchronously on a limited number of records and synchronously on a unlimited (finite but potentially big) number of records.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137133490


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.CassandraSource;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** tests for query generation and query sanity checks. */
+class CassandraQueryTest {
+
+    private static final Pattern SELECT_PATTERN = Pattern.compile(CassandraSource.SELECT_REGEXP);
+
+    @Test
+    public void testKeySpaceTableExtractionRegexp() {
+        Arrays.asList(
+                        "select field FROM keyspace.table where field = value;",
+                        "select * FROM keyspace.table;",
+                        "select field1, field2 from keyspace.table;",
+                        "select field1, field2 from keyspace.table LIMIT(1000);",
+                        "select field1 from keyspace.table ;",
+                        "select field1 from keyspace.table where field1=1;")
+                .forEach(CassandraQueryTest::assertQueryFormatCorrect);
+
+        Arrays.asList(
+                        "select field1 from table;", // missing keyspace
+                        "select field1 from keyspace.table" // missing ";"
+                        )
+                .forEach(CassandraQueryTest::assertQueryFormatIncorrect);
+    }
+
+    @Test
+    public void testProhibitedClauses() {
+        Arrays.asList(
+                        "SELECT COUNT(*) from flink.table;",
+                        "SELECT AVG(*) from flink.table;",
+                        "SELECT MIN(*) from flink.table;",
+                        "SELECT MAX(*) from flink.table;",
+                        "SELECT SUM(*) from flink.table;",
+                        "SELECT field1, field2 from flink.table ORDER BY field1;",
+                        "SELECT field1, field2 from flink.table GROUP BY field1;")
+                .forEach(CassandraQueryTest::assertProhibitedClauseRejected);
+    }
+
+    @Test
+    public void testGenerateRangeQuery() {
+        String query;
+        String outputQuery;
+
+        // query with where clause
+        query = "SELECT field FROM keyspace.table WHERE field = value;";
+        outputQuery = CassandraSplitReader.generateRangeQuery(query, "field");
+        assertThat(outputQuery)
+                .isEqualTo(
+                        "SELECT field FROM keyspace.table WHERE (token(field) >= ?) AND (token(field) < ?) AND field = value;");
+
+        // query without where clause
+        query = "SELECT * FROM keyspace.table;";
+        outputQuery = CassandraSplitReader.generateRangeQuery(query, "field");
+        assertThat(outputQuery)
+                .isEqualTo(
+                        "SELECT * FROM keyspace.table WHERE (token(field) >= ?) AND (token(field) < ?);");
+
+        // query without where clause but with another trailing clause

Review Comment:
   :+1: yes missing one of the 4 combinations. Thanks



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137185252


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 20;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        100_000_000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // tableSize / maxSplitMemorySize is too little compared to parallelism falling back to
+        // number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too small split size set")
+    public void testGenerateSplitsWithTooSmallSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        1L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // tableSize / maxSplitMemorySize is too big compared to parallelism falling back to
+        // number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+    }
+
+    // overridden to use unordered checks
+    @Override
+    protected void checkResultWithSemantic(
+            CloseableIterator<Pojo> resultIterator,
+            List<List<Pojo>> testData,
+            CheckpointingMode semantic,
+            Integer limit) {
+        if (limit != null) {
+            Runnable runnable =
+                    () ->
+                            CollectIteratorAssertions.assertUnordered(resultIterator)
+                                    .withNumRecordsLimit(limit)
+                                    .matchesRecordsFromSource(testData, semantic);
+
+            assertThat(runAsync(runnable)).succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT);

Review Comment:
   because it was already like this  in `SourceTestSuiteBase` if just took the same impl and replaced `CollectIteratorAssertions#assertThat` by `CollectIteratorAssertions#assertUnordered`



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137188385


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraRecordEmitter.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.api.connector.source.SourceOutput;
+import org.apache.flink.connector.base.source.reader.RecordEmitter;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.ExecutionInfo;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * {@link RecordEmitter} that converts the {@link CassandraRow} read by the {@link
+ * CassandraSplitReader} to specified POJO and output it. This class uses the Cassandra driver
+ * mapper to map the row to the POJO.
+ *
+ * @param <OUT> type of POJO record to output
+ */
+class CassandraRecordEmitter<OUT> implements RecordEmitter<CassandraRow, OUT, CassandraSplit> {
+
+    private final Function<ResultSet, OUT> map;
+
+    public CassandraRecordEmitter(Function<ResultSet, OUT> map) {
+        this.map = map;
+    }
+
+    @Override
+    public void emitRecord(
+            CassandraRow cassandraRow, SourceOutput<OUT> output, CassandraSplit cassandraSplit) {
+        final Row row = cassandraRow.getRow();
+        // Mapping from a row to a Class<OUT> is a complex operation involving reflection API.
+        // It is better to use Cassandra mapper for it.
+        // but the mapper takes only a resultSet as input hence forging one containing only the row
+        ResultSet resultSet =
+                new ResultSet() {

Review Comment:
   :+1: thanks for catching it



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1454671877

   @mosche contributed testContainers configuration that allows to avoid having a private docker image + timeouts etc...


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1477517177

   @zentol I addressed all your latest comments. Can we merge the PR ?
   


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] RyanSkraba commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
RyanSkraba commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1327674570

   I'm taking a look!  Thanks for the ping :+1: 


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101574664


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializerTest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
+
+import org.junit.jupiter.api.Test;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link CassandraEnumeratorStateSerializer}. */
+public class CassandraEnumeratorStateSerializerTest {

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101573610


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitSerializerTest.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.math.BigInteger;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link CassandraSplitSerializer}. */
+public class CassandraSplitSerializerTest {

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101701294


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitState.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import com.datastax.driver.core.ResultSet;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+
+/**
+ * Mutable {@link CassandraSplit} that keeps track of the reading process of the associated split.
+ */
+public class CassandraSplitState {
+    private final CassandraSplit cassandraSplit;
+    // Cassandra ResultSet is paginated, a new page is read only if all the records of the previous
+    // one were consumed. fetch() can be interrupted so we use the resultSet to keep track of the
+    // reading process.
+    // It is null when reading has not started (before fetch is called on the split).
+    @Nullable private ResultSet resultSet;

Review Comment:
   As discussed in the other comment ResultSet is just a handle so the status of the read will not be part on the checkpoint leading to a re-read of the already output data indeed. The only way is to manage the memory size of the split at the enumerator level and either output all the split or not a all. That way in case of interrupted fetch nothing will be output and the split could be read again from the beginning after recovery leading to no duplicates.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1055410141


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Sets;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            // needs to be mutable

Review Comment:
   I just figured out that the set of ringRanges is passed by reference from the `CassandraSplit` to the `CassandraSplitState`. So when the mutable `CassandraSplitState` updates its `ringRanges` it modifies `CassandraSplit` ones in reality. As `CassandraSplit` is supposed to be inmutable. I should copy the `ringRanges` from the `CassandraSplit`.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087928810


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {

Review Comment:
   No longer needs to be serializable; also applies to other classes.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089225257


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   
   It is true that fetch early exit should be mentioned in the docs. I'll open a ticket for that and do the PR later. https://issues.apache.org/jira/browse/FLINK-30802. Can you assign it to me ?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100124063


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -48,6 +52,30 @@ public CassandraSplitState toSplitState() {
         return new CassandraSplitState(new HashSet<>(ringRanges), splitId());
     }
 
+    public void serialize(ObjectOutputStream objectOutputStream) throws IOException {
+        objectOutputStream.writeInt(ringRanges.size());
+        for (RingRange ringRange : ringRanges) {
+            objectOutputStream.writeObject(ringRange.getStart());
+            objectOutputStream.writeObject(ringRange.getEnd());
+        }
+    }
+
+    public static CassandraSplit deserialize(ObjectInputStream objectInputStream)
+            throws IOException {
+        try {
+            final int nbRingRanges = objectInputStream.readInt();
+            Set<RingRange> ringRanges = new HashSet<>(nbRingRanges);

Review Comment:
   resolving that there are no more ringRanges set



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100204504


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -48,6 +52,30 @@ public CassandraSplitState toSplitState() {
         return new CassandraSplitState(new HashSet<>(ringRanges), splitId());
     }
 
+    public void serialize(ObjectOutputStream objectOutputStream) throws IOException {

Review Comment:
   same for deserialize and EnumeratorStateSerializer



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1104612653


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializer.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.core.memory.DataInputDeserializer;
+import org.apache.flink.core.memory.DataOutputSerializer;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+/** Serializer for {@link CassandraEnumeratorState}. */
+public class CassandraEnumeratorStateSerializer
+        implements SimpleVersionedSerializer<CassandraEnumeratorState> {
+
+    public static final CassandraEnumeratorStateSerializer INSTANCE =
+            new CassandraEnumeratorStateSerializer();
+    private static final ThreadLocal<DataOutputSerializer> SERIALIZER_CACHE =
+            ThreadLocal.withInitial(() -> new DataOutputSerializer(64));

Review Comment:
   This is overkill for the enumerator state.
   In contrast to splits (of which there can be thousands...[hmm](https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1104624477)) there's only 1 state at all times.
   
   Fun fact about thread locals: They can easily leak the user-code classloader. If any thread that out-lives the task (like say, a shared thread pool) ever accesses the ThreadLocal, then the thread retains a reference to the thread local, due to which it can't be GC'd. Just thought I'd throw this out there.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100252874


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        this.unprocessedSplits = new HashSet<>();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();

Review Comment:
   Yes I thought about that but the problem I had was when to close the session/cluster. I you prefer this solution, I can override `SourceReaderBase#close()` and close them there + create a static factory to initialize cluster and session at the creation of SourceReader before the super(...) calll. Also, I like the design of passing a map function to the emitter from the source reader. 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101584861


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and Flink
+ * source parallelism.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final CassandraPartitioner partitioner;
+
+    public SplitsGenerator(CassandraPartitioner partitioner) {
+        this.partitioner = partitioner;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the ring.
+     *
+     * @param numSplits requested number of splits
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long numSplits) {

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101601871


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and Flink
+ * source parallelism.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final CassandraPartitioner partitioner;
+
+    public SplitsGenerator(CassandraPartitioner partitioner) {
+        this.partitioner = partitioner;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the ring.
+     *
+     * @param numSplits requested number of splits
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long numSplits) {
+        if (numSplits == 1) {
+            return Collections.singletonList(
+                    new CassandraSplit(partitioner.minToken(), partitioner.maxToken()));
+        }
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger splitSize =
+                (partitioner.ringSize()).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken, endToken = partitioner.minToken();
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            startToken = endToken;
+            endToken = startToken.add(splitSize);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken();
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+        }

Review Comment:
   :+1: for the general change but regarding `splitCount == numSplits` it is to make sure the last split in the list covers the max token for division rounding



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101496694


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -48,6 +52,30 @@ public CassandraSplitState toSplitState() {
         return new CassandraSplitState(new HashSet<>(ringRanges), splitId());
     }
 
+    public void serialize(ObjectOutputStream objectOutputStream) throws IOException {

Review Comment:
   > For symmetry I'd prefer this method to be static and accept a CassandraSplit instance 
   
   I made the serializer take an OutputStream to avoid creating an OutputStream for each split in the EnumeratorState serialization which would create a stream header for each split. I managed to solve this issue by caching the outputStream like it was done in the file source. So now I stick to the usual serde signatures
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1106073628


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        checkReaderRegistered(subtaskId);
+        final CassandraSplit cassandraSplit = state.getASplit();
+        if (cassandraSplit != null) {
+            LOG.info("Assigning splits to reader {}", subtaskId);
+            enumeratorContext.assignSplit(cassandraSplit, subtaskId);
+        } else {
+            LOG.info(
+                    "No split assigned to reader {} because the enumerator has no unassigned split left",
+                    subtaskId);
+        }
+        if (!state.hasMoreSplits()) {
+            LOG.info(
+                    "No more CassandraSplits to assign. Sending NoMoreSplitsEvent to reader {}.",
+                    subtaskId);
+            enumeratorContext.signalNoMoreSplits(subtaskId);
+        }
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits);
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitionerName = clusterMetadata.getPartitioner();
+        final SplitsGenerator.CassandraPartitioner partitioner =
+                partitionerName.contains(MURMUR3PARTITIONER.className())
+                        ? MURMUR3PARTITIONER
+                        : RANDOMPARTITIONER;
+        return new SplitsGenerator(partitioner).generateSplits(numberOfSplits);

Review Comment:
   See when I push the new enumerator/split code but I think we could define safeguard fallbacks to avoid too many splits, too few splits etc...



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1468389988

   > > Instead of this whole JMX business we could use nodetool directly.
   > > (Note: This review is specifically about the JMX parts; nothing else)
   > 
   > Indeed, this binary is included in the offical cassandra image ! Way simpler. Thanks for the suggestion, I removed all the JMX related code.
   
   


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot closed pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot closed pull request #3: [FLINK-26822] Add Cassandra Source
URL: https://github.com/apache/flink-connector-cassandra/pull/3


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1143714659


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializer.java:
##########
@@ -86,10 +87,7 @@ public CassandraEnumeratorState deserialize(int version, byte[] serialized) thro
             for (int i = 0; i < splitsToReassignSize; i++) {
                 final int splitSize = objectInputStream.readInt();
                 final byte[] splitBytes = new byte[splitSize];
-                if (objectInputStream.read(splitBytes) == -1) {
-                    throw new IOException(
-                            "EOF received while deserializing CassandraEnumeratorState.splitsToReassign");
-                }
+                IOUtils.readFully(objectInputStream, splitBytes, 0, splitSize);

Review Comment:
   Yet another flink util that I did not know. Thanks



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestContext.java:
##########
@@ -88,7 +88,6 @@ public List<URL> getConnectorJarPaths() {
 
         return new CassandraSource<>(
                 clusterBuilder,
-                CassandraSource.MAX_SPLIT_MEMORY_SIZE_DEFAULT,

Review Comment:
   :+1: forgot this cleaning



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitSerializer.java:
##########
@@ -44,12 +45,8 @@ public int getVersion() {
     @Override
     public byte[] serialize(CassandraSplit cassandraSplit) throws IOException {
         final DataOutputSerializer out = SERIALIZER_CACHE.get();
-        final byte[] ringRangeStart = cassandraSplit.getRingRangeStart().toByteArray();

Review Comment:
   Ah yes, I did the change in the enumerator state serializer but forgot the split serializer. Thanks



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -99,35 +99,34 @@
     private final String table;
     private final MapperOptions mapperOptions;
 
-    private final Long maxSplitMemorySize;

Review Comment:
   I removed the `@Nullable`s but forgot these cleanings. Thanks



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138381568


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high

Review Comment:
   Well, off course this ratio is totally arbitrary. I just sketched a quick code to serve as a base for discussion about safeguards on the number of splits. I agree with your arguments on limit cases (freedom for the user and dangerousness of sudden threshold crossing). 
   
   Regarding enforcing a minimum size for user defined split size: sure it protects from having too many splits but what if the defined value is way above the table size? We fallback to one split ?
   
   I'll implement the lazy split generation.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138384453


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low

Review Comment:
   cf https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138381568



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138822645


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high

Review Comment:
   That is what I proposed first falling back to numSplits = parallelism when too few splits. So I'll keep doing that.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1142211308


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This class prepares the generation of {@link CassandraSplit}s based on Cassandra cluster
+ * partitioner and cluster statistics. It estimates the total size of the table using Cassandra
+ * system table system.size_estimates.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    @VisibleForTesting public Long minSplitMemorySize = 10_000_000L; // 10 MB
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Prepare the {@param CassandraEnumeratorState} for lazy generation of {@link CassandraSplit}s:
+     * calculate {@code numSplitsToGenerate} based on estimated target table size and provided
+     * {@code maxSplitMemorySize} and calculate {@code increment} which is the size of a split in
+     * tokens.
+     */
+    public CassandraEnumeratorState prepareSplits() {
+        final long numSplitsToGenerate = decideOnNumSplits();
+        final BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplitsToGenerate)));
+        final BigInteger startToken = partitioner.minToken;
+        return new CassandraEnumeratorState(
+                numSplitsToGenerate,
+                increment,
+                startToken,
+                partitioner.maxToken,
+                new ArrayDeque<>());
+    }
+
+    /**
+     * Determine {@code numSplits} based on the estimation of the target table size and user defined
+     * {@code maxSplitMemorySize}. Provide fallbacks when table size is unavailable, too few or too
+     * many splits are calculated.
+     */
+    private long decideOnNumSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            checkState(
+                    maxSplitMemorySize >= minSplitMemorySize,
+                    "Defined maxSplitMemorySize (%s) is below minimum (%s)",
+                    maxSplitMemorySize,
+                    minSplitMemorySize);
+            final long estimateTableSize = estimateTableSize();
+            if (estimateTableSize == 0) { // size estimates unavailable
+                LOG.info(
+                        "Cassandra size estimates are not available for {}.{} table. Creating as many splits as parallelism ({})",
+                        keyspace,
+                        table,
+                        parallelism);
+                numSplits = parallelism;
+            } else {
+                LOG.debug(
+                        "Estimated size for {}.{} table is {} bytes",
+                        keyspace,
+                        table,
+                        estimateTableSize);
+                numSplits =
+                        estimateTableSize / maxSplitMemorySize == 0
+                                ? parallelism // estimateTableSize under sizes maxSplitMemorySize
+                                // creating as many splits as parallelism

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138363561


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high
+                LOG.info(
+                        "maxSplitMemorySize set value leads to {} splits with a task parallelism of {}. Creating as many splits as parallelism",
+                        numSplits,
+                        parallelism);
+                numSplits = parallelism;
+            }
+        } else { // not defined
+            LOG.info("maxSplitMemorySize not set. Creating as many splits as parallelism");
+            numSplits = parallelism;
+        }
+
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken = partitioner.minToken;
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            BigInteger endToken = startToken.add(increment);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken;
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+            startToken = endToken;
+        }
+        LOG.debug("Generated {} splits : {}", splits.size(), splits);
+        return splits;
+    }
+
+    /**
+     * Estimates the size of the table in bytes. Cassandra size estimates can be 0 if the data was
+     * just inserted and the amount of data in the table was small. This is very common situation
+     * during tests.
+     */
+    @VisibleForTesting
+    public long estimateTableSize() {
+        List<TokenRange> tokenRanges = getTokenRangesOfTable();
+        long size = 0L;
+        for (TokenRange tokenRange : tokenRanges) {
+            size += tokenRange.meanPartitionSize * tokenRange.partitionCount;
+        }
+        return Math.round(size / getRingFraction(tokenRanges));
+    }
+
+    /**
+     * The values that we get from system.size_estimates are for one node. We need to extrapolate to
+     * the whole cluster. This method estimates the percentage, the node represents in the cluster.
+     *
+     * @param tokenRanges The list of {@link TokenRange} to estimate
+     * @return The percentage the node represent in the whole cluster
+     */
+    private float getRingFraction(List<TokenRange> tokenRanges) {
+        BigInteger addressedTokens = BigInteger.ZERO;
+        for (TokenRange tokenRange : tokenRanges) {
+            addressedTokens =
+                    addressedTokens.add(distance(tokenRange.rangeStart, tokenRange.rangeEnd));
+        }
+        // it is < 1 because it is a percentage
+        return Float.valueOf(addressedTokens.divide(partitioner.ringSize).toString());

Review Comment:
   Indeed, you're right if the `size_estimates` table is empty, then calculated `ringFraction` is 0 leading to division by 0. Thanks for catching it ! In that case I'll return a 0 estimated table size, leading to numsplits = 1 (cf your suggestion in a comment above).



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137080083


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializer.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+/** Serializer for {@link CassandraEnumeratorState}. */
+public class CassandraEnumeratorStateSerializer

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137073849


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -80,27 +85,55 @@
 public class CassandraSource<OUT>
         implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
 
-    public static final String CQL_PROHIBITTED_CLAUSES_REGEXP =
+    public static final String CQL_PROHIBITED_CLAUSES_REGEXP =
             "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
     private static final long serialVersionUID = 1L;
 
     private final ClusterBuilder clusterBuilder;
+    @Nullable private final Long maxSplitMemorySize;
     private final Class<OUT> pojoClass;
     private final String query;
+    private final String keyspace;
+    private final String table;
     private final MapperOptions mapperOptions;
 
     public CassandraSource(
             ClusterBuilder clusterBuilder,
             Class<OUT> pojoClass,
             String query,
             MapperOptions mapperOptions) {
+        this(clusterBuilder, null, pojoClass, query, mapperOptions);
+    }
+
+    public CassandraSource(
+            ClusterBuilder clusterBuilder,
+            Long maxSplitMemorySize,
+            Class<OUT> pojoClass,
+            String query,
+            MapperOptions mapperOptions) {
         checkNotNull(clusterBuilder, "ClusterBuilder required but not provided");
+        checkState(
+                maxSplitMemorySize == null || maxSplitMemorySize > 0,
+                "Max split size in bytes provided but set to an invalid value {}",
+                maxSplitMemorySize);
         checkNotNull(pojoClass, "POJO class required but not provided");
-        checkQueryValidity(query);

Review Comment:
   If you prefer that, fair enough



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137154417


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
zentol commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1398165328

   > I don't think it is an ASF rule but fair enough,
   
   FYI; By and large it should be viewed as a legal requirement. By copying code from cassandra you have to adhere to their licensing, which among other state that you must have prominent notices for changes to a file.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073520909


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {
+        return (start.compareTo(end) >= 0)
+                ? end.subtract(start).add(ringSize)

Review Comment:
   :+1:
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1085595610


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set

Review Comment:
   `CassandraSourceReader` is  a `SingleThreadMultiplexSourceReaderBase` so each fetcher thread has one instance of `CassandraSplitReader` owing to the doc. but when we run `SourceTestSuiteBase#testIdleReader` (4 splits with parallelism = 5) accessing `CassandraSplitReader#unprocessedSplits`raises a `ConcurrentModificationException` with one of the 5 fetcher threads.  unprocessedSplits is there because a given split reader might be assigned several splits in case the **strange splitGenerator** returns more splits than requested parallelism. With the new splitGenerator there will be one split per split reader.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1085316765


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {
+            // allow to interrupt the reading of splits as requested in the API
+            if (wakeup.get()) {

Review Comment:
   session.execute() . It was just to interrupt the for loop on a timely manner (as the doc says it should not be blocking). If you think it is overkill, I can remove it



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {
+            // allow to interrupt the reading of splits as requested in the API
+            if (wakeup.get()) {

Review Comment:
   `session.execute()` . It was just to interrupt the for loop on a timely manner (as the doc says it should not be blocking). If you think it is overkill, I can remove it



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1055410141


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Sets;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            // needs to be mutable

Review Comment:
   I just figured out that the set of ringRanges is passed by reference from the `CassandraSplit` to the `CassandraSplitState`. So when the mutable `CassandraSplitState` updates its `ringRanges` it modifies `CassandraSplit` ones in reality. As `CassandraSplit` is supposed to be inmutable. I should copy the `ringRanges` from the `CassandraSplit`



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1364182149

   @RyanSkraba I finished everything I wanted to change PTAL


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073557252


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            return Collections.singletonList(
+                    new CassandraSplit(Collections.singleton(totalRingRange)));
+        }
+        int tokenRangeCount = ringTokens.size();
+
+        List<RingRange> ringRanges = new ArrayList<>();
+        for (int i = 0; i < tokenRangeCount; i++) {
+            BigInteger start = ringTokens.get(i);
+            BigInteger stop = ringTokens.get((i + 1) % tokenRangeCount);
+
+            if (isNotInRange(start) || isNotInRange(stop)) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s) not in range of %s", start, stop, partitioner));
+            }
+            if (start.equals(stop) && tokenRangeCount != 1) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s): two nodes have the same token", start, stop));
+            }
+
+            BigInteger rangeSize = stop.subtract(start);
+            if (rangeSize.compareTo(BigInteger.ZERO) <= 0) {
+                // wrap around case
+                rangeSize = rangeSize.add(this.rangeSize);
+            }
+
+            // the below, in essence, does this:
+            // splitCount = Maths.ceil((rangeSize / cluster range size) * totalSplitCount)
+            BigInteger[] splitCountAndRemainder =
+                    rangeSize
+                            .multiply(BigInteger.valueOf(totalSplitCount))
+                            .divideAndRemainder(this.rangeSize);
+
+            int splitCount =
+                    splitCountAndRemainder[0].intValue()
+                            + (splitCountAndRemainder[1].equals(BigInteger.ZERO) ? 0 : 1);
+
+            LOG.debug("Dividing token range [{},{}) into {} splits", start, stop, splitCount);
+
+            // Make BigInteger list of all the endpoints for the splits, including both start and
+            // stop
+            List<BigInteger> endpointTokens = new ArrayList<>();
+            for (int j = 0; j <= splitCount; j++) {
+                BigInteger offset =
+                        rangeSize
+                                .multiply(BigInteger.valueOf(j))
+                                .divide(BigInteger.valueOf(splitCount));
+                BigInteger token = start.add(offset);
+                if (token.compareTo(rangeMax) > 0) {
+                    token = token.subtract(this.rangeSize);
+                }
+                // Long.MIN_VALUE is not a valid token and has to be silently incremented.
+                // See https://issues.apache.org/jira/browse/CASSANDRA-14684
+                endpointTokens.add(
+                        token.equals(BigInteger.valueOf(Long.MIN_VALUE))
+                                ? token.add(BigInteger.ONE)
+                                : token);
+            }
+
+            // Append the ringRanges between the endpoints
+            for (int j = 0; j < splitCount; j++) {
+                ringRanges.add(RingRange.of(endpointTokens.get(j), endpointTokens.get(j + 1)));
+                LOG.debug(
+                        "Split #{}: [{},{})",
+                        j + 1,
+                        endpointTokens.get(j),
+                        endpointTokens.get(j + 1));
+            }
+        }
+
+        BigInteger total = BigInteger.ZERO;
+        for (RingRange split : ringRanges) {
+            BigInteger size = split.span(rangeSize);
+            total = total.add(size);
+        }
+        if (!total.equals(rangeSize)) {
+            throw new RuntimeException(
+                    "Some tokens are missing from the splits. This should not happen.");
+        }
+        return coalesceRingRanges(getTargetSplitSize(totalSplitCount), ringRanges);
+    }
+
+    private boolean isNotInRange(BigInteger token) {
+        return token.compareTo(rangeMin) < 0 || token.compareTo(rangeMax) > 0;
+    }
+
+    private List<CassandraSplit> coalesceRingRanges(
+            BigInteger targetSplitSize, List<RingRange> ringRanges) {
+        List<CassandraSplit> coalescedSplits = new ArrayList<>();
+        List<RingRange> tokenRangesForCurrentSplit = new ArrayList<>();
+        BigInteger tokenCount = BigInteger.ZERO;
+
+        for (RingRange tokenRange : ringRanges) {
+            if (tokenRange.span(rangeSize).add(tokenCount).compareTo(targetSplitSize) > 0
+                    && !tokenRangesForCurrentSplit.isEmpty()) {
+                // enough tokens in that segment
+                LOG.debug(
+                        "Got enough tokens for one split ({}) : {}",
+                        tokenCount,
+                        tokenRangesForCurrentSplit);
+                coalescedSplits.add(new CassandraSplit(new HashSet<>(tokenRangesForCurrentSplit)));
+                tokenRangesForCurrentSplit = new ArrayList<>();
+                tokenCount = BigInteger.ZERO;
+            }
+
+            tokenCount = tokenCount.add(tokenRange.span(rangeSize));
+            tokenRangesForCurrentSplit.add(tokenRange);
+        }
+
+        if (!tokenRangesForCurrentSplit.isEmpty()) {
+            coalescedSplits.add(new CassandraSplit(new HashSet<>(tokenRangesForCurrentSplit)));
+        }
+        return coalescedSplits;
+    }
+
+    private BigInteger getTargetSplitSize(long splitCount) {
+        return rangeMax.subtract(rangeMin).divide(BigInteger.valueOf(splitCount));

Review Comment:
   See my general comments about Apache Beam Cassandra partitionner



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073627456


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitSerializer.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+/** Serializer for {@link CassandraSplit}. */
+public class CassandraSplitSerializer implements SimpleVersionedSerializer<CassandraSplit> {
+
+    public static final CassandraSplitSerializer INSTANCE = new CassandraSplitSerializer();
+
+    public static final int CURRENT_VERSION = 0;
+
+    private CassandraSplitSerializer() {}
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(CassandraSplit cassandraSplit) throws IOException {
+        try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                ObjectOutputStream objectOutputStream =
+                        new ObjectOutputStream(byteArrayOutputStream)) {
+            objectOutputStream.writeObject(cassandraSplit);

Review Comment:
   :+1:



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1072189999


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.api.connector.source.SourceSplit;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * Immutable {@link SourceSplit} for Cassandra source. A Cassandra split is just a set of {@link

Review Comment:
   ```suggestion
    * Immutable {@link SourceSplit} for Cassandra source. A Cassandra split is a set of {@link
   ```
   Documentation guidelines and all that.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {

Review Comment:
   Reminder to check why this is serializable.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }

Review Comment:
   strange that this logic is different to RingRange#span.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            return Collections.singletonList(
+                    new CassandraSplit(Collections.singleton(totalRingRange)));
+        }
+        int tokenRangeCount = ringTokens.size();
+
+        List<RingRange> ringRanges = new ArrayList<>();
+        for (int i = 0; i < tokenRangeCount; i++) {
+            BigInteger start = ringTokens.get(i);
+            BigInteger stop = ringTokens.get((i + 1) % tokenRangeCount);
+
+            if (isNotInRange(start) || isNotInRange(stop)) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s) not in range of %s", start, stop, partitioner));
+            }
+            if (start.equals(stop) && tokenRangeCount != 1) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s): two nodes have the same token", start, stop));
+            }
+
+            BigInteger rangeSize = stop.subtract(start);
+            if (rangeSize.compareTo(BigInteger.ZERO) <= 0) {
+                // wrap around case
+                rangeSize = rangeSize.add(this.rangeSize);
+            }
+
+            // the below, in essence, does this:
+            // splitCount = Maths.ceil((rangeSize / cluster range size) * totalSplitCount)

Review Comment:
   This is the first time that "cluster range size" is mentioned and it's not explained anywhere or what it's relation to `rangeSize` is.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {
+        return (start.compareTo(end) >= 0)
+                ? end.subtract(start).add(ringSize)
+                : end.subtract(start);
+    }
+
+    /** @return true if the ringRange overlaps. Note that if start == end, then wrapping is true */

Review Comment:
   Not sure what is meant with "overlaps".
   Why do we return true if start == end?
   
   Does start=end mean only a single element is part of the range?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   Am I reading this correctly that you're reading _all_ records from _all_ splits into memory before passing them on?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorStateSerializer;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraSplitEnumerator;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSourceReader;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSplitReader;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A bounded source to read from Cassandra and return a collection of entities as {@code
+ * DataStream<Entity>}. An entity is built by Cassandra mapper ({@code
+ * com.datastax.driver.mapping.EntityMapper}) based on a POJO containing annotations (as described
+ * in <a
+ * href="https://docs.datastax.com/en/developer/java-driver/3.11/manual/object_mapper/creating/">
+ * Cassandra object mapper</a>).
+ *
+ * <p>To use it, do the following:
+ *
+ * <pre>{@code
+ * ClusterBuilder clusterBuilder = new ClusterBuilder() {
+ *   @Override
+ *   protected Cluster buildCluster(Cluster.Builder builder) {
+ *     return builder.addContactPointsWithPorts(new InetSocketAddress(HOST,PORT))
+ *                   .withQueryOptions(new QueryOptions().setConsistencyLevel(CL))
+ *                   .withSocketOptions(new SocketOptions()
+ *                   .setConnectTimeoutMillis(CONNECT_TIMEOUT)
+ *                   .setReadTimeoutMillis(READ_TIMEOUT))
+ *                   .build();
+ *   }
+ * };
+ * Source cassandraSource = new CassandraSource(clusterBuilder,
+ *                                              Pojo.class,
+ *                                              "select ... from KEYSPACE.TABLE ...;",
+ *                                              () -> new Mapper.Option[] {Mapper.Option.saveNullFields(true)});
+ *
+ * DataStream<Pojo> stream = env.fromSource(cassandraSource, WatermarkStrategy.noWatermarks(),
+ * "CassandraSource");
+ * }</pre>
+ */
+@PublicEvolving
+public class CassandraSource<OUT>
+        implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
+
+    public static final String CQL_PROHIBITTED_CLAUSES_REGEXP =
+            "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    private static final long serialVersionUID = 7773196541275567433L;

Review Comment:
   serial version ids should start at 1. https://flink.apache.org/contributing/code-style-and-quality-java.html#java-serialization



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {
+        return (start.compareTo(end) >= 0)

Review Comment:
   ```suggestion
           return isWrapping()
   ```



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {

Review Comment:
   Should probably use an enum.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {

Review Comment:
   So why is this called span instead of "size"?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {
+        return (start.compareTo(end) >= 0)
+                ? end.subtract(start).add(ringSize)

Review Comment:
   ```suggestion
                   ? ringSize.subtract(end).add(start)
   ```
   This was easier for me to visualize.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraRow.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+
+import com.datastax.driver.core.ExecutionInfo;
+import com.datastax.driver.core.Row;
+
+/**
+ * Wrapper for Cassandra {@link Row} that stores associated {@link RingRange} to be able to update
+ * split states. It also stores {@link ExecutionInfo} Cassandra statistics about the query execution
+ * that produced this row.
+ */
+public class CassandraRow {
+
+    private final Row row;
+    private final RingRange associatedRingRange;
+    private final ExecutionInfo executionInfo;

Review Comment:
   Who is actually using this? I see we pass it to the result set when creating the mapper; is it actually relevant there?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {
+        return (start.compareTo(end) >= 0)
+                ? end.subtract(start).add(ringSize)
+                : end.subtract(start);
+    }
+
+    /** @return true if the ringRange overlaps. Note that if start == end, then wrapping is true */
+    public boolean isWrapping() {
+        return start.compareTo(end) >= 0;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("(%s,%s]", start.toString(), end.toString());
+    }
+
+    @Override
+    public boolean equals(@Nullable Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        RingRange ringRange = (RingRange) o;
+
+        if (getStart() != null
+                ? !getStart().equals(ringRange.getStart())
+                : ringRange.getStart() != null) {
+            return false;
+        }
+        return getEnd() != null ? getEnd().equals(ringRange.getEnd()) : ringRange.getEnd() == null;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getStart() != null ? getStart().hashCode() : 0;

Review Comment:
   Are start/end actually nullable?
   
   If so I'd like to know why and various Nullable annotations would be missing.
   if not then hashcode/equals can be simplified.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}

Review Comment:
   Why does it potentially create more splits than requested? Under what circumstances does that happen?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits, enumeratorContext.currentParallelism());
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitioner = clusterMetadata.getPartitioner();
+        final SplitsGenerator splitsGenerator = new SplitsGenerator(partitioner);
+        if (MURMUR3PARTITIONER.equals(partitioner)) {
+            LOG.info("Murmur3Partitioner detected, splitting");
+            List<BigInteger> tokens =
+                    clusterMetadata.getTokenRanges().stream()
+                            .map(
+                                    tokenRange ->
+                                            new BigInteger(
+                                                    tokenRange.getEnd().getValue().toString()))
+                            .collect(Collectors.toList());
+            return splitsGenerator.generateSplits(numberOfSplits, tokens);
+        } else {
+            // Murmur3Partitioner is the default and recommended partitioner for Cassandra 1.2+
+            // see
+            // https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerAbout.html
+            LOG.warn(
+                    "The current Cassandra partitioner is {}, only Murmur3Partitioner is supported "
+                            + "for splitting, using an single split",
+                    partitioner);
+            return splitsGenerator.generateSplits(1, Collections.emptyList());
+        }
+    }
+
+    @Override
+    public void addSplitsBack(List<CassandraSplit> splits, int subtaskId) {
+        LOG.info("Add {} splits back to CassandraSplitEnumerator.", splits.size());
+        state.addNewSplits(splits, enumeratorContext.currentParallelism());
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        LOG.info("Adding reader {} to CassandraSplitEnumerator.", subtaskId);
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    private void assignUnprocessedSplitsToReader(int readerId) {
+        checkReaderRegistered(readerId);
+
+        final Set<CassandraSplit> splitsForReader = state.getSplitsForReader(readerId);

Review Comment:
   For something that is mostly targeted at batch workloads it seems strange to pre-assign splits to readers. Rather they should be fully distributed on demand to prevent stragglers.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializer.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+/** Serializer for {@link CassandraEnumeratorState}. */
+public class CassandraEnumeratorStateSerializer
+        implements SimpleVersionedSerializer<CassandraEnumeratorState> {
+
+    public static final CassandraEnumeratorStateSerializer INSTANCE =
+            new CassandraEnumeratorStateSerializer();
+    public static final int CURRENT_VERSION = 0;
+
+    private CassandraEnumeratorStateSerializer() { // singleton
+    }
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(CassandraEnumeratorState cassandraEnumeratorState) throws IOException {
+        try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                ObjectOutputStream objectOutputStream =
+                        new ObjectOutputStream(byteArrayOutputStream)) {
+            objectOutputStream.writeObject(cassandraEnumeratorState);

Review Comment:
   Don't use java serialization. Manually write the components that make of a split for easier migration in the future.
   https://flink.apache.org/contributing/code-style-and-quality-java.html#java-serialization



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            return Collections.singletonList(
+                    new CassandraSplit(Collections.singleton(totalRingRange)));
+        }
+        int tokenRangeCount = ringTokens.size();
+
+        List<RingRange> ringRanges = new ArrayList<>();
+        for (int i = 0; i < tokenRangeCount; i++) {
+            BigInteger start = ringTokens.get(i);
+            BigInteger stop = ringTokens.get((i + 1) % tokenRangeCount);
+
+            if (isNotInRange(start) || isNotInRange(stop)) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s) not in range of %s", start, stop, partitioner));
+            }
+            if (start.equals(stop) && tokenRangeCount != 1) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s): two nodes have the same token", start, stop));
+            }
+
+            BigInteger rangeSize = stop.subtract(start);
+            if (rangeSize.compareTo(BigInteger.ZERO) <= 0) {
+                // wrap around case
+                rangeSize = rangeSize.add(this.rangeSize);
+            }

Review Comment:
   this is the third implementation for computing the size/span of a range.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();

Review Comment:
   Where do these numbers come from?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {
+            // allow to interrupt the reading of splits as requested in the API
+            if (wakeup.get()) {
+                break;
+            }
+            if (!cassandraSplitState.isEmpty()) {
+                try {
+                    final Set<RingRange> ringRanges =
+                            cassandraSplitState.getUnprocessedRingRanges();
+                    final String cassandraSplitId = cassandraSplitState.getSplitId();
+
+                    for (RingRange ringRange : ringRanges) {
+                        Token startToken =
+                                clusterMetadata.newToken(ringRange.getStart().toString());
+                        Token endToken = clusterMetadata.newToken(ringRange.getEnd().toString());
+                        if (ringRange.isWrapping()) {
+                            // A wrapping range is one that overlaps from the end of the partitioner
+                            // range and its
+                            // start (ie : when the start token of the split is greater than the end
+                            // token)
+                            // We need to generate two queries here : one that goes from the start
+                            // token to the end
+                            // of
+                            // the partitioner range, and the other from the start of the
+                            // partitioner range to the
+                            // end token of the split.
+
+                            addRecordsToOutput(
+                                    session.execute(
+                                            getLowestSplitQuery(
+                                                    query, partitionKey, ringRange.getEnd())),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                            addRecordsToOutput(
+                                    session.execute(
+                                            getHighestSplitQuery(
+                                                    query, partitionKey, ringRange.getStart())),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                        } else {
+                            addRecordsToOutput(
+                                    session.execute(
+                                            preparedStatement
+                                                    .bind()
+                                                    .setToken(0, startToken)
+                                                    .setToken(1, endToken)),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                        }
+                        cassandraSplitState.markRingRangeAsFinished(ringRange);
+                    }
+                    // put the already read split to finished splits
+                    finishedSplits.add(cassandraSplitState.getSplitId());
+                    // for reentrant calls: if fetch is woken up,
+                    // do not reprocess the already processed splits
+                    unprocessedSplits.remove(cassandraSplitState);
+                } catch (Exception ex) {
+                    LOG.error("Error while reading split ", ex);
+                }
+            } else {
+                finishedSplits.add(cassandraSplitState.getSplitId());
+            }
+        }
+        return new RecordsBySplits<>(recordsBySplit, finishedSplits);
+    }
+
+    private String getPartitionKey(Metadata clusterMetadata) {
+        Matcher queryMatcher = Pattern.compile(SELECT_REGEXP).matcher(query);
+        if (!queryMatcher.matches()) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Failed to extract keyspace and table out of the provided query: %s",
+                            query));
+        }
+        String keyspace = queryMatcher.group(1);
+        String table = queryMatcher.group(2);
+        return clusterMetadata.getKeyspace(keyspace).getTable(table).getPartitionKey().stream()
+                .map(ColumnMetadata::getName)
+                .collect(Collectors.joining(","));
+    }
+
+    @Override
+    public void wakeUp() {
+        wakeup.compareAndSet(false, true);
+    }
+
+    @Override
+    public void handleSplitsChanges(SplitsChange<CassandraSplit> splitsChanges) {
+        for (CassandraSplit cassandraSplit : splitsChanges.splits()) {
+            unprocessedSplits.add(cassandraSplit.toSplitState());
+        }
+    }
+
+    @VisibleForTesting
+    static String getHighestSplitQuery(String query, String partitionKey, BigInteger highest) {
+        return generateQuery(
+                query, partitionKey, highest, " (token(%s) >= %d) AND", " WHERE (token(%s) >= %d)");
+    }
+
+    @VisibleForTesting
+    static String getLowestSplitQuery(String query, String partitionKey, BigInteger lowest) {
+        return generateQuery(
+                query, partitionKey, lowest, " (token(%s) < %d) AND", " WHERE (token(%s) < %d)");
+    }
+
+    @VisibleForTesting
+    static String generateRangeQuery(String query, String partitionKey) {
+        return generateQuery(
+                query,
+                partitionKey,
+                null,
+                " (token(%s) >= ?) AND (token(%s) < ?) AND",
+                " WHERE (token(%s) >= ?) AND (token(%s) < ?)");
+    }
+
+    private static String generateQuery(
+            String query,
+            String partitionKey,
+            @Nullable BigInteger token,
+            String whereFilter,
+            String noWhereFilter) {
+        Matcher queryMatcher = Pattern.compile(SELECT_REGEXP).matcher(query);
+        if (!queryMatcher.matches()) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Failed to extract keyspace and table out of the provided query: %s",
+                            query));
+        }
+        final int whereIndex = query.toLowerCase().indexOf("where");
+        int insertionPoint;
+        String filter;
+        if (whereIndex != -1) {
+            insertionPoint = whereIndex + "where".length();
+            filter =
+                    (token == null)
+                            ? String.format(whereFilter, partitionKey, partitionKey)
+                            : String.format(whereFilter, partitionKey, token);
+        } else {
+            // end of keyspace.table
+            insertionPoint = queryMatcher.end(2);
+            filter =
+                    (token == null)
+                            ? String.format(noWhereFilter, partitionKey, partitionKey)
+                            : String.format(noWhereFilter, partitionKey, token);
+        }
+        return String.format(
+                "%s%s%s",
+                query.substring(0, insertionPoint), filter, query.substring(insertionPoint));
+    }
+
+    /**
+     * This method populates the {@code Map<String, Collection<CassandraRow>> recordsBySplit} map
+     * that is used to create the {@link RecordsBySplits} that are output by the fetch method. It
+     * modifies its {@code output} parameter.
+     */
+    private void addRecordsToOutput(
+            ResultSet resultSet,

Review Comment:
   Why aren't we doing the row->pojo mapping here?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            return Collections.singletonList(
+                    new CassandraSplit(Collections.singleton(totalRingRange)));
+        }
+        int tokenRangeCount = ringTokens.size();
+
+        List<RingRange> ringRanges = new ArrayList<>();
+        for (int i = 0; i < tokenRangeCount; i++) {
+            BigInteger start = ringTokens.get(i);
+            BigInteger stop = ringTokens.get((i + 1) % tokenRangeCount);
+
+            if (isNotInRange(start) || isNotInRange(stop)) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s) not in range of %s", start, stop, partitioner));
+            }
+            if (start.equals(stop) && tokenRangeCount != 1) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s): two nodes have the same token", start, stop));
+            }
+
+            BigInteger rangeSize = stop.subtract(start);
+            if (rangeSize.compareTo(BigInteger.ZERO) <= 0) {
+                // wrap around case
+                rangeSize = rangeSize.add(this.rangeSize);
+            }
+
+            // the below, in essence, does this:
+            // splitCount = Maths.ceil((rangeSize / cluster range size) * totalSplitCount)
+            BigInteger[] splitCountAndRemainder =

Review Comment:
   I'd appreciate some more comments as to what _all_ the code below this line does.
   My brain can't really comprehend it right now.
   It looks like we so far computed the token ranges, now we're splitting them (to get closer to the desired split count?) and then merge them again for some token requirement or something.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitSerializer.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+/** Serializer for {@link CassandraSplit}. */
+public class CassandraSplitSerializer implements SimpleVersionedSerializer<CassandraSplit> {
+
+    public static final CassandraSplitSerializer INSTANCE = new CassandraSplitSerializer();
+
+    public static final int CURRENT_VERSION = 0;
+
+    private CassandraSplitSerializer() {}
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(CassandraSplit cassandraSplit) throws IOException {
+        try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                ObjectOutputStream objectOutputStream =
+                        new ObjectOutputStream(byteArrayOutputStream)) {
+            objectOutputStream.writeObject(cassandraSplit);

Review Comment:
   Don't use java serialization. Manually write the components that make of a split for easier migration in the future.
   https://flink.apache.org/contributing/code-style-and-quality-java.html#java-serialization



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            return Collections.singletonList(
+                    new CassandraSplit(Collections.singleton(totalRingRange)));
+        }
+        int tokenRangeCount = ringTokens.size();
+
+        List<RingRange> ringRanges = new ArrayList<>();
+        for (int i = 0; i < tokenRangeCount; i++) {
+            BigInteger start = ringTokens.get(i);
+            BigInteger stop = ringTokens.get((i + 1) % tokenRangeCount);
+
+            if (isNotInRange(start) || isNotInRange(stop)) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s) not in range of %s", start, stop, partitioner));
+            }
+            if (start.equals(stop) && tokenRangeCount != 1) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s): two nodes have the same token", start, stop));
+            }
+
+            BigInteger rangeSize = stop.subtract(start);
+            if (rangeSize.compareTo(BigInteger.ZERO) <= 0) {
+                // wrap around case
+                rangeSize = rangeSize.add(this.rangeSize);
+            }
+
+            // the below, in essence, does this:
+            // splitCount = Maths.ceil((rangeSize / cluster range size) * totalSplitCount)
+            BigInteger[] splitCountAndRemainder =
+                    rangeSize
+                            .multiply(BigInteger.valueOf(totalSplitCount))
+                            .divideAndRemainder(this.rangeSize);
+
+            int splitCount =
+                    splitCountAndRemainder[0].intValue()
+                            + (splitCountAndRemainder[1].equals(BigInteger.ZERO) ? 0 : 1);
+
+            LOG.debug("Dividing token range [{},{}) into {} splits", start, stop, splitCount);
+
+            // Make BigInteger list of all the endpoints for the splits, including both start and
+            // stop
+            List<BigInteger> endpointTokens = new ArrayList<>();
+            for (int j = 0; j <= splitCount; j++) {
+                BigInteger offset =
+                        rangeSize
+                                .multiply(BigInteger.valueOf(j))
+                                .divide(BigInteger.valueOf(splitCount));
+                BigInteger token = start.add(offset);
+                if (token.compareTo(rangeMax) > 0) {
+                    token = token.subtract(this.rangeSize);
+                }
+                // Long.MIN_VALUE is not a valid token and has to be silently incremented.
+                // See https://issues.apache.org/jira/browse/CASSANDRA-14684
+                endpointTokens.add(
+                        token.equals(BigInteger.valueOf(Long.MIN_VALUE))
+                                ? token.add(BigInteger.ONE)
+                                : token);
+            }
+
+            // Append the ringRanges between the endpoints
+            for (int j = 0; j < splitCount; j++) {
+                ringRanges.add(RingRange.of(endpointTokens.get(j), endpointTokens.get(j + 1)));
+                LOG.debug(
+                        "Split #{}: [{},{})",
+                        j + 1,
+                        endpointTokens.get(j),
+                        endpointTokens.get(j + 1));
+            }
+        }
+
+        BigInteger total = BigInteger.ZERO;
+        for (RingRange split : ringRanges) {
+            BigInteger size = split.span(rangeSize);
+            total = total.add(size);
+        }
+        if (!total.equals(rangeSize)) {
+            throw new RuntimeException(
+                    "Some tokens are missing from the splits. This should not happen.");
+        }
+        return coalesceRingRanges(getTargetSplitSize(totalSplitCount), ringRanges);
+    }
+
+    private boolean isNotInRange(BigInteger token) {
+        return token.compareTo(rangeMin) < 0 || token.compareTo(rangeMax) > 0;
+    }
+
+    private List<CassandraSplit> coalesceRingRanges(
+            BigInteger targetSplitSize, List<RingRange> ringRanges) {
+        List<CassandraSplit> coalescedSplits = new ArrayList<>();
+        List<RingRange> tokenRangesForCurrentSplit = new ArrayList<>();
+        BigInteger tokenCount = BigInteger.ZERO;
+
+        for (RingRange tokenRange : ringRanges) {
+            if (tokenRange.span(rangeSize).add(tokenCount).compareTo(targetSplitSize) > 0
+                    && !tokenRangesForCurrentSplit.isEmpty()) {
+                // enough tokens in that segment

Review Comment:
   enough tokens _for what_? Can you explain a bit what tokens are?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set

Review Comment:
   explain _why_ we need it



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            return Collections.singletonList(
+                    new CassandraSplit(Collections.singleton(totalRingRange)));
+        }
+        int tokenRangeCount = ringTokens.size();
+
+        List<RingRange> ringRanges = new ArrayList<>();
+        for (int i = 0; i < tokenRangeCount; i++) {
+            BigInteger start = ringTokens.get(i);
+            BigInteger stop = ringTokens.get((i + 1) % tokenRangeCount);
+
+            if (isNotInRange(start) || isNotInRange(stop)) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s) not in range of %s", start, stop, partitioner));
+            }
+            if (start.equals(stop) && tokenRangeCount != 1) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s): two nodes have the same token", start, stop));
+            }
+
+            BigInteger rangeSize = stop.subtract(start);
+            if (rangeSize.compareTo(BigInteger.ZERO) <= 0) {
+                // wrap around case
+                rangeSize = rangeSize.add(this.rangeSize);
+            }
+
+            // the below, in essence, does this:
+            // splitCount = Maths.ceil((rangeSize / cluster range size) * totalSplitCount)
+            BigInteger[] splitCountAndRemainder =
+                    rangeSize
+                            .multiply(BigInteger.valueOf(totalSplitCount))
+                            .divideAndRemainder(this.rangeSize);
+
+            int splitCount =
+                    splitCountAndRemainder[0].intValue()
+                            + (splitCountAndRemainder[1].equals(BigInteger.ZERO) ? 0 : 1);
+
+            LOG.debug("Dividing token range [{},{}) into {} splits", start, stop, splitCount);
+
+            // Make BigInteger list of all the endpoints for the splits, including both start and
+            // stop
+            List<BigInteger> endpointTokens = new ArrayList<>();
+            for (int j = 0; j <= splitCount; j++) {
+                BigInteger offset =
+                        rangeSize
+                                .multiply(BigInteger.valueOf(j))
+                                .divide(BigInteger.valueOf(splitCount));
+                BigInteger token = start.add(offset);
+                if (token.compareTo(rangeMax) > 0) {
+                    token = token.subtract(this.rangeSize);
+                }
+                // Long.MIN_VALUE is not a valid token and has to be silently incremented.
+                // See https://issues.apache.org/jira/browse/CASSANDRA-14684
+                endpointTokens.add(
+                        token.equals(BigInteger.valueOf(Long.MIN_VALUE))
+                                ? token.add(BigInteger.ONE)
+                                : token);
+            }
+
+            // Append the ringRanges between the endpoints
+            for (int j = 0; j < splitCount; j++) {
+                ringRanges.add(RingRange.of(endpointTokens.get(j), endpointTokens.get(j + 1)));
+                LOG.debug(
+                        "Split #{}: [{},{})",
+                        j + 1,
+                        endpointTokens.get(j),
+                        endpointTokens.get(j + 1));
+            }
+        }
+
+        BigInteger total = BigInteger.ZERO;
+        for (RingRange split : ringRanges) {
+            BigInteger size = split.span(rangeSize);
+            total = total.add(size);
+        }
+        if (!total.equals(rangeSize)) {
+            throw new RuntimeException(
+                    "Some tokens are missing from the splits. This should not happen.");
+        }
+        return coalesceRingRanges(getTargetSplitSize(totalSplitCount), ringRanges);
+    }
+
+    private boolean isNotInRange(BigInteger token) {
+        return token.compareTo(rangeMin) < 0 || token.compareTo(rangeMax) > 0;
+    }
+
+    private List<CassandraSplit> coalesceRingRanges(
+            BigInteger targetSplitSize, List<RingRange> ringRanges) {
+        List<CassandraSplit> coalescedSplits = new ArrayList<>();
+        List<RingRange> tokenRangesForCurrentSplit = new ArrayList<>();
+        BigInteger tokenCount = BigInteger.ZERO;
+
+        for (RingRange tokenRange : ringRanges) {
+            if (tokenRange.span(rangeSize).add(tokenCount).compareTo(targetSplitSize) > 0
+                    && !tokenRangesForCurrentSplit.isEmpty()) {
+                // enough tokens in that segment
+                LOG.debug(
+                        "Got enough tokens for one split ({}) : {}",
+                        tokenCount,
+                        tokenRangesForCurrentSplit);
+                coalescedSplits.add(new CassandraSplit(new HashSet<>(tokenRangesForCurrentSplit)));
+                tokenRangesForCurrentSplit = new ArrayList<>();
+                tokenCount = BigInteger.ZERO;
+            }
+
+            tokenCount = tokenCount.add(tokenRange.span(rangeSize));
+            tokenRangesForCurrentSplit.add(tokenRange);
+        }
+
+        if (!tokenRangesForCurrentSplit.isEmpty()) {
+            coalescedSplits.add(new CassandraSplit(new HashSet<>(tokenRangesForCurrentSplit)));
+        }
+        return coalescedSplits;
+    }
+
+    private BigInteger getTargetSplitSize(long splitCount) {
+        return rangeMax.subtract(rangeMin).divide(BigInteger.valueOf(splitCount));

Review Comment:
   Are tokens evenly distributed in the range or do they depend on the number of elements within a range?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraRecordEmitter.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.api.connector.source.SourceOutput;
+import org.apache.flink.connector.base.source.reader.RecordEmitter;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.ExecutionInfo;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.mapping.Mapper;
+import com.datastax.driver.mapping.MappingManager;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * {@link RecordEmitter} that converts the {@link CassandraRow} read by the {@link
+ * CassandraSplitReader} to specified POJO and output it while updating splits state. This class
+ * uses the Cassandra driver mapper to map the row to the POJO.
+ *
+ * @param <OUT> type of POJO record to output
+ */
+public class CassandraRecordEmitter<OUT>
+        implements RecordEmitter<CassandraRow, OUT, CassandraSplitState> {
+
+    private final Mapper<OUT> mapper;
+
+    public CassandraRecordEmitter(
+            Class<OUT> pojoClass, ClusterBuilder clusterBuilder, MapperOptions mapperOptions) {
+        // session and cluster are managed at the SplitReader level. So we need to create one
+        // locally here just to me able to create the mapper.
+        final Cluster cluster = clusterBuilder.getCluster();
+        final Session session = cluster.connect();

Review Comment:
   this seems sketchy. Have you considered just having the user provide a mapping function from row -> pojo?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {
+            // allow to interrupt the reading of splits as requested in the API
+            if (wakeup.get()) {
+                break;
+            }
+            if (!cassandraSplitState.isEmpty()) {
+                try {
+                    final Set<RingRange> ringRanges =
+                            cassandraSplitState.getUnprocessedRingRanges();
+                    final String cassandraSplitId = cassandraSplitState.getSplitId();
+
+                    for (RingRange ringRange : ringRanges) {
+                        Token startToken =
+                                clusterMetadata.newToken(ringRange.getStart().toString());
+                        Token endToken = clusterMetadata.newToken(ringRange.getEnd().toString());
+                        if (ringRange.isWrapping()) {
+                            // A wrapping range is one that overlaps from the end of the partitioner
+                            // range and its
+                            // start (ie : when the start token of the split is greater than the end
+                            // token)
+                            // We need to generate two queries here : one that goes from the start
+                            // token to the end
+                            // of
+                            // the partitioner range, and the other from the start of the
+                            // partitioner range to the
+                            // end token of the split.
+
+                            addRecordsToOutput(
+                                    session.execute(
+                                            getLowestSplitQuery(
+                                                    query, partitionKey, ringRange.getEnd())),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                            addRecordsToOutput(
+                                    session.execute(
+                                            getHighestSplitQuery(
+                                                    query, partitionKey, ringRange.getStart())),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                        } else {
+                            addRecordsToOutput(
+                                    session.execute(
+                                            preparedStatement
+                                                    .bind()
+                                                    .setToken(0, startToken)
+                                                    .setToken(1, endToken)),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                        }
+                        cassandraSplitState.markRingRangeAsFinished(ringRange);
+                    }
+                    // put the already read split to finished splits
+                    finishedSplits.add(cassandraSplitState.getSplitId());
+                    // for reentrant calls: if fetch is woken up,
+                    // do not reprocess the already processed splits
+                    unprocessedSplits.remove(cassandraSplitState);
+                } catch (Exception ex) {
+                    LOG.error("Error while reading split ", ex);
+                }
+            } else {
+                finishedSplits.add(cassandraSplitState.getSplitId());
+            }
+        }
+        return new RecordsBySplits<>(recordsBySplit, finishedSplits);
+    }
+
+    private String getPartitionKey(Metadata clusterMetadata) {
+        Matcher queryMatcher = Pattern.compile(SELECT_REGEXP).matcher(query);
+        if (!queryMatcher.matches()) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Failed to extract keyspace and table out of the provided query: %s",
+                            query));
+        }
+        String keyspace = queryMatcher.group(1);
+        String table = queryMatcher.group(2);
+        return clusterMetadata.getKeyspace(keyspace).getTable(table).getPartitionKey().stream()
+                .map(ColumnMetadata::getName)
+                .collect(Collectors.joining(","));
+    }
+
+    @Override
+    public void wakeUp() {
+        wakeup.compareAndSet(false, true);
+    }
+
+    @Override
+    public void handleSplitsChanges(SplitsChange<CassandraSplit> splitsChanges) {
+        for (CassandraSplit cassandraSplit : splitsChanges.splits()) {
+            unprocessedSplits.add(cassandraSplit.toSplitState());
+        }
+    }
+
+    @VisibleForTesting
+    static String getHighestSplitQuery(String query, String partitionKey, BigInteger highest) {
+        return generateQuery(
+                query, partitionKey, highest, " (token(%s) >= %d) AND", " WHERE (token(%s) >= %d)");
+    }
+
+    @VisibleForTesting
+    static String getLowestSplitQuery(String query, String partitionKey, BigInteger lowest) {
+        return generateQuery(
+                query, partitionKey, lowest, " (token(%s) < %d) AND", " WHERE (token(%s) < %d)");
+    }
+
+    @VisibleForTesting
+    static String generateRangeQuery(String query, String partitionKey) {
+        return generateQuery(
+                query,
+                partitionKey,
+                null,
+                " (token(%s) >= ?) AND (token(%s) < ?) AND",
+                " WHERE (token(%s) >= ?) AND (token(%s) < ?)");
+    }
+
+    private static String generateQuery(

Review Comment:
   can we get some before/after examples for this.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {
+            // allow to interrupt the reading of splits as requested in the API
+            if (wakeup.get()) {

Review Comment:
   Which calls in this method are blocking?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {
+            // allow to interrupt the reading of splits as requested in the API
+            if (wakeup.get()) {
+                break;
+            }
+            if (!cassandraSplitState.isEmpty()) {
+                try {
+                    final Set<RingRange> ringRanges =
+                            cassandraSplitState.getUnprocessedRingRanges();
+                    final String cassandraSplitId = cassandraSplitState.getSplitId();
+
+                    for (RingRange ringRange : ringRanges) {
+                        Token startToken =
+                                clusterMetadata.newToken(ringRange.getStart().toString());
+                        Token endToken = clusterMetadata.newToken(ringRange.getEnd().toString());
+                        if (ringRange.isWrapping()) {
+                            // A wrapping range is one that overlaps from the end of the partitioner
+                            // range and its
+                            // start (ie : when the start token of the split is greater than the end
+                            // token)
+                            // We need to generate two queries here : one that goes from the start
+                            // token to the end
+                            // of
+                            // the partitioner range, and the other from the start of the
+                            // partitioner range to the
+                            // end token of the split.

Review Comment:
   Why not model this as 2 separate ranges from the beginning?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";

Review Comment:
   Do we really need to hard-code the class here? If we can't access the class to infer the name, would it make sense to just check whether the returned name _contains_ "Murmur3Partitioner"?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits, enumeratorContext.currentParallelism());
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitioner = clusterMetadata.getPartitioner();
+        final SplitsGenerator splitsGenerator = new SplitsGenerator(partitioner);
+        if (MURMUR3PARTITIONER.equals(partitioner)) {
+            LOG.info("Murmur3Partitioner detected, splitting");
+            List<BigInteger> tokens =
+                    clusterMetadata.getTokenRanges().stream()
+                            .map(
+                                    tokenRange ->
+                                            new BigInteger(
+                                                    tokenRange.getEnd().getValue().toString()))
+                            .collect(Collectors.toList());
+            return splitsGenerator.generateSplits(numberOfSplits, tokens);
+        } else {
+            // Murmur3Partitioner is the default and recommended partitioner for Cassandra 1.2+
+            // see
+            // https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerAbout.html
+            LOG.warn(
+                    "The current Cassandra partitioner is {}, only Murmur3Partitioner is supported "

Review Comment:
   What about the random partitioner? Shouldn't it be possible to create multiple splits?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits, enumeratorContext.currentParallelism());
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitioner = clusterMetadata.getPartitioner();
+        final SplitsGenerator splitsGenerator = new SplitsGenerator(partitioner);
+        if (MURMUR3PARTITIONER.equals(partitioner)) {
+            LOG.info("Murmur3Partitioner detected, splitting");
+            List<BigInteger> tokens =
+                    clusterMetadata.getTokenRanges().stream()
+                            .map(
+                                    tokenRange ->
+                                            new BigInteger(
+                                                    tokenRange.getEnd().getValue().toString()))
+                            .collect(Collectors.toList());
+            return splitsGenerator.generateSplits(numberOfSplits, tokens);
+        } else {
+            // Murmur3Partitioner is the default and recommended partitioner for Cassandra 1.2+
+            // see
+            // https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerAbout.html
+            LOG.warn(
+                    "The current Cassandra partitioner is {}, only Murmur3Partitioner is supported "
+                            + "for splitting, using an single split",

Review Comment:
   ```suggestion
                               + "for splitting, using a single split",
   ```



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorStateSerializer;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraSplitEnumerator;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSourceReader;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSplitReader;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A bounded source to read from Cassandra and return a collection of entities as {@code
+ * DataStream<Entity>}. An entity is built by Cassandra mapper ({@code
+ * com.datastax.driver.mapping.EntityMapper}) based on a POJO containing annotations (as described
+ * in <a
+ * href="https://docs.datastax.com/en/developer/java-driver/3.11/manual/object_mapper/creating/">
+ * Cassandra object mapper</a>).
+ *
+ * <p>To use it, do the following:
+ *
+ * <pre>{@code
+ * ClusterBuilder clusterBuilder = new ClusterBuilder() {
+ *   @Override
+ *   protected Cluster buildCluster(Cluster.Builder builder) {
+ *     return builder.addContactPointsWithPorts(new InetSocketAddress(HOST,PORT))
+ *                   .withQueryOptions(new QueryOptions().setConsistencyLevel(CL))
+ *                   .withSocketOptions(new SocketOptions()
+ *                   .setConnectTimeoutMillis(CONNECT_TIMEOUT)
+ *                   .setReadTimeoutMillis(READ_TIMEOUT))
+ *                   .build();
+ *   }
+ * };
+ * Source cassandraSource = new CassandraSource(clusterBuilder,
+ *                                              Pojo.class,
+ *                                              "select ... from KEYSPACE.TABLE ...;",
+ *                                              () -> new Mapper.Option[] {Mapper.Option.saveNullFields(true)});
+ *
+ * DataStream<Pojo> stream = env.fromSource(cassandraSource, WatermarkStrategy.noWatermarks(),
+ * "CassandraSource");
+ * }</pre>
+ */
+@PublicEvolving
+public class CassandraSource<OUT>
+        implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
+
+    public static final String CQL_PROHIBITTED_CLAUSES_REGEXP =
+            "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    private static final long serialVersionUID = 7773196541275567433L;
+
+    private final ClusterBuilder clusterBuilder;
+    private final Class<OUT> pojoClass;
+    private final String query;
+    private final MapperOptions mapperOptions;
+
+    public CassandraSource(
+            ClusterBuilder clusterBuilder,
+            Class<OUT> pojoClass,
+            String query,
+            MapperOptions mapperOptions) {
+        checkNotNull(clusterBuilder, "ClusterBuilder required but not provided");
+        checkNotNull(pojoClass, "POJO class required but not provided");
+        checkQueryValidity(query);
+        this.clusterBuilder = clusterBuilder;
+        ClosureCleaner.clean(clusterBuilder, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+        this.pojoClass = pojoClass;
+        this.query = query;
+        this.mapperOptions = mapperOptions;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return Boundedness.BOUNDED;
+    }
+
+    @Internal
+    @Override
+    public SourceReader<OUT, CassandraSplit> createReader(SourceReaderContext readerContext) {
+        return new CassandraSourceReader<>(
+                readerContext, clusterBuilder, pojoClass, query, mapperOptions);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> createEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext) {
+        return new CassandraSplitEnumerator(enumContext, null, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SplitEnumerator<CassandraSplit, CassandraEnumeratorState> restoreEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumContext,
+            CassandraEnumeratorState enumCheckpoint) {
+        return new CassandraSplitEnumerator(enumContext, enumCheckpoint, clusterBuilder);
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraSplit> getSplitSerializer() {
+        return CassandraSplitSerializer.INSTANCE;
+    }
+
+    @Internal
+    @Override
+    public SimpleVersionedSerializer<CassandraEnumeratorState> getEnumeratorCheckpointSerializer() {
+        return CassandraEnumeratorStateSerializer.INSTANCE;
+    }
+
+    @Override
+    public TypeInformation<OUT> getProducedType() {
+        return TypeInformation.of(pojoClass);
+    }
+
+    @VisibleForTesting
+    public static void checkQueryValidity(String query) {
+        checkNotNull(query, "query required but not provided");
+        checkState(
+                query.matches(CassandraSplitReader.SELECT_REGEXP),
+                "query must be of the form select ... from keyspace.table ...;");
+        checkState(
+                !query.matches(CQL_PROHIBITTED_CLAUSES_REGEXP),
+                "query must not contain aggregate or order clauses because they will be done per split. "
+                        + "So they will be incorrect after merging the splits");

Review Comment:
   users wont understand why splits are being merged,



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073636678


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits, enumeratorContext.currentParallelism());
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitioner = clusterMetadata.getPartitioner();
+        final SplitsGenerator splitsGenerator = new SplitsGenerator(partitioner);
+        if (MURMUR3PARTITIONER.equals(partitioner)) {
+            LOG.info("Murmur3Partitioner detected, splitting");
+            List<BigInteger> tokens =
+                    clusterMetadata.getTokenRanges().stream()
+                            .map(
+                                    tokenRange ->
+                                            new BigInteger(
+                                                    tokenRange.getEnd().getValue().toString()))
+                            .collect(Collectors.toList());
+            return splitsGenerator.generateSplits(numberOfSplits, tokens);
+        } else {
+            // Murmur3Partitioner is the default and recommended partitioner for Cassandra 1.2+
+            // see
+            // https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerAbout.html
+            LOG.warn(
+                    "The current Cassandra partitioner is {}, only Murmur3Partitioner is supported "

Review Comment:
   See my general comments about Apache Beam Cassandra partitionner



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
zentol commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1396864643

   > I did not author the RingRange and SplitsGenerator classes. I got them from the Apache Beam Cassandra connector.
   
   If you're copying stuff from other projects it please add some notice/attribution to the files and update the Flink source notice accordingly.
   
   > Back in 2017 I coded a [splitter for Cassandra Beam connector](https://github.com/echauchot/beam/blob/bfa33b85b6b310556ffa5c44c99bef50575b2c56/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java#L346) that works [with tokens](https://github.com/echauchot/beam/blob/BEAM-245-CassandraIO/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/DataSizeEstimates.java) also but that is simpler and supports all the Cassandra partitionners. Would you prefer that we use this other approach ?
   
   Not sure? Why didn't it make it into Beam? Do you know why the Beam code is written the way it is?


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073721942


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitSerializer.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+/** Serializer for {@link CassandraSplit}. */
+public class CassandraSplitSerializer implements SimpleVersionedSerializer<CassandraSplit> {
+
+    public static final CassandraSplitSerializer INSTANCE = new CassandraSplitSerializer();
+
+    public static final int CURRENT_VERSION = 0;
+
+    private CassandraSplitSerializer() {}
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(CassandraSplit cassandraSplit) throws IOException {
+        try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                ObjectOutputStream objectOutputStream =
+                        new ObjectOutputStream(byteArrayOutputStream)) {
+            objectOutputStream.writeObject(cassandraSplit);

Review Comment:
   For my knowledge, where is the use of java serialization acceptable in Flink ? In the docs it says only for RPC data but I see it in use for state snapshots as well



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1085585517


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   No. `CassandraSplits` are assigned to `CassandraSplitReader` (see `CassandraSplitEnumerator#assignUnprocessedSplitsToReader`). A `CassandraSplitReader` might be assigned several splits in case the **strange** splitGenerator returns more splits than requested parallelism. With the new `splitGenerator` there will be exactly as many splits as the parallelism.  So here we read all the records that belong to **the splits assigned to the reader***. 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1086404409


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {
+            // allow to interrupt the reading of splits as requested in the API
+            if (wakeup.get()) {
+                break;
+            }
+            if (!cassandraSplitState.isEmpty()) {
+                try {
+                    final Set<RingRange> ringRanges =
+                            cassandraSplitState.getUnprocessedRingRanges();
+                    final String cassandraSplitId = cassandraSplitState.getSplitId();
+
+                    for (RingRange ringRange : ringRanges) {
+                        Token startToken =
+                                clusterMetadata.newToken(ringRange.getStart().toString());
+                        Token endToken = clusterMetadata.newToken(ringRange.getEnd().toString());
+                        if (ringRange.isWrapping()) {
+                            // A wrapping range is one that overlaps from the end of the partitioner
+                            // range and its
+                            // start (ie : when the start token of the split is greater than the end
+                            // token)
+                            // We need to generate two queries here : one that goes from the start
+                            // token to the end
+                            // of
+                            // the partitioner range, and the other from the start of the
+                            // partitioner range to the
+                            // end token of the split.
+
+                            addRecordsToOutput(
+                                    session.execute(
+                                            getLowestSplitQuery(
+                                                    query, partitionKey, ringRange.getEnd())),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                            addRecordsToOutput(
+                                    session.execute(
+                                            getHighestSplitQuery(
+                                                    query, partitionKey, ringRange.getStart())),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                        } else {
+                            addRecordsToOutput(
+                                    session.execute(
+                                            preparedStatement
+                                                    .bind()
+                                                    .setToken(0, startToken)
+                                                    .setToken(1, endToken)),
+                                    recordsBySplit,
+                                    cassandraSplitId,
+                                    ringRange);
+                        }
+                        cassandraSplitState.markRingRangeAsFinished(ringRange);
+                    }
+                    // put the already read split to finished splits
+                    finishedSplits.add(cassandraSplitState.getSplitId());
+                    // for reentrant calls: if fetch is woken up,
+                    // do not reprocess the already processed splits
+                    unprocessedSplits.remove(cassandraSplitState);
+                } catch (Exception ex) {
+                    LOG.error("Error while reading split ", ex);
+                }
+            } else {
+                finishedSplits.add(cassandraSplitState.getSplitId());
+            }
+        }
+        return new RecordsBySplits<>(recordsBySplit, finishedSplits);
+    }
+
+    private String getPartitionKey(Metadata clusterMetadata) {
+        Matcher queryMatcher = Pattern.compile(SELECT_REGEXP).matcher(query);
+        if (!queryMatcher.matches()) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Failed to extract keyspace and table out of the provided query: %s",
+                            query));
+        }
+        String keyspace = queryMatcher.group(1);
+        String table = queryMatcher.group(2);
+        return clusterMetadata.getKeyspace(keyspace).getTable(table).getPartitionKey().stream()
+                .map(ColumnMetadata::getName)
+                .collect(Collectors.joining(","));
+    }
+
+    @Override
+    public void wakeUp() {
+        wakeup.compareAndSet(false, true);
+    }
+
+    @Override
+    public void handleSplitsChanges(SplitsChange<CassandraSplit> splitsChanges) {
+        for (CassandraSplit cassandraSplit : splitsChanges.splits()) {
+            unprocessedSplits.add(cassandraSplit.toSplitState());
+        }
+    }
+
+    @VisibleForTesting
+    static String getHighestSplitQuery(String query, String partitionKey, BigInteger highest) {
+        return generateQuery(
+                query, partitionKey, highest, " (token(%s) >= %d) AND", " WHERE (token(%s) >= %d)");
+    }
+
+    @VisibleForTesting
+    static String getLowestSplitQuery(String query, String partitionKey, BigInteger lowest) {
+        return generateQuery(
+                query, partitionKey, lowest, " (token(%s) < %d) AND", " WHERE (token(%s) < %d)");
+    }
+
+    @VisibleForTesting
+    static String generateRangeQuery(String query, String partitionKey) {
+        return generateQuery(
+                query,
+                partitionKey,
+                null,
+                " (token(%s) >= ?) AND (token(%s) < ?) AND",
+                " WHERE (token(%s) >= ?) AND (token(%s) < ?)");
+    }
+
+    private static String generateQuery(
+            String query,
+            String partitionKey,
+            @Nullable BigInteger token,
+            String whereFilter,
+            String noWhereFilter) {
+        Matcher queryMatcher = Pattern.compile(SELECT_REGEXP).matcher(query);
+        if (!queryMatcher.matches()) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Failed to extract keyspace and table out of the provided query: %s",
+                            query));
+        }
+        final int whereIndex = query.toLowerCase().indexOf("where");
+        int insertionPoint;
+        String filter;
+        if (whereIndex != -1) {
+            insertionPoint = whereIndex + "where".length();
+            filter =
+                    (token == null)
+                            ? String.format(whereFilter, partitionKey, partitionKey)
+                            : String.format(whereFilter, partitionKey, token);
+        } else {
+            // end of keyspace.table
+            insertionPoint = queryMatcher.end(2);
+            filter =
+                    (token == null)
+                            ? String.format(noWhereFilter, partitionKey, partitionKey)
+                            : String.format(noWhereFilter, partitionKey, token);
+        }
+        return String.format(
+                "%s%s%s",
+                query.substring(0, insertionPoint), filter, query.substring(insertionPoint));
+    }
+
+    /**
+     * This method populates the {@code Map<String, Collection<CassandraRow>> recordsBySplit} map
+     * that is used to create the {@link RecordsBySplits} that are output by the fetch method. It
+     * modifies its {@code output} parameter.
+     */
+    private void addRecordsToOutput(
+            ResultSet resultSet,

Review Comment:
   Because in the javadoc of `SingleThreadMultiplexSourceReaderBase` the responsibilities are set as this:
   `A RecordEmitter that takes a record from the Split Reader and updates the checkpointing state and converts it into the final form. For example for Kafka, the Record Emitter takes a ConsumerRecord, puts the offset information into state, transforms the records with the deserializers into the final type, and emits the record.` So  `CassandraSplitReader` deals with Cassandra `Row` and `CassandraRecordEmitter` deals with pojo.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1085585517


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   No. `CassandraSplits` are assigned to `CassandraSplitReader` (see `CassandraSplitEnumerator#assignUnprocessedSplitsToReader`). 
   Here we read all the records that belong to **the splits assigned to the reader**.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1085595610


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set

Review Comment:
   `CassandraSourceReader` is  a `SingleThreadMultiplexSourceReaderBase` so each fetcher thread has one instance of `CassandraSplitReader` owing to the doc. but when we run `SourceTestSuiteBase#testIdleReader` (4 splits with parallelism = 5) accessing `CassandraSplitReader#unprocessedSplits`raises a `ConcurrentModificationException` with one of the 5 fetcher threads.  Am I right to state that the number of splits in a given source should always be equal to the parallelism? And thus that each SplitReader should only read a single split ?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1405116976

   @zentol I'm almost done replacing the whole split mechanism. I could even remove the size estimates as Flink does not rely on expected split memory size as Beam. So I could use a very simple splitting mechanism based only on Cassandra min and max tokens per partitioner (min,x][x, max) for 2 splits for example. I also could consider that a Cassandra split processes only a single ring range and get rid of RingRange class. It'll change much. I also need to do a final pass before it is ready for another round. Otherwise you'll see sketchy things. don't waste your time reviewing now. I'll ping you to take a look


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087605427


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits, enumeratorContext.currentParallelism());
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitioner = clusterMetadata.getPartitioner();
+        final SplitsGenerator splitsGenerator = new SplitsGenerator(partitioner);
+        if (MURMUR3PARTITIONER.equals(partitioner)) {
+            LOG.info("Murmur3Partitioner detected, splitting");
+            List<BigInteger> tokens =
+                    clusterMetadata.getTokenRanges().stream()
+                            .map(
+                                    tokenRange ->
+                                            new BigInteger(
+                                                    tokenRange.getEnd().getValue().toString()))
+                            .collect(Collectors.toList());
+            return splitsGenerator.generateSplits(numberOfSplits, tokens);
+        } else {
+            // Murmur3Partitioner is the default and recommended partitioner for Cassandra 1.2+
+            // see
+            // https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerAbout.html
+            LOG.warn(
+                    "The current Cassandra partitioner is {}, only Murmur3Partitioner is supported "
+                            + "for splitting, using an single split",
+                    partitioner);
+            return splitsGenerator.generateSplits(1, Collections.emptyList());
+        }
+    }
+
+    @Override
+    public void addSplitsBack(List<CassandraSplit> splits, int subtaskId) {
+        LOG.info("Add {} splits back to CassandraSplitEnumerator.", splits.size());
+        state.addNewSplits(splits, enumeratorContext.currentParallelism());
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        LOG.info("Adding reader {} to CassandraSplitEnumerator.", subtaskId);
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    private void assignUnprocessedSplitsToReader(int readerId) {
+        checkReaderRegistered(readerId);
+
+        final Set<CassandraSplit> splitsForReader = state.getSplitsForReader(readerId);

Review Comment:
   Still, I find something strange. If I don't call `assignUnprocessedSplitsToReader()` in `addReader()` splits never get assigned because`handleSplitRequest()` was never called by the source framework.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089015655


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   https://issues.apache.org/jira/browse/FLINK-30802. Can you assign it to me ?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1332013091

   > I'm taking a look! Thanks for the ping +1
   
   @RyanSkraba any review comments so far ?


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1344284613

   > > Is there anything I can do to help the review ?
   > 
   > For better or worse, no :)
   > 
   > Holidays are coming up so I won't review it this year (scream) at the very least. If nothing happened til January (which I'd like to avoid!) I'll take a look myself to get things moving.
   
   Thanks @zentol 


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1368798535

   @zentol Ryan gave a LGTM can you take a look at this PR?


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100135736


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -48,6 +52,30 @@ public CassandraSplitState toSplitState() {
         return new CassandraSplitState(new HashSet<>(ringRanges), splitId());
     }
 
+    public void serialize(ObjectOutputStream objectOutputStream) throws IOException {

Review Comment:
   this method can be package private only if it resides in the serializer and it makes sense that it takes a CassandraSplit instance only if it resides in the serializer. So I'll move the method and make it static
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100131432


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -115,29 +109,22 @@ private List<CassandraSplit> discoverSplits() {
     @Override
     public void addSplitsBack(List<CassandraSplit> splits, int subtaskId) {
         LOG.info("Add {} splits back to CassandraSplitEnumerator.", splits.size());
-        state.addNewSplits(splits, enumeratorContext.currentParallelism());
-        assignUnprocessedSplitsToReader(subtaskId);
+        state.addNewSplits(splits);
     }
 
     @Override
     public void addReader(int subtaskId) {
         LOG.info("Adding reader {} to CassandraSplitEnumerator.", subtaskId);
-        assignUnprocessedSplitsToReader(subtaskId);
+        assignUnprocessedSplitToReader(subtaskId);
     }
 
-    private void assignUnprocessedSplitsToReader(int readerId) {
+    private void assignUnprocessedSplitToReader(int readerId) {
         checkReaderRegistered(readerId);
-
-        final Set<CassandraSplit> splitsForReader = state.getSplitsForReader(readerId);
-        if (splitsForReader != null && !splitsForReader.isEmpty()) {
-            Map<Integer, List<CassandraSplit>> assignment = new HashMap<>();
-            assignment.put(readerId, Lists.newArrayList(splitsForReader));
-            LOG.info("Assigning splits to reader {}", assignment);
-            enumeratorContext.assignSplits(new SplitsAssignment<>(assignment));
-        }
-
-        // periodically partition discovery is disabled, signal NoMoreSplitsEvent to the reader
-        LOG.debug(
+        final CassandraSplit cassandraSplit = state.getASplit();

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137359449


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high
+                LOG.info(
+                        "maxSplitMemorySize set value leads to {} splits with a task parallelism of {}. Creating as many splits as parallelism",
+                        numSplits,
+                        parallelism);
+                numSplits = parallelism;
+            }
+        } else { // not defined
+            LOG.info("maxSplitMemorySize not set. Creating as many splits as parallelism");
+            numSplits = parallelism;
+        }
+
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken = partitioner.minToken;
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            BigInteger endToken = startToken.add(increment);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken;
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+            startToken = endToken;
+        }
+        LOG.debug("Generated {} splits : {}", splits.size(), splits);
+        return splits;
+    }
+
+    /**
+     * Estimates the size of the table in bytes. Cassandra size estimates can be 0 if the data was
+     * just inserted and the amount of data in the table was small. This is very common situation
+     * during tests.
+     */
+    @VisibleForTesting
+    public long estimateTableSize() {
+        List<TokenRange> tokenRanges = getTokenRangesOfTable();
+        long size = 0L;
+        for (TokenRange tokenRange : tokenRanges) {
+            size += tokenRange.meanPartitionSize * tokenRange.partitionCount;
+        }
+        return Math.round(size / getRingFraction(tokenRanges));
+    }
+
+    /**
+     * The values that we get from system.size_estimates are for one node. We need to extrapolate to
+     * the whole cluster. This method estimates the percentage, the node represents in the cluster.
+     *
+     * @param tokenRanges The list of {@link TokenRange} to estimate
+     * @return The percentage the node represent in the whole cluster
+     */
+    private float getRingFraction(List<TokenRange> tokenRanges) {
+        BigInteger addressedTokens = BigInteger.ZERO;
+        for (TokenRange tokenRange : tokenRanges) {
+            addressedTokens =
+                    addressedTokens.add(distance(tokenRange.rangeStart, tokenRange.rangeEnd));
+        }
+        // it is < 1 because it is a percentage
+        return Float.valueOf(addressedTokens.divide(partitioner.ringSize).toString());

Review Comment:
   no, `ringSize` is a constant depending on the configured Cassandra cluster partitioner 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137150250


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);

Review Comment:
   :+1: . Your assumption is correct.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1465798685

   @zentol PTAL


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137346895


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high
+                LOG.info(
+                        "maxSplitMemorySize set value leads to {} splits with a task parallelism of {}. Creating as many splits as parallelism",
+                        numSplits,
+                        parallelism);
+                numSplits = parallelism;
+            }
+        } else { // not defined
+            LOG.info("maxSplitMemorySize not set. Creating as many splits as parallelism");
+            numSplits = parallelism;
+        }
+
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken = partitioner.minToken;
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            BigInteger endToken = startToken.add(increment);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken;
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+            startToken = endToken;
+        }
+        LOG.debug("Generated {} splits : {}", splits.size(), splits);
+        return splits;
+    }
+
+    /**
+     * Estimates the size of the table in bytes. Cassandra size estimates can be 0 if the data was
+     * just inserted and the amount of data in the table was small. This is very common situation
+     * during tests.
+     */
+    @VisibleForTesting
+    public long estimateTableSize() {
+        List<TokenRange> tokenRanges = getTokenRangesOfTable();
+        long size = 0L;
+        for (TokenRange tokenRange : tokenRanges) {
+            size += tokenRange.meanPartitionSize * tokenRange.partitionCount;
+        }
+        return Math.round(size / getRingFraction(tokenRanges));
+    }
+
+    /**
+     * The values that we get from system.size_estimates are for one node. We need to extrapolate to
+     * the whole cluster. This method estimates the percentage, the node represents in the cluster.
+     *
+     * @param tokenRanges The list of {@link TokenRange} to estimate
+     * @return The percentage the node represent in the whole cluster
+     */
+    private float getRingFraction(List<TokenRange> tokenRanges) {
+        BigInteger addressedTokens = BigInteger.ZERO;
+        for (TokenRange tokenRange : tokenRanges) {
+            addressedTokens =
+                    addressedTokens.add(distance(tokenRange.rangeStart, tokenRange.rangeEnd));
+        }
+        // it is < 1 because it is a percentage
+        return Float.valueOf(addressedTokens.divide(partitioner.ringSize).toString());

Review Comment:
   Can this cause a divide-by-zero if the size_estimates table was empty?
   (if tokenRanges is empty, then addressedTokens is 0, then the result is 0, then estimateTableSize divides by zero)



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1136881612


##########
flink-connector-cassandra/pom.xml:
##########
@@ -180,7 +187,7 @@ under the License.
 			<scope>provided</scope>
 		</dependency>
 
-		<!-- Test dependencies -->
+	<!-- Test dependencies -->

Review Comment:
   ```suggestion
   		<!-- Test dependencies -->
   ```



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -80,27 +85,55 @@
 public class CassandraSource<OUT>
         implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
 
-    public static final String CQL_PROHIBITTED_CLAUSES_REGEXP =
+    public static final String CQL_PROHIBITED_CLAUSES_REGEXP =
             "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
     private static final long serialVersionUID = 1L;
 
     private final ClusterBuilder clusterBuilder;
+    @Nullable private final Long maxSplitMemorySize;
     private final Class<OUT> pojoClass;
     private final String query;
+    private final String keyspace;
+    private final String table;
     private final MapperOptions mapperOptions;
 
     public CassandraSource(
             ClusterBuilder clusterBuilder,
             Class<OUT> pojoClass,
             String query,
             MapperOptions mapperOptions) {
+        this(clusterBuilder, null, pojoClass, query, mapperOptions);
+    }
+
+    public CassandraSource(
+            ClusterBuilder clusterBuilder,
+            Long maxSplitMemorySize,
+            Class<OUT> pojoClass,
+            String query,
+            MapperOptions mapperOptions) {
         checkNotNull(clusterBuilder, "ClusterBuilder required but not provided");
+        checkState(
+                maxSplitMemorySize == null || maxSplitMemorySize > 0,
+                "Max split size in bytes provided but set to an invalid value {}",
+                maxSplitMemorySize);
         checkNotNull(pojoClass, "POJO class required but not provided");
-        checkQueryValidity(query);

Review Comment:
   I feel like we could've kept this method if we'd have it return a `Matcher`.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializer.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+/** Serializer for {@link CassandraEnumeratorState}. */
+public class CassandraEnumeratorStateSerializer

Review Comment:
   Add private constructor to enforce singleton usage?



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.net.InetSocketAddress;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.

Review Comment:
   ```suggestion
       // flushing mem table to SS tables is an asynchronous operation that may take a while
       private static final long FLUSH_MEMTABLES_DELAY = 30_000L; 
   ```



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(

Review Comment:
   ```suggestion
       public void testGenerateSplitsWithTooLargeSize(
   ```



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorStateSerializer;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraSplitEnumerator;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSourceReaderFactory;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import javax.annotation.Nullable;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A bounded source to read from Cassandra and return a collection of entities as {@code
+ * DataStream<Entity>}. An entity is built by Cassandra mapper ({@code
+ * com.datastax.driver.mapping.EntityMapper}) based on a POJO containing annotations (as described
+ * in <a
+ * href="https://docs.datastax.com/en/developer/java-driver/3.11/manual/object_mapper/creating/">
+ * Cassandra object mapper</a>).
+ *
+ * <p>To use it, do the following:
+ *
+ * <pre>{@code
+ * ClusterBuilder clusterBuilder = new ClusterBuilder() {
+ *   @Override
+ *   protected Cluster buildCluster(Cluster.Builder builder) {
+ *     return builder.addContactPointsWithPorts(new InetSocketAddress(HOST,PORT))
+ *                   .withQueryOptions(new QueryOptions().setConsistencyLevel(CL))
+ *                   .withSocketOptions(new SocketOptions()
+ *                   .setConnectTimeoutMillis(CONNECT_TIMEOUT)
+ *                   .setReadTimeoutMillis(READ_TIMEOUT))
+ *                   .build();
+ *   }
+ * };
+ * long maxSplitMemorySize = ... //optional max split size in bytes. If not set, maxSplitMemorySize = tableSize / parallelism
+ * Source cassandraSource = new CassandraSource(clusterBuilder,
+ *                                              maxSplitMemorySize,
+ *                                              Pojo.class,
+ *                                              "select ... from KEYSPACE.TABLE ...;",
+ *                                              () -> new Mapper.Option[] {Mapper.Option.saveNullFields(true)});
+ *
+ * DataStream<Pojo> stream = env.fromSource(cassandraSource, WatermarkStrategy.noWatermarks(),
+ * "CassandraSource");
+ * }</pre>
+ */
+@PublicEvolving
+public class CassandraSource<OUT>
+        implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
+
+    public static final String CQL_PROHIBITED_CLAUSES_REGEXP =
+            "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";

Review Comment:
   nit: Both of these could be a `Pattern`.



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);

Review Comment:
   ```suggestion
           long tableSize = generator.estimateTableSize();
           assertThat(tableSize).isEqualTo(35840L);
           List<CassandraSplit> splits = generator.generateSplits();
           assertThat(splits.size()).isEqualTo(tableSize / maxSplitMemorySize);
   ```



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.CassandraSource;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** tests for query generation and query sanity checks. */
+class CassandraQueryTest {
+
+    private static final Pattern SELECT_PATTERN = Pattern.compile(CassandraSource.SELECT_REGEXP);
+
+    @Test
+    public void testKeySpaceTableExtractionRegexp() {
+        Arrays.asList(
+                        "select field FROM keyspace.table where field = value;",
+                        "select * FROM keyspace.table;",
+                        "select field1, field2 from keyspace.table;",
+                        "select field1, field2 from keyspace.table LIMIT(1000);",
+                        "select field1 from keyspace.table ;",
+                        "select field1 from keyspace.table where field1=1;")
+                .forEach(CassandraQueryTest::assertQueryFormatCorrect);
+
+        Arrays.asList(
+                        "select field1 from table;", // missing keyspace
+                        "select field1 from keyspace.table" // missing ";"
+                        )
+                .forEach(CassandraQueryTest::assertQueryFormatIncorrect);
+    }
+
+    @Test
+    public void testProhibitedClauses() {
+        Arrays.asList(
+                        "SELECT COUNT(*) from flink.table;",
+                        "SELECT AVG(*) from flink.table;",
+                        "SELECT MIN(*) from flink.table;",
+                        "SELECT MAX(*) from flink.table;",
+                        "SELECT SUM(*) from flink.table;",
+                        "SELECT field1, field2 from flink.table ORDER BY field1;",
+                        "SELECT field1, field2 from flink.table GROUP BY field1;")
+                .forEach(CassandraQueryTest::assertProhibitedClauseRejected);
+    }
+
+    @Test
+    public void testGenerateRangeQuery() {
+        String query;
+        String outputQuery;
+
+        // query with where clause
+        query = "SELECT field FROM keyspace.table WHERE field = value;";
+        outputQuery = CassandraSplitReader.generateRangeQuery(query, "field");
+        assertThat(outputQuery)
+                .isEqualTo(
+                        "SELECT field FROM keyspace.table WHERE (token(field) >= ?) AND (token(field) < ?) AND field = value;");
+
+        // query without where clause
+        query = "SELECT * FROM keyspace.table;";
+        outputQuery = CassandraSplitReader.generateRangeQuery(query, "field");
+        assertThat(outputQuery)
+                .isEqualTo(
+                        "SELECT * FROM keyspace.table WHERE (token(field) >= ?) AND (token(field) < ?);");
+
+        // query without where clause but with another trailing clause

Review Comment:
   also test WHERE + LIMIT?



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 20;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        100_000_000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // tableSize / maxSplitMemorySize is too little compared to parallelism falling back to
+        // number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too small split size set")
+    public void testGenerateSplitsWithTooSmallSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        1L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // tableSize / maxSplitMemorySize is too big compared to parallelism falling back to

Review Comment:
   Reminder for me to check how this is determined.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraRecordEmitter.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.api.connector.source.SourceOutput;
+import org.apache.flink.connector.base.source.reader.RecordEmitter;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.ExecutionInfo;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * {@link RecordEmitter} that converts the {@link CassandraRow} read by the {@link
+ * CassandraSplitReader} to specified POJO and output it. This class uses the Cassandra driver
+ * mapper to map the row to the POJO.
+ *
+ * @param <OUT> type of POJO record to output
+ */
+class CassandraRecordEmitter<OUT> implements RecordEmitter<CassandraRow, OUT, CassandraSplit> {
+
+    private final Function<ResultSet, OUT> map;
+
+    public CassandraRecordEmitter(Function<ResultSet, OUT> map) {
+        this.map = map;
+    }
+
+    @Override
+    public void emitRecord(
+            CassandraRow cassandraRow, SourceOutput<OUT> output, CassandraSplit cassandraSplit) {
+        final Row row = cassandraRow.getRow();
+        // Mapping from a row to a Class<OUT> is a complex operation involving reflection API.
+        // It is better to use Cassandra mapper for it.
+        // but the mapper takes only a resultSet as input hence forging one containing only the row
+        ResultSet resultSet =
+                new ResultSet() {

Review Comment:
   Create a dedicated class that we pass the row into as a constructor argument. Saves us from defining a new class for each row.



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);

Review Comment:
   add a comment or error message that this is a sanity check for data being flushed to SSTs



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high

Review Comment:
   Not sure how you arrived at this particular ratio; 10 splits per subtasks seems perfectly fine to me. Overall it seems a bit...arbitrary?
   
   I don't think we should limit this ratio; so long as things work we're good, no?
   If someone wants to process a 1GB table in 1 MB chunks with 1 subtask they should be free to do so.
   We can solve the # number of in-memory splits by generating them lazily instead.
   
   I'd be inclined to drop this ratio stuff and enforce a minimum size (a few MB I guess) instead to avoid some stupid cases (like 1).
   
   As-is this can just blow-up suddenly in surprising ways. For example, a periodic batch job against a growing table may suddenly crash with a OOM because they now hit this ratio and get massive splits instead.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high
+                LOG.info(
+                        "maxSplitMemorySize set value leads to {} splits with a task parallelism of {}. Creating as many splits as parallelism",
+                        numSplits,
+                        parallelism);
+                numSplits = parallelism;
+            }
+        } else { // not defined
+            LOG.info("maxSplitMemorySize not set. Creating as many splits as parallelism");

Review Comment:
   ```suggestion
               LOG.info("maxSplitMemorySize not set. Creating as many splits as parallelism ({})", parallelism);
   ```



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low

Review Comment:
   ```suggestion
                       || numSplits < parallelism // too low
   ```
   I don't understand why there should be a ratio here.
   If `numSplits` is 10, then why should we treat p=90 and p=100 differently?



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(

Review Comment:
   The test name is a bit ambiguous in that it's not clear whether it refers to the size of:
   * the split
   * the table
   * the max split size
   
   Maybe name it `testGenerateSplitsWithTableSizeLowerThanMaximumSplitSize`



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);

Review Comment:
   I guess we can't really verify the actual size of a split (without reading it)?



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.CassandraSource;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** tests for query generation and query sanity checks. */
+class CassandraQueryTest {
+
+    private static final Pattern SELECT_PATTERN = Pattern.compile(CassandraSource.SELECT_REGEXP);
+
+    @Test
+    public void testKeySpaceTableExtractionRegexp() {
+        Arrays.asList(
+                        "select field FROM keyspace.table where field = value;",
+                        "select * FROM keyspace.table;",
+                        "select field1, field2 from keyspace.table;",
+                        "select field1, field2 from keyspace.table LIMIT(1000);",
+                        "select field1 from keyspace.table ;",
+                        "select field1 from keyspace.table where field1=1;")
+                .forEach(CassandraQueryTest::assertQueryFormatCorrect);
+
+        Arrays.asList(
+                        "select field1 from table;", // missing keyspace
+                        "select field1 from keyspace.table" // missing ";"
+                        )

Review Comment:
   ```suggestion
                           "select field1 from table;", // missing keyspace
                           "select field1 from .table" // missing keyspace
                           "select field1 from keyspace;", // missing table
                           "select field1 from keyspace.;", // missing table
                           "select field1 from keyspace.table" // missing ";"
                           )
   ```
   Some more cases we could consider. In particular the `.table` variants could be interesting because they'd happen when users use something like `${keyspace}.${table}` in a script but one of the variables is empty/undefined.



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 20;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        100_000_000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // tableSize / maxSplitMemorySize is too little compared to parallelism falling back to
+        // number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too small split size set")
+    public void testGenerateSplitsWithTooSmallSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        1L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // tableSize / maxSplitMemorySize is too big compared to parallelism falling back to
+        // number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+    }
+
+    // overridden to use unordered checks
+    @Override
+    protected void checkResultWithSemantic(
+            CloseableIterator<Pojo> resultIterator,
+            List<List<Pojo>> testData,
+            CheckpointingMode semantic,
+            Integer limit) {
+        if (limit != null) {
+            Runnable runnable =
+                    () ->
+                            CollectIteratorAssertions.assertUnordered(resultIterator)
+                                    .withNumRecordsLimit(limit)
+                                    .matchesRecordsFromSource(testData, semantic);
+
+            assertThat(runAsync(runnable)).succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT);

Review Comment:
   Why is this using runAsync here but not in the else branch?



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 20;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        100_000_000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // tableSize / maxSplitMemorySize is too little compared to parallelism falling back to
+        // number of splits = parallelism

Review Comment:
   Wondering if this shouldn't result in a single split instead :thinking: 



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.net.InetSocketAddress;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+
+    private Cluster cluster;
+    private Session session;
+    private ClusterBuilder clusterBuilder;
+
+    public CassandraTestEnvironment() {
+        cassandraContainer = new CassandraContainer(DOCKER_CASSANDRA_IMAGE);
+        // more generous timeouts
+        addJavaOpts(
+                cassandraContainer,
+                "-Dcassandra.request_timeout_in_ms=30000",
+                "-Dcassandra.read_request_timeout_in_ms=15000",
+                "-Dcassandra.write_request_timeout_in_ms=6000");
+    }
+
+    @Override
+    public void startUp() throws Exception {
+        startEnv();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        stopEnv();
+    }
+
+    private static void addJavaOpts(GenericContainer<?> container, String... opts) {
+        String jvmOpts = container.getEnvMap().getOrDefault("JVM_OPTS", "");
+        container.withEnv("JVM_OPTS", jvmOpts + " " + StringUtils.join(opts, " "));
+    }
+
+    private void startEnv() throws Exception {
+        // configure container start to wait until cassandra is ready to receive queries
+        cassandraContainer.waitingFor(new CassandraQueryWaitStrategy());
+        // start with retrials
+        cassandraContainer.start();
+        cassandraContainer.followOutput(
+                new Slf4jLogConsumer(LOG),
+                OutputFrame.OutputType.END,
+                OutputFrame.OutputType.STDERR,
+                OutputFrame.OutputType.STDOUT);
+
+        cluster = cassandraContainer.getCluster();
+        clusterBuilder =
+                createBuilderWithConsistencyLevel(
+                        ConsistencyLevel.ONE,
+                        cassandraContainer.getHost(),
+                        cassandraContainer.getMappedPort(CQL_PORT));
+
+        session = cluster.connect();
+        session.execute(requestWithTimeout(CREATE_KEYSPACE_QUERY));
+        // create a dedicated table for split size tests (to avoid having to flush with each test)
+        insertTestDataForSplitSizeTests();

Review Comment:
   Note: If we ever use this environment in multiple tests we'd want this to be an opt-in thing so we don't wait unnecessarily for a table that we don't even need.



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.net.InetSocketAddress;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+
+    private Cluster cluster;
+    private Session session;
+    private ClusterBuilder clusterBuilder;
+
+    public CassandraTestEnvironment() {
+        cassandraContainer = new CassandraContainer(DOCKER_CASSANDRA_IMAGE);
+        // more generous timeouts
+        addJavaOpts(
+                cassandraContainer,
+                "-Dcassandra.request_timeout_in_ms=30000",
+                "-Dcassandra.read_request_timeout_in_ms=15000",
+                "-Dcassandra.write_request_timeout_in_ms=6000");
+    }
+
+    @Override
+    public void startUp() throws Exception {
+        startEnv();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        stopEnv();
+    }
+
+    private static void addJavaOpts(GenericContainer<?> container, String... opts) {
+        String jvmOpts = container.getEnvMap().getOrDefault("JVM_OPTS", "");
+        container.withEnv("JVM_OPTS", jvmOpts + " " + StringUtils.join(opts, " "));
+    }
+
+    private void startEnv() throws Exception {
+        // configure container start to wait until cassandra is ready to receive queries
+        cassandraContainer.waitingFor(new CassandraQueryWaitStrategy());
+        // start with retrials
+        cassandraContainer.start();
+        cassandraContainer.followOutput(
+                new Slf4jLogConsumer(LOG),
+                OutputFrame.OutputType.END,
+                OutputFrame.OutputType.STDERR,
+                OutputFrame.OutputType.STDOUT);
+
+        cluster = cassandraContainer.getCluster();
+        clusterBuilder =
+                createBuilderWithConsistencyLevel(
+                        ConsistencyLevel.ONE,
+                        cassandraContainer.getHost(),
+                        cassandraContainer.getMappedPort(CQL_PORT));
+
+        session = cluster.connect();
+        session.execute(requestWithTimeout(CREATE_KEYSPACE_QUERY));
+        // create a dedicated table for split size tests (to avoid having to flush with each test)
+        insertTestDataForSplitSizeTests();
+    }
+
+    private void insertTestDataForSplitSizeTests() throws Exception {
+        session.execute(requestWithTimeout(CREATE_SPLITS_TABLE_QUERY));
+        for (int i = 0; i < NB_SPLITS_RECORDS; i++) {
+            session.execute(requestWithTimeout(String.format(INSERT_INTO_FLINK_SPLITS, i, i)));
+        }
+        flushMemTables(SPLITS_TABLE);
+    }
+
+    private void stopEnv() {
+
+        if (session != null) {
+            session.close();
+        }
+        if (cluster != null) {
+            cluster.close();
+        }
+        cassandraContainer.stop();
+    }
+
+    private ClusterBuilder createBuilderWithConsistencyLevel(
+            ConsistencyLevel consistencyLevel, String host, int port) {
+        return new ClusterBuilder() {
+            @Override
+            protected Cluster buildCluster(Cluster.Builder builder) {
+                return builder.addContactPointsWithPorts(new InetSocketAddress(host, port))
+                        .withQueryOptions(
+                                new QueryOptions()
+                                        .setConsistencyLevel(consistencyLevel)
+                                        .setSerialConsistencyLevel(ConsistencyLevel.LOCAL_SERIAL))
+                        .withSocketOptions(
+                                new SocketOptions()
+                                        // default timeout x 3
+                                        .setConnectTimeoutMillis(15000)
+                                        // default timeout x3 and higher than
+                                        // request_timeout_in_ms at the cluster level
+                                        .setReadTimeoutMillis(READ_TIMEOUT_MILLIS))
+                        .withoutJMXReporting()
+                        .withoutMetrics()
+                        .build();
+            }
+        };
+    }
+
+    /**
+     * Force the flush of cassandra memTables to SSTables in order to update size_estimates. It is
+     * needed for the tests because we just inserted records, we need to force cassandra to update
+     * size_estimates system table.
+     */
+    void flushMemTables(String table) throws Exception {
+        cassandraContainer.execInContainer("nodetool", "flush", KEYSPACE, table);

Review Comment:
   what's the difference between `flush`and https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/tools/toolsRefreshSizeEstimates.html?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        checkReaderRegistered(subtaskId);
+        final CassandraSplit cassandraSplit = state.getASplit();
+        if (cassandraSplit != null) {
+            LOG.info("Assigning splits to reader {}", subtaskId);
+            enumeratorContext.assignSplit(cassandraSplit, subtaskId);
+        } else {
+            LOG.info(
+                    "No split assigned to reader {} because the enumerator has no unassigned split left",
+                    subtaskId);
+        }
+        if (!state.hasMoreSplits()) {
+            LOG.info(
+                    "No more CassandraSplits to assign. Sending NoMoreSplitsEvent to reader {}.",
+                    subtaskId);
+            enumeratorContext.signalNoMoreSplits(subtaskId);
+        }
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits);
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitionerName = clusterMetadata.getPartitioner();
+        final SplitsGenerator.CassandraPartitioner partitioner =
+                partitionerName.contains(MURMUR3PARTITIONER.className())
+                        ? MURMUR3PARTITIONER
+                        : RANDOMPARTITIONER;
+        return new SplitsGenerator(partitioner).generateSplits(numberOfSplits);

Review Comment:
   Looking at SplitsGenerator#generateaSplits it should be straight-forward to lazily generate splits, no?
   The start-/endToken, increment and numSplits would be your state.
   
   This seems safer than to add safeguards for split counts, which'd be difficult to asses imo as to how effective/dangerous they are.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1474052060

   @zentol I pushed for security reasons, I have only the addSplitsBack case to deal with in lazy splits generation scenario. You can start to take a look at the other things if you have time or just wait for this part to be finished.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101575295


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorState.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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Objects;
+import java.util.Queue;
+
+/** Sate for {@link CassandraSplitEnumerator} to track the splits yet to assign. */

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1104371424


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSourceReaderFactory.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.mapping.Mapper;
+import com.datastax.driver.mapping.MappingManager;
+
+/**
+ * Factory to create {@link CassandraSourceReader}s and allow creating the cluster and the session
+ * objects.

Review Comment:
   ```suggestion
    * Factory to create {@link CassandraSourceReader}s and allow the sharing of cluster and the session
    * objects.
   ```



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+public class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {

Review Comment:
   ```suggestion
   class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
   ```



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -31,84 +32,35 @@
 /** tests for query generation and query sanity checks. */
 class CassandraQueryTest {
 
+    private static final Pattern PATTERN = Pattern.compile(CassandraSplitReader.SELECT_REGEXP);
+
     @Test
     public void testKeySpaceTableExtractionRegexp() {
-        final Pattern pattern = Pattern.compile(CassandraSplitReader.SELECT_REGEXP);
-        Matcher matcher;
-        matcher = pattern.matcher("SELECT field FROM keyspace.table where field = value;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
-
-        matcher = pattern.matcher("SELECT * FROM keyspace.table;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
 
-        matcher = pattern.matcher("select field1, field2 from keyspace.table;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
+        assertQueryFormatCorrect("SELECT field FROM keyspace.table where field = value;");
+        assertQueryFormatCorrect("SELECT * FROM keyspace.table;");
+        assertQueryFormatCorrect("select field1, field2 from keyspace.table;");
+        assertQueryFormatCorrect("select field1, field2 from keyspace.table LIMIT(1000);");
+        assertQueryFormatCorrect("select field1 from keyspace.table ;");
+        assertQueryFormatCorrect("select field1 from keyspace.table where field1=1;");
 
-        matcher = pattern.matcher("select field1, field2 from keyspace.table LIMIT(1000);");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
-
-        matcher = pattern.matcher("select field1 from keyspace.table ;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
-
-        matcher = pattern.matcher("select field1 from keyspace.table where field1=1;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
-
-        matcher = pattern.matcher("select field1 from table;"); // missing keyspace
-        assertThat(matcher.matches()).isFalse();
-
-        matcher = pattern.matcher("select field1 from keyspace.table"); // missing ";"
-        assertThat(matcher.matches()).isFalse();
+        assertQueryFormatIncorrect("select field1 from table;"); // missing keyspace
+        assertQueryFormatIncorrect("select field1 from keyspace.table"); // missing ";"
     }
 
     @Test
     public void testProhibitedClauses() {
-        assertThatThrownBy(
-                        () ->
-                                CassandraSource.checkQueryValidity(
-                                        "SELECT COUNT(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () -> CassandraSource.checkQueryValidity("SELECT AVG(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-
-        assertThatThrownBy(
-                        () -> CassandraSource.checkQueryValidity("SELECT MIN(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () -> CassandraSource.checkQueryValidity("SELECT MAX(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () -> CassandraSource.checkQueryValidity("SELECT SUM(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () ->
-                                CassandraSource.checkQueryValidity(
-                                        "SELECT field1, field2 from flink.table ORDER BY field1;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () ->
-                                CassandraSource.checkQueryValidity(
-                                        "SELECT field1, field2 from flink.table GROUP BY field1;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
+        Arrays.stream(
+                        new String[] {

Review Comment:
   `Arrays.asList()` would allow you to avoid `new String[] {}`



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializer.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.core.memory.DataInputDeserializer;
+import org.apache.flink.core.memory.DataOutputSerializer;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+/** Serializer for {@link CassandraEnumeratorState}. */
+public class CassandraEnumeratorStateSerializer
+        implements SimpleVersionedSerializer<CassandraEnumeratorState> {
+
+    public static final CassandraEnumeratorStateSerializer INSTANCE =
+            new CassandraEnumeratorStateSerializer();
+    private static final ThreadLocal<DataOutputSerializer> SERIALIZER_CACHE =
+            ThreadLocal.withInitial(() -> new DataOutputSerializer(64));

Review Comment:
   This is overkill for the enumerator state.
   In contrast to splits (of which there can be thousands...[hmm]()) there's only 1 state at all times.
   
   Fun fact about thread locals: They can easily leak the user-code classloader. If any thread that out-lives the task (like say, a shared thread pool) ever accesses the ThreadLocal, then the thread retains a reference to the thread local, due to which it can't be GC'd. Just thought I'd throw this out there.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   > I'd prefer that the user conf parameter is in bytes.
   
   I agree. (well it should be a `MemorySize`, but the same idea)
   
   > I can get my size estimates from Beam 2017 code so that the SplitGenerator provides splits of a given size. Then all the records of each split will be stored in memory to create the needed RecordsBySplits.
   
   Sounds good. We do something similar in the [mongodb connector](https://github.com/apache/flink-connector-mongodb/blob/main/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.java).



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorState.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Objects;
+import java.util.Queue;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** State for {@link CassandraSplitEnumerator} to track the splits yet to assign. */
+public class CassandraEnumeratorState {
+
+    private final Queue<CassandraSplit> unassignedSplits;
+
+    CassandraEnumeratorState() {
+        this.unassignedSplits = new ArrayDeque<>();
+    }
+
+    CassandraEnumeratorState(Queue<CassandraSplit> unassignedSplits) {
+        checkNotNull(unassignedSplits);
+        this.unassignedSplits = unassignedSplits;
+    }
+
+    public Queue<CassandraSplit> getUnassignedSplits() {
+        return unassignedSplits;
+    }
+
+    public void addNewSplits(Collection<CassandraSplit> newSplits) {
+        unassignedSplits.addAll(newSplits);
+    }
+
+    public @Nullable CassandraSplit getASplit() {
+        return unassignedSplits.poll();
+    }
+
+    boolean hasMoreSplits() {
+        return unassignedSplits.size() != 0;

Review Comment:
   ```suggestion
           return !unassignedSplits.isEmpty();
   ```



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        checkReaderRegistered(subtaskId);
+        final CassandraSplit cassandraSplit = state.getASplit();
+        if (cassandraSplit != null) {
+            LOG.info("Assigning splits to reader {}", subtaskId);
+            enumeratorContext.assignSplit(cassandraSplit, subtaskId);
+        } else {
+            LOG.info(
+                    "No split assigned to reader {} because the enumerator has no unassigned split left",
+                    subtaskId);
+        }
+        if (!state.hasMoreSplits()) {
+            LOG.info(
+                    "No more CassandraSplits to assign. Sending NoMoreSplitsEvent to reader {}.",
+                    subtaskId);
+            enumeratorContext.signalNoMoreSplits(subtaskId);
+        }
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits);
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitionerName = clusterMetadata.getPartitioner();
+        final SplitsGenerator.CassandraPartitioner partitioner =
+                partitionerName.contains(MURMUR3PARTITIONER.className())
+                        ? MURMUR3PARTITIONER
+                        : RANDOMPARTITIONER;
+        return new SplitsGenerator(partitioner).generateSplits(numberOfSplits);

Review Comment:
   I'm wondering if splits could be generated lazily. Slightly worried about us eagerly creating millions of splits when a user configures a small split size.



##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -31,84 +32,35 @@
 /** tests for query generation and query sanity checks. */
 class CassandraQueryTest {
 
+    private static final Pattern PATTERN = Pattern.compile(CassandraSplitReader.SELECT_REGEXP);
+
     @Test
     public void testKeySpaceTableExtractionRegexp() {
-        final Pattern pattern = Pattern.compile(CassandraSplitReader.SELECT_REGEXP);
-        Matcher matcher;
-        matcher = pattern.matcher("SELECT field FROM keyspace.table where field = value;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
-
-        matcher = pattern.matcher("SELECT * FROM keyspace.table;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
 
-        matcher = pattern.matcher("select field1, field2 from keyspace.table;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
+        assertQueryFormatCorrect("SELECT field FROM keyspace.table where field = value;");
+        assertQueryFormatCorrect("SELECT * FROM keyspace.table;");
+        assertQueryFormatCorrect("select field1, field2 from keyspace.table;");
+        assertQueryFormatCorrect("select field1, field2 from keyspace.table LIMIT(1000);");
+        assertQueryFormatCorrect("select field1 from keyspace.table ;");
+        assertQueryFormatCorrect("select field1 from keyspace.table where field1=1;");
 
-        matcher = pattern.matcher("select field1, field2 from keyspace.table LIMIT(1000);");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
-
-        matcher = pattern.matcher("select field1 from keyspace.table ;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
-
-        matcher = pattern.matcher("select field1 from keyspace.table where field1=1;");
-        assertThat(matcher.matches()).isTrue();
-        assertThat(matcher.group(1)).isEqualTo("keyspace");
-        assertThat(matcher.group(2)).isEqualTo("table");
-
-        matcher = pattern.matcher("select field1 from table;"); // missing keyspace
-        assertThat(matcher.matches()).isFalse();
-
-        matcher = pattern.matcher("select field1 from keyspace.table"); // missing ";"
-        assertThat(matcher.matches()).isFalse();
+        assertQueryFormatIncorrect("select field1 from table;"); // missing keyspace
+        assertQueryFormatIncorrect("select field1 from keyspace.table"); // missing ";"
     }
 
     @Test
     public void testProhibitedClauses() {
-        assertThatThrownBy(
-                        () ->
-                                CassandraSource.checkQueryValidity(
-                                        "SELECT COUNT(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () -> CassandraSource.checkQueryValidity("SELECT AVG(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-
-        assertThatThrownBy(
-                        () -> CassandraSource.checkQueryValidity("SELECT MIN(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () -> CassandraSource.checkQueryValidity("SELECT MAX(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () -> CassandraSource.checkQueryValidity("SELECT SUM(*) from flink.table;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () ->
-                                CassandraSource.checkQueryValidity(
-                                        "SELECT field1, field2 from flink.table ORDER BY field1;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
-        assertThatThrownBy(
-                        () ->
-                                CassandraSource.checkQueryValidity(
-                                        "SELECT field1, field2 from flink.table GROUP BY field1;"))
-                .isInstanceOf(IllegalStateException.class)
-                .hasMessageContaining("Aggregations/OrderBy are not supported");
+        Arrays.stream(
+                        new String[] {

Review Comment:
   Why is this one using `forEach` unlike `testKeySpaceTableExtractionRegexp`?



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -57,7 +57,22 @@ public CassandraSplitEnumerator(
 
     @Override
     public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
-        assignUnprocessedSplitToReader(subtaskId);
+        checkReaderRegistered(subtaskId);
+        final CassandraSplit cassandraSplit = state.getASplit();
+        if (cassandraSplit != null) {
+            LOG.info("Assigning splits to reader {}", subtaskId);
+            enumeratorContext.assignSplit(cassandraSplit, subtaskId);
+        } else {
+            LOG.info(
+                    "No split assigned to reader {} because the enumerator has no unassigned split left",
+                    subtaskId);
+        }
+        if (!state.hasMoreSplits()) {
+            LOG.info(
+                    "No more CassandraSplits to assign. Sending NoMoreSplitsEvent to reader {}.",
+                    subtaskId);
+            enumeratorContext.signalNoMoreSplits(subtaskId);
+        }

Review Comment:
   Isn't the else branch and this branch always executed together?
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1142134134


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializer.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.math.BigInteger;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+/** Serializer for {@link CassandraEnumeratorState}. */
+public class CassandraEnumeratorStateSerializer
+        implements SimpleVersionedSerializer<CassandraEnumeratorState> {
+
+    public static final CassandraEnumeratorStateSerializer INSTANCE =
+            new CassandraEnumeratorStateSerializer();
+    public static final int CURRENT_VERSION = 0;
+
+    private CassandraEnumeratorStateSerializer() {}
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(CassandraEnumeratorState cassandraEnumeratorState) throws IOException {
+        try (final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                final ObjectOutputStream objectOutputStream =
+                        new ObjectOutputStream(byteArrayOutputStream)) {
+            final Queue<CassandraSplit> splitsToReassign =
+                    cassandraEnumeratorState.getSplitsToReassign();
+            objectOutputStream.writeInt(splitsToReassign.size());
+            for (CassandraSplit cassandraSplit : splitsToReassign) {
+                final byte[] serializedSplit =
+                        CassandraSplitSerializer.INSTANCE.serialize(cassandraSplit);
+                objectOutputStream.writeInt(serializedSplit.length);
+                objectOutputStream.write(serializedSplit);
+            }
+
+            objectOutputStream.writeLong(cassandraEnumeratorState.getNumSplitsLeftToGenerate());
+
+            final byte[] increment = cassandraEnumeratorState.getIncrement().toByteArray();
+            objectOutputStream.writeInt(increment.length);
+            objectOutputStream.write(increment);
+
+            final byte[] startToken = cassandraEnumeratorState.getStartToken().toByteArray();
+            objectOutputStream.writeInt(startToken.length);
+            objectOutputStream.write(startToken);
+
+            final byte[] maxToken = cassandraEnumeratorState.getMaxToken().toByteArray();
+            objectOutputStream.writeInt(maxToken.length);
+            objectOutputStream.write(maxToken);
+
+            objectOutputStream.flush();
+            return byteArrayOutputStream.toByteArray();
+        }
+    }
+
+    @Override
+    public CassandraEnumeratorState deserialize(int version, byte[] serialized) throws IOException {
+        try (final ByteArrayInputStream byteArrayInputStream =
+                        new ByteArrayInputStream(serialized);
+                final ObjectInputStream objectInputStream =
+                        new ObjectInputStream(byteArrayInputStream)) {
+            final Queue<CassandraSplit> splitsToReassign = new ArrayDeque<>();
+            final int splitsToReassignSize = objectInputStream.readInt();
+            for (int i = 0; i < splitsToReassignSize; i++) {
+                final int splitSize = objectInputStream.readInt();
+                final byte[] splitBytes = new byte[splitSize];
+                if (objectInputStream.read(splitBytes) == -1) {
+                    throw new IOException(
+                            "EOF received while deserializing CassandraEnumeratorState.splitsToReassign");
+                }
+                final CassandraSplit split =
+                        CassandraSplitSerializer.INSTANCE.deserialize(
+                                CassandraSplitSerializer.CURRENT_VERSION, splitBytes);
+                splitsToReassign.add(split);
+            }
+
+            final long numSplitsLeftToGenerate = objectInputStream.readLong();
+
+            final int incrementSize = objectInputStream.readInt();
+            final byte[] incrementBytes = new byte[incrementSize];
+            if (objectInputStream.read(incrementBytes) == -1) {
+                throw new IOException(
+                        "EOF received while deserializing cassandraEnumeratorState.increment");
+            }
+            final BigInteger increment = new BigInteger(incrementBytes);
+
+            final int startTokenSize = objectInputStream.readInt();
+            final byte[] startTokenBytes = new byte[startTokenSize];
+            if (objectInputStream.read(startTokenBytes) == -1) {
+                throw new IOException(
+                        "EOF received while deserializing cassandraEnumeratorState.startToken");
+            }
+            final BigInteger startToken = new BigInteger(startTokenBytes);
+
+            final int maxTokenSize = objectInputStream.readInt();
+            final byte[] maxTokenBytes = new byte[maxTokenSize];
+            if (objectInputStream.read(maxTokenBytes) == -1) {
+                throw new IOException(
+                        "EOF received while deserializing cassandraEnumeratorState.maxToken");
+            }
+            final BigInteger maxToken = new BigInteger(maxTokenBytes);

Review Comment:
   :+1
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1142134134


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializer.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.math.BigInteger;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+/** Serializer for {@link CassandraEnumeratorState}. */
+public class CassandraEnumeratorStateSerializer
+        implements SimpleVersionedSerializer<CassandraEnumeratorState> {
+
+    public static final CassandraEnumeratorStateSerializer INSTANCE =
+            new CassandraEnumeratorStateSerializer();
+    public static final int CURRENT_VERSION = 0;
+
+    private CassandraEnumeratorStateSerializer() {}
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(CassandraEnumeratorState cassandraEnumeratorState) throws IOException {
+        try (final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                final ObjectOutputStream objectOutputStream =
+                        new ObjectOutputStream(byteArrayOutputStream)) {
+            final Queue<CassandraSplit> splitsToReassign =
+                    cassandraEnumeratorState.getSplitsToReassign();
+            objectOutputStream.writeInt(splitsToReassign.size());
+            for (CassandraSplit cassandraSplit : splitsToReassign) {
+                final byte[] serializedSplit =
+                        CassandraSplitSerializer.INSTANCE.serialize(cassandraSplit);
+                objectOutputStream.writeInt(serializedSplit.length);
+                objectOutputStream.write(serializedSplit);
+            }
+
+            objectOutputStream.writeLong(cassandraEnumeratorState.getNumSplitsLeftToGenerate());
+
+            final byte[] increment = cassandraEnumeratorState.getIncrement().toByteArray();
+            objectOutputStream.writeInt(increment.length);
+            objectOutputStream.write(increment);
+
+            final byte[] startToken = cassandraEnumeratorState.getStartToken().toByteArray();
+            objectOutputStream.writeInt(startToken.length);
+            objectOutputStream.write(startToken);
+
+            final byte[] maxToken = cassandraEnumeratorState.getMaxToken().toByteArray();
+            objectOutputStream.writeInt(maxToken.length);
+            objectOutputStream.write(maxToken);
+
+            objectOutputStream.flush();
+            return byteArrayOutputStream.toByteArray();
+        }
+    }
+
+    @Override
+    public CassandraEnumeratorState deserialize(int version, byte[] serialized) throws IOException {
+        try (final ByteArrayInputStream byteArrayInputStream =
+                        new ByteArrayInputStream(serialized);
+                final ObjectInputStream objectInputStream =
+                        new ObjectInputStream(byteArrayInputStream)) {
+            final Queue<CassandraSplit> splitsToReassign = new ArrayDeque<>();
+            final int splitsToReassignSize = objectInputStream.readInt();
+            for (int i = 0; i < splitsToReassignSize; i++) {
+                final int splitSize = objectInputStream.readInt();
+                final byte[] splitBytes = new byte[splitSize];
+                if (objectInputStream.read(splitBytes) == -1) {
+                    throw new IOException(
+                            "EOF received while deserializing CassandraEnumeratorState.splitsToReassign");
+                }
+                final CassandraSplit split =
+                        CassandraSplitSerializer.INSTANCE.deserialize(
+                                CassandraSplitSerializer.CURRENT_VERSION, splitBytes);
+                splitsToReassign.add(split);
+            }
+
+            final long numSplitsLeftToGenerate = objectInputStream.readLong();
+
+            final int incrementSize = objectInputStream.readInt();
+            final byte[] incrementBytes = new byte[incrementSize];
+            if (objectInputStream.read(incrementBytes) == -1) {
+                throw new IOException(
+                        "EOF received while deserializing cassandraEnumeratorState.increment");
+            }
+            final BigInteger increment = new BigInteger(incrementBytes);
+
+            final int startTokenSize = objectInputStream.readInt();
+            final byte[] startTokenBytes = new byte[startTokenSize];
+            if (objectInputStream.read(startTokenBytes) == -1) {
+                throw new IOException(
+                        "EOF received while deserializing cassandraEnumeratorState.startToken");
+            }
+            final BigInteger startToken = new BigInteger(startTokenBytes);
+
+            final int maxTokenSize = objectInputStream.readInt();
+            final byte[] maxTokenBytes = new byte[maxTokenSize];
+            if (objectInputStream.read(maxTokenBytes) == -1) {
+                throw new IOException(
+                        "EOF received while deserializing cassandraEnumeratorState.maxToken");
+            }
+            final BigInteger maxToken = new BigInteger(maxTokenBytes);

Review Comment:
   :+1:
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1105844502


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -57,7 +57,22 @@ public CassandraSplitEnumerator(
 
     @Override
     public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
-        assignUnprocessedSplitToReader(subtaskId);
+        checkReaderRegistered(subtaskId);
+        final CassandraSplit cassandraSplit = state.getASplit();
+        if (cassandraSplit != null) {
+            LOG.info("Assigning splits to reader {}", subtaskId);
+            enumeratorContext.assignSplit(cassandraSplit, subtaskId);
+        } else {
+            LOG.info(
+                    "No split assigned to reader {} because the enumerator has no unassigned split left",
+                    subtaskId);
+        }
+        if (!state.hasMoreSplits()) {
+            LOG.info(
+                    "No more CassandraSplits to assign. Sending NoMoreSplitsEvent to reader {}.",
+                    subtaskId);
+            enumeratorContext.signalNoMoreSplits(subtaskId);
+        }

Review Comment:
   Sketchy thing, not ready for review. But yes you're right



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100252874


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        this.unprocessedSplits = new HashSet<>();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();

Review Comment:
   Yes I thought about that but the problem I had was when to close the session/cluster. I you prefer this solution, I can override `SourceReaderBase#close()` and close them there + create a factory to initialize cluster, session and mapper at the creation of SourceReader before the super(...) calll. Also, I find elegant the design of passing a map function to the emitter from the source reader. I'll do so



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101576215


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorState.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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Objects;
+import java.util.Queue;
+
+/** Sate for {@link CassandraSplitEnumerator} to track the splits yet to assign. */
+public class CassandraEnumeratorState {
+    private final Queue<CassandraSplit> unassignedSplits;
+
+    public CassandraEnumeratorState() {
+        this.unassignedSplits = new ArrayDeque<>();
+    }
+
+    public CassandraEnumeratorState(Queue<CassandraSplit> unassignedSplits) {
+        this.unassignedSplits = unassignedSplits;
+    }
+
+    public void addNewSplits(Collection<CassandraSplit> newSplits) {
+        unassignedSplits.addAll(newSplits);
+    }
+
+    public CassandraSplit getASplit() {

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101601871


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and Flink
+ * source parallelism.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final CassandraPartitioner partitioner;
+
+    public SplitsGenerator(CassandraPartitioner partitioner) {
+        this.partitioner = partitioner;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the ring.
+     *
+     * @param numSplits requested number of splits
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long numSplits) {
+        if (numSplits == 1) {
+            return Collections.singletonList(
+                    new CassandraSplit(partitioner.minToken(), partitioner.maxToken()));
+        }
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger splitSize =
+                (partitioner.ringSize()).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken, endToken = partitioner.minToken();
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            startToken = endToken;
+            endToken = startToken.add(splitSize);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken();
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+        }

Review Comment:
   :+1: for the general change but regarding `splitCount == numSplits` it is to make sure the last split in the list covers the max token be for division rounding. If I do this outside of the loop and add a final split it will be very small. I think it is better to extend the previous one.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073520909


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {
+        return (start.compareTo(end) >= 0)
+                ? end.subtract(start).add(ringSize)

Review Comment:
   See my general comments about Apache Beam Cassandra partitionner



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {
+        return (start.compareTo(end) >= 0)
+                ? end.subtract(start).add(ringSize)
+                : end.subtract(start);
+    }
+
+    /** @return true if the ringRange overlaps. Note that if start == end, then wrapping is true */

Review Comment:
   See my general comments about Apache Beam Cassandra partitionner



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073523022


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {
+        return (start.compareTo(end) >= 0)
+                ? end.subtract(start).add(ringSize)
+                : end.subtract(start);
+    }
+
+    /** @return true if the ringRange overlaps. Note that if start == end, then wrapping is true */
+    public boolean isWrapping() {
+        return start.compareTo(end) >= 0;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("(%s,%s]", start.toString(), end.toString());
+    }
+
+    @Override
+    public boolean equals(@Nullable Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        RingRange ringRange = (RingRange) o;
+
+        if (getStart() != null
+                ? !getStart().equals(ringRange.getStart())
+                : ringRange.getStart() != null) {
+            return false;
+        }
+        return getEnd() != null ? getEnd().equals(ringRange.getEnd()) : ringRange.getEnd() == null;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getStart() != null ? getStart().hashCode() : 0;

Review Comment:
   not nullable, I will adapt hashcode/equals 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1387058843

   > @zentol I did not author the `RingRange` and `SplitsGenerator` classes. I got them from the `Apache Beam` Cassandra connector. I agree, some notions need clarification. I'll add comments.
   
   I was thinking: this `SplitsGenerator/RingRanges` I got from the Apache Beam project seems weird in many aspects that you pointed out. Back in 2017 I coded a [partitionner for Cassandra Beam connector]( https://github.com/echauchot/beam/blob/BEAM-245-CassandraIO/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/DataSizeEstimates.java) that works with tokens also but that is simpler and supports all the Cassandra partitionners. Would you prefer that we use this other approach ?


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073554806


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            return Collections.singletonList(
+                    new CassandraSplit(Collections.singleton(totalRingRange)));
+        }
+        int tokenRangeCount = ringTokens.size();
+
+        List<RingRange> ringRanges = new ArrayList<>();
+        for (int i = 0; i < tokenRangeCount; i++) {
+            BigInteger start = ringTokens.get(i);
+            BigInteger stop = ringTokens.get((i + 1) % tokenRangeCount);
+
+            if (isNotInRange(start) || isNotInRange(stop)) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s) not in range of %s", start, stop, partitioner));
+            }
+            if (start.equals(stop) && tokenRangeCount != 1) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s): two nodes have the same token", start, stop));
+            }
+
+            BigInteger rangeSize = stop.subtract(start);
+            if (rangeSize.compareTo(BigInteger.ZERO) <= 0) {
+                // wrap around case
+                rangeSize = rangeSize.add(this.rangeSize);
+            }

Review Comment:
   See my general comments about Apache Beam Cassandra partitionner



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            return Collections.singletonList(
+                    new CassandraSplit(Collections.singleton(totalRingRange)));
+        }
+        int tokenRangeCount = ringTokens.size();
+
+        List<RingRange> ringRanges = new ArrayList<>();
+        for (int i = 0; i < tokenRangeCount; i++) {
+            BigInteger start = ringTokens.get(i);
+            BigInteger stop = ringTokens.get((i + 1) % tokenRangeCount);
+
+            if (isNotInRange(start) || isNotInRange(stop)) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s) not in range of %s", start, stop, partitioner));
+            }
+            if (start.equals(stop) && tokenRangeCount != 1) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s): two nodes have the same token", start, stop));
+            }
+
+            BigInteger rangeSize = stop.subtract(start);
+            if (rangeSize.compareTo(BigInteger.ZERO) <= 0) {
+                // wrap around case
+                rangeSize = rangeSize.add(this.rangeSize);
+            }
+
+            // the below, in essence, does this:
+            // splitCount = Maths.ceil((rangeSize / cluster range size) * totalSplitCount)

Review Comment:
   See my general comments about Apache Beam Cassandra partitionner



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
+     * splits. Each split can contain several token ranges in order to reduce the overhead of
+     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
+     * share the same replicas.
+     *
+     * @param totalSplitCount requested total amount of splits. This function may generate more
+     *     splits.
+     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
+     *     order.
+     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
+        if (totalSplitCount == 1) {
+            RingRange totalRingRange = RingRange.of(rangeMin, rangeMax);
+            return Collections.singletonList(
+                    new CassandraSplit(Collections.singleton(totalRingRange)));
+        }
+        int tokenRangeCount = ringTokens.size();
+
+        List<RingRange> ringRanges = new ArrayList<>();
+        for (int i = 0; i < tokenRangeCount; i++) {
+            BigInteger start = ringTokens.get(i);
+            BigInteger stop = ringTokens.get((i + 1) % tokenRangeCount);
+
+            if (isNotInRange(start) || isNotInRange(stop)) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s) not in range of %s", start, stop, partitioner));
+            }
+            if (start.equals(stop) && tokenRangeCount != 1) {
+                throw new RuntimeException(
+                        String.format(
+                                "Tokens (%s,%s): two nodes have the same token", start, stop));
+            }
+
+            BigInteger rangeSize = stop.subtract(start);
+            if (rangeSize.compareTo(BigInteger.ZERO) <= 0) {
+                // wrap around case
+                rangeSize = rangeSize.add(this.rangeSize);
+            }
+
+            // the below, in essence, does this:
+            // splitCount = Maths.ceil((rangeSize / cluster range size) * totalSplitCount)
+            BigInteger[] splitCountAndRemainder =
+                    rangeSize
+                            .multiply(BigInteger.valueOf(totalSplitCount))
+                            .divideAndRemainder(this.rangeSize);
+
+            int splitCount =
+                    splitCountAndRemainder[0].intValue()
+                            + (splitCountAndRemainder[1].equals(BigInteger.ZERO) ? 0 : 1);
+
+            LOG.debug("Dividing token range [{},{}) into {} splits", start, stop, splitCount);
+
+            // Make BigInteger list of all the endpoints for the splits, including both start and
+            // stop
+            List<BigInteger> endpointTokens = new ArrayList<>();
+            for (int j = 0; j <= splitCount; j++) {
+                BigInteger offset =
+                        rangeSize
+                                .multiply(BigInteger.valueOf(j))
+                                .divide(BigInteger.valueOf(splitCount));
+                BigInteger token = start.add(offset);
+                if (token.compareTo(rangeMax) > 0) {
+                    token = token.subtract(this.rangeSize);
+                }
+                // Long.MIN_VALUE is not a valid token and has to be silently incremented.
+                // See https://issues.apache.org/jira/browse/CASSANDRA-14684
+                endpointTokens.add(
+                        token.equals(BigInteger.valueOf(Long.MIN_VALUE))
+                                ? token.add(BigInteger.ONE)
+                                : token);
+            }
+
+            // Append the ringRanges between the endpoints
+            for (int j = 0; j < splitCount; j++) {
+                ringRanges.add(RingRange.of(endpointTokens.get(j), endpointTokens.get(j + 1)));
+                LOG.debug(
+                        "Split #{}: [{},{})",
+                        j + 1,
+                        endpointTokens.get(j),
+                        endpointTokens.get(j + 1));
+            }
+        }
+
+        BigInteger total = BigInteger.ZERO;
+        for (RingRange split : ringRanges) {
+            BigInteger size = split.span(rangeSize);
+            total = total.add(size);
+        }
+        if (!total.equals(rangeSize)) {
+            throw new RuntimeException(
+                    "Some tokens are missing from the splits. This should not happen.");
+        }
+        return coalesceRingRanges(getTargetSplitSize(totalSplitCount), ringRanges);
+    }
+
+    private boolean isNotInRange(BigInteger token) {
+        return token.compareTo(rangeMin) < 0 || token.compareTo(rangeMax) > 0;
+    }
+
+    private List<CassandraSplit> coalesceRingRanges(
+            BigInteger targetSplitSize, List<RingRange> ringRanges) {
+        List<CassandraSplit> coalescedSplits = new ArrayList<>();
+        List<RingRange> tokenRangesForCurrentSplit = new ArrayList<>();
+        BigInteger tokenCount = BigInteger.ZERO;
+
+        for (RingRange tokenRange : ringRanges) {
+            if (tokenRange.span(rangeSize).add(tokenCount).compareTo(targetSplitSize) > 0
+                    && !tokenRangesForCurrentSplit.isEmpty()) {
+                // enough tokens in that segment

Review Comment:
   See my general comments about Apache Beam Cassandra partitionner



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073635879


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits, enumeratorContext.currentParallelism());
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitioner = clusterMetadata.getPartitioner();
+        final SplitsGenerator splitsGenerator = new SplitsGenerator(partitioner);
+        if (MURMUR3PARTITIONER.equals(partitioner)) {
+            LOG.info("Murmur3Partitioner detected, splitting");
+            List<BigInteger> tokens =
+                    clusterMetadata.getTokenRanges().stream()
+                            .map(
+                                    tokenRange ->
+                                            new BigInteger(
+                                                    tokenRange.getEnd().getValue().toString()))
+                            .collect(Collectors.toList());
+            return splitsGenerator.generateSplits(numberOfSplits, tokens);
+        } else {
+            // Murmur3Partitioner is the default and recommended partitioner for Cassandra 1.2+
+            // see
+            // https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerAbout.html
+            LOG.warn(
+                    "The current Cassandra partitioner is {}, only Murmur3Partitioner is supported "
+                            + "for splitting, using an single split",

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073534572


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {

Review Comment:
   :+1:



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073721942


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitSerializer.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+/** Serializer for {@link CassandraSplit}. */
+public class CassandraSplitSerializer implements SimpleVersionedSerializer<CassandraSplit> {
+
+    public static final CassandraSplitSerializer INSTANCE = new CassandraSplitSerializer();
+
+    public static final int CURRENT_VERSION = 0;
+
+    private CassandraSplitSerializer() {}
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(CassandraSplit cassandraSplit) throws IOException {
+        try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                ObjectOutputStream objectOutputStream =
+                        new ObjectOutputStream(byteArrayOutputStream)) {
+            objectOutputStream.writeObject(cassandraSplit);

Review Comment:
   For my knowledge, where is the use of java serialization acceptable in Flink ? In the docs it says only for RPC data but I see it in use for state snapshots as well (random example: https://github.com/apache/flink/blob/b35cef5997568d75611307d11fca0dec415f99b4/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java#L302)



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073713994


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplitSerializer.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+/** Serializer for {@link CassandraSplit}. */
+public class CassandraSplitSerializer implements SimpleVersionedSerializer<CassandraSplit> {
+
+    public static final CassandraSplitSerializer INSTANCE = new CassandraSplitSerializer();
+
+    public static final int CURRENT_VERSION = 0;
+
+    private CassandraSplitSerializer() {}
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(CassandraSplit cassandraSplit) throws IOException {
+        try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                ObjectOutputStream objectOutputStream =
+                        new ObjectOutputStream(byteArrayOutputStream)) {
+            objectOutputStream.writeObject(cassandraSplit);

Review Comment:
   :+1:



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
zentol commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1385401376

   @echauchot I'm taking a look now.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100252874


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        this.unprocessedSplits = new HashSet<>();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();

Review Comment:
   Yes I thought about that but the problem I had was when to close the session/cluster. If you prefer this solution, I can override `SourceReaderBase#close()` and close them there + create a factory to initialize cluster, session and mapper at the creation of SourceReader before the super(...) calll. Also, I find elegant the design of passing a map function to the emitter from the source reader. I'll do so



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101601871


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and Flink
+ * source parallelism.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final CassandraPartitioner partitioner;
+
+    public SplitsGenerator(CassandraPartitioner partitioner) {
+        this.partitioner = partitioner;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the ring.
+     *
+     * @param numSplits requested number of splits
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits(long numSplits) {
+        if (numSplits == 1) {
+            return Collections.singletonList(
+                    new CassandraSplit(partitioner.minToken(), partitioner.maxToken()));
+        }
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger splitSize =
+                (partitioner.ringSize()).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken, endToken = partitioner.minToken();
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            startToken = endToken;
+            endToken = startToken.add(splitSize);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken();
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+        }

Review Comment:
   :+1: for the general change but regarding `splitCount == numSplits` it is to make sure the last split in the list covers the max token for division rounding. If I do this outside of the loop and add a final split it will be very small. I think it is better to extend the previous one.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1032611085


##########
flink-connector-cassandra/pom.xml:
##########
@@ -78,6 +78,12 @@ under the License.
 			<scope>provided</scope>
 		</dependency>
 
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-base</artifactId>

Review Comment:
   Please set this to `provided`. It is quite problematic w.r.t. API compatibility, because it may internally rely unstable APIs (since it's developed as part of Flink).



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol merged pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol merged PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137299698


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.net.InetSocketAddress;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+
+    private Cluster cluster;
+    private Session session;
+    private ClusterBuilder clusterBuilder;
+
+    public CassandraTestEnvironment() {
+        cassandraContainer = new CassandraContainer(DOCKER_CASSANDRA_IMAGE);
+        // more generous timeouts
+        addJavaOpts(
+                cassandraContainer,
+                "-Dcassandra.request_timeout_in_ms=30000",
+                "-Dcassandra.read_request_timeout_in_ms=15000",
+                "-Dcassandra.write_request_timeout_in_ms=6000");
+    }
+
+    @Override
+    public void startUp() throws Exception {
+        startEnv();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        stopEnv();
+    }
+
+    private static void addJavaOpts(GenericContainer<?> container, String... opts) {
+        String jvmOpts = container.getEnvMap().getOrDefault("JVM_OPTS", "");
+        container.withEnv("JVM_OPTS", jvmOpts + " " + StringUtils.join(opts, " "));
+    }
+
+    private void startEnv() throws Exception {
+        // configure container start to wait until cassandra is ready to receive queries
+        cassandraContainer.waitingFor(new CassandraQueryWaitStrategy());
+        // start with retrials
+        cassandraContainer.start();
+        cassandraContainer.followOutput(
+                new Slf4jLogConsumer(LOG),
+                OutputFrame.OutputType.END,
+                OutputFrame.OutputType.STDERR,
+                OutputFrame.OutputType.STDOUT);
+
+        cluster = cassandraContainer.getCluster();
+        clusterBuilder =
+                createBuilderWithConsistencyLevel(
+                        ConsistencyLevel.ONE,
+                        cassandraContainer.getHost(),
+                        cassandraContainer.getMappedPort(CQL_PORT));
+
+        session = cluster.connect();
+        session.execute(requestWithTimeout(CREATE_KEYSPACE_QUERY));
+        // create a dedicated table for split size tests (to avoid having to flush with each test)
+        insertTestDataForSplitSizeTests();

Review Comment:
   :+1: agree



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137343912


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 20;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        100_000_000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // tableSize / maxSplitMemorySize is too little compared to parallelism falling back to
+        // number of splits = parallelism

Review Comment:
   I guess its better to fall back to `parallelism` than `1` in case the table size metric couldn't be properly read from cassandra.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137115950


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/CassandraSource.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorStateSerializer;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraSplitEnumerator;
+import org.apache.flink.connector.cassandra.source.reader.CassandraSourceReaderFactory;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import javax.annotation.Nullable;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A bounded source to read from Cassandra and return a collection of entities as {@code
+ * DataStream<Entity>}. An entity is built by Cassandra mapper ({@code
+ * com.datastax.driver.mapping.EntityMapper}) based on a POJO containing annotations (as described
+ * in <a
+ * href="https://docs.datastax.com/en/developer/java-driver/3.11/manual/object_mapper/creating/">
+ * Cassandra object mapper</a>).
+ *
+ * <p>To use it, do the following:
+ *
+ * <pre>{@code
+ * ClusterBuilder clusterBuilder = new ClusterBuilder() {
+ *   @Override
+ *   protected Cluster buildCluster(Cluster.Builder builder) {
+ *     return builder.addContactPointsWithPorts(new InetSocketAddress(HOST,PORT))
+ *                   .withQueryOptions(new QueryOptions().setConsistencyLevel(CL))
+ *                   .withSocketOptions(new SocketOptions()
+ *                   .setConnectTimeoutMillis(CONNECT_TIMEOUT)
+ *                   .setReadTimeoutMillis(READ_TIMEOUT))
+ *                   .build();
+ *   }
+ * };
+ * long maxSplitMemorySize = ... //optional max split size in bytes. If not set, maxSplitMemorySize = tableSize / parallelism
+ * Source cassandraSource = new CassandraSource(clusterBuilder,
+ *                                              maxSplitMemorySize,
+ *                                              Pojo.class,
+ *                                              "select ... from KEYSPACE.TABLE ...;",
+ *                                              () -> new Mapper.Option[] {Mapper.Option.saveNullFields(true)});
+ *
+ * DataStream<Pojo> stream = env.fromSource(cassandraSource, WatermarkStrategy.noWatermarks(),
+ * "CassandraSource");
+ * }</pre>
+ */
+@PublicEvolving
+public class CassandraSource<OUT>
+        implements Source<OUT, CassandraSplit, CassandraEnumeratorState>, ResultTypeQueryable<OUT> {
+
+    public static final String CQL_PROHIBITED_CLAUSES_REGEXP =
+            "(?i).*(AVG|COUNT|MIN|MAX|SUM|ORDER|GROUP BY).*";
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137359264


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraTestEnvironment.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.testframe.TestResource;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.Statement;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.CassandraContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.CassandraQueryWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.net.InetSocketAddress;
+
+/**
+ * Junit test environment that contains everything needed at the test suite level: testContainer
+ * setup, keyspace setup, Cassandra cluster/session management ClusterBuilder setup).
+ */
+@Testcontainers
+public class CassandraTestEnvironment implements TestResource {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraTestEnvironment.class);
+    private static final String DOCKER_CASSANDRA_IMAGE = "cassandra:4.0.8";
+    private static final int CQL_PORT = 9042;
+
+    private static final int READ_TIMEOUT_MILLIS = 36000;
+
+    private static final long FLUSH_MEMTABLES_DELAY =
+            30_000L; // updating flushing mem table to SS tables is long, it is the minimum delay.
+
+    static final String KEYSPACE = "flink";
+
+    private static final String CREATE_KEYSPACE_QUERY =
+            "CREATE KEYSPACE "
+                    + KEYSPACE
+                    + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+
+    static final String SPLITS_TABLE = "flinksplits";
+    private static final String CREATE_SPLITS_TABLE_QUERY =
+            "CREATE TABLE " + KEYSPACE + "." + SPLITS_TABLE + " (id int PRIMARY KEY, counter int);";
+    private static final String INSERT_INTO_FLINK_SPLITS =
+            "INSERT INTO " + KEYSPACE + "." + SPLITS_TABLE + " (id, counter)" + " VALUES (%d, %d)";
+    private static final int NB_SPLITS_RECORDS = 1000;
+
+    @Container private final CassandraContainer cassandraContainer;
+
+    private Cluster cluster;
+    private Session session;
+    private ClusterBuilder clusterBuilder;
+
+    public CassandraTestEnvironment() {
+        cassandraContainer = new CassandraContainer(DOCKER_CASSANDRA_IMAGE);
+        // more generous timeouts
+        addJavaOpts(
+                cassandraContainer,
+                "-Dcassandra.request_timeout_in_ms=30000",
+                "-Dcassandra.read_request_timeout_in_ms=15000",
+                "-Dcassandra.write_request_timeout_in_ms=6000");
+    }
+
+    @Override
+    public void startUp() throws Exception {
+        startEnv();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        stopEnv();
+    }
+
+    private static void addJavaOpts(GenericContainer<?> container, String... opts) {
+        String jvmOpts = container.getEnvMap().getOrDefault("JVM_OPTS", "");
+        container.withEnv("JVM_OPTS", jvmOpts + " " + StringUtils.join(opts, " "));
+    }
+
+    private void startEnv() throws Exception {
+        // configure container start to wait until cassandra is ready to receive queries
+        cassandraContainer.waitingFor(new CassandraQueryWaitStrategy());
+        // start with retrials
+        cassandraContainer.start();
+        cassandraContainer.followOutput(
+                new Slf4jLogConsumer(LOG),
+                OutputFrame.OutputType.END,
+                OutputFrame.OutputType.STDERR,
+                OutputFrame.OutputType.STDOUT);
+
+        cluster = cassandraContainer.getCluster();
+        clusterBuilder =
+                createBuilderWithConsistencyLevel(
+                        ConsistencyLevel.ONE,
+                        cassandraContainer.getHost(),
+                        cassandraContainer.getMappedPort(CQL_PORT));
+
+        session = cluster.connect();
+        session.execute(requestWithTimeout(CREATE_KEYSPACE_QUERY));
+        // create a dedicated table for split size tests (to avoid having to flush with each test)
+        insertTestDataForSplitSizeTests();
+    }
+
+    private void insertTestDataForSplitSizeTests() throws Exception {
+        session.execute(requestWithTimeout(CREATE_SPLITS_TABLE_QUERY));
+        for (int i = 0; i < NB_SPLITS_RECORDS; i++) {
+            session.execute(requestWithTimeout(String.format(INSERT_INTO_FLINK_SPLITS, i, i)));
+        }
+        flushMemTables(SPLITS_TABLE);
+    }
+
+    private void stopEnv() {
+
+        if (session != null) {
+            session.close();
+        }
+        if (cluster != null) {
+            cluster.close();
+        }
+        cassandraContainer.stop();
+    }
+
+    private ClusterBuilder createBuilderWithConsistencyLevel(
+            ConsistencyLevel consistencyLevel, String host, int port) {
+        return new ClusterBuilder() {
+            @Override
+            protected Cluster buildCluster(Cluster.Builder builder) {
+                return builder.addContactPointsWithPorts(new InetSocketAddress(host, port))
+                        .withQueryOptions(
+                                new QueryOptions()
+                                        .setConsistencyLevel(consistencyLevel)
+                                        .setSerialConsistencyLevel(ConsistencyLevel.LOCAL_SERIAL))
+                        .withSocketOptions(
+                                new SocketOptions()
+                                        // default timeout x 3
+                                        .setConnectTimeoutMillis(15000)
+                                        // default timeout x3 and higher than
+                                        // request_timeout_in_ms at the cluster level
+                                        .setReadTimeoutMillis(READ_TIMEOUT_MILLIS))
+                        .withoutJMXReporting()
+                        .withoutMetrics()
+                        .build();
+            }
+        };
+    }
+
+    /**
+     * Force the flush of cassandra memTables to SSTables in order to update size_estimates. It is
+     * needed for the tests because we just inserted records, we need to force cassandra to update
+     * size_estimates system table.
+     */
+    void flushMemTables(String table) throws Exception {
+        cassandraContainer.execInContainer("nodetool", "flush", KEYSPACE, table);

Review Comment:
   ok. Just stumbled on this in an SO thread and wanted to double-check.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137317001


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 20;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        100_000_000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // tableSize / maxSplitMemorySize is too little compared to parallelism falling back to
+        // number of splits = parallelism

Review Comment:
   I asked myself the same but I concluded that having a parallelism > 1 is profitable compared to 1 (the reduced reading time compensates the added task synchronisation time) as long as the data is not too little. Of course everything depends on the threshold set for "too little'



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138381568


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high

Review Comment:
   Well, off course this ratio is totally arbitrary. I just sketched a quick code to serve as a base discussion on safety measures for the number of splits. I agree with your arguments on limit cases (freedom for the user and dangerousness of sudden threshold crossing). 
   
   I'll implement the lazy splits.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1142173968


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private CassandraEnumeratorState state;
+    private final Cluster cluster;
+    private final Long maxSplitMemorySize;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder,
+            Long maxSplitMemorySize,
+            String keyspace,
+            String table) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+        this.maxSplitMemorySize = maxSplitMemorySize;
+        this.session = cluster.newSession();
+        this.keyspace = keyspace;
+        this.table = table;
+    }
+
+    @Override
+    public void start() {
+        enumeratorContext.callAsync(
+                this::prepareSplits,
+                (preparedState, throwable) -> {
+                    LOG.debug("Initialized CassandraEnumeratorState: {}", preparedState.toString());
+                    state = preparedState;
+                });
+    }
+
+    private CassandraEnumeratorState prepareSplits() {
+        final int parallelism = enumeratorContext.currentParallelism();
+        final String partitionerName = cluster.getMetadata().getPartitioner();
+        final SplitsGenerator.CassandraPartitioner partitioner =
+                partitionerName.contains(MURMUR3PARTITIONER.getClassName())
+                        ? MURMUR3PARTITIONER
+                        : RANDOMPARTITIONER;
+        final SplitsGenerator splitsGenerator =
+                new SplitsGenerator(
+                        partitioner, session, keyspace, table, parallelism, maxSplitMemorySize);
+        return splitsGenerator.prepareSplits();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        checkReaderRegistered(subtaskId);
+        final CassandraSplit cassandraSplit = state.getNextSplit();
+        if (cassandraSplit != null) {
+            LOG.info("Assigning splits to reader {}", subtaskId);
+            enumeratorContext.assignSplit(cassandraSplit, subtaskId);
+        } else {
+            LOG.info(
+                    "No split assigned to reader {} because the enumerator has no unassigned split left. Sending NoMoreSplitsEvent to reader",
+                    subtaskId);
+            enumeratorContext.signalNoMoreSplits(subtaskId);
+        }
+    }
+
+    @Override
+    public void addSplitsBack(List<CassandraSplit> splits, int subtaskId) {
+        // splits that were assigned to a failed reader and that were not part of a checkpoint, so
+        // after
+        // restoration, they need to be reassigned.

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087835183


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -115,29 +109,22 @@ private List<CassandraSplit> discoverSplits() {
     @Override
     public void addSplitsBack(List<CassandraSplit> splits, int subtaskId) {
         LOG.info("Add {} splits back to CassandraSplitEnumerator.", splits.size());
-        state.addNewSplits(splits, enumeratorContext.currentParallelism());
-        assignUnprocessedSplitsToReader(subtaskId);
+        state.addNewSplits(splits);
     }
 
     @Override
     public void addReader(int subtaskId) {
         LOG.info("Adding reader {} to CassandraSplitEnumerator.", subtaskId);
-        assignUnprocessedSplitsToReader(subtaskId);
+        assignUnprocessedSplitToReader(subtaskId);
     }
 
-    private void assignUnprocessedSplitsToReader(int readerId) {
+    private void assignUnprocessedSplitToReader(int readerId) {
         checkReaderRegistered(readerId);
-
-        final Set<CassandraSplit> splitsForReader = state.getSplitsForReader(readerId);
-        if (splitsForReader != null && !splitsForReader.isEmpty()) {
-            Map<Integer, List<CassandraSplit>> assignment = new HashMap<>();
-            assignment.put(readerId, Lists.newArrayList(splitsForReader));
-            LOG.info("Assigning splits to reader {}", assignment);
-            enumeratorContext.assignSplits(new SplitsAssignment<>(assignment));
-        }
-
-        // periodically partition discovery is disabled, signal NoMoreSplitsEvent to the reader
-        LOG.debug(
+        final CassandraSplit cassandraSplit = state.getASplit();

Review Comment:
   Doesn't this have to handle the case where the split is `null`?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087607875


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set

Review Comment:
   I pushed an impl that assigns only a single split to each reader.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1085585517


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   No. `CassandraSplits` are assigned to `CassandraSplitReader` (see `CassandraSplitEnumerator#assignUnprocessedSplitsToReader`). A `CassandraSplitReader` might be assigned several splits in case the **strange** splitGenerator returns more splits than requested parallelism. With the new `splitGenerator` there will be exactly as many splits as the parallelism.  There will be one split per split reader so no need for the set.
   
   Here we read all the records that belong to **the splits assigned to the reader**. After split generator change no more loop.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087957846


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+    private static final String MURMUR3PARTITIONER = "org.apache.cassandra.dht.Murmur3Partitioner";
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits, enumeratorContext.currentParallelism());
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitioner = clusterMetadata.getPartitioner();
+        final SplitsGenerator splitsGenerator = new SplitsGenerator(partitioner);
+        if (MURMUR3PARTITIONER.equals(partitioner)) {
+            LOG.info("Murmur3Partitioner detected, splitting");
+            List<BigInteger> tokens =
+                    clusterMetadata.getTokenRanges().stream()
+                            .map(
+                                    tokenRange ->
+                                            new BigInteger(
+                                                    tokenRange.getEnd().getValue().toString()))
+                            .collect(Collectors.toList());
+            return splitsGenerator.generateSplits(numberOfSplits, tokens);
+        } else {
+            // Murmur3Partitioner is the default and recommended partitioner for Cassandra 1.2+
+            // see
+            // https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerAbout.html
+            LOG.warn(
+                    "The current Cassandra partitioner is {}, only Murmur3Partitioner is supported "
+                            + "for splitting, using an single split",
+                    partitioner);
+            return splitsGenerator.generateSplits(1, Collections.emptyList());
+        }
+    }
+
+    @Override
+    public void addSplitsBack(List<CassandraSplit> splits, int subtaskId) {
+        LOG.info("Add {} splits back to CassandraSplitEnumerator.", splits.size());
+        state.addNewSplits(splits, enumeratorContext.currentParallelism());
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        LOG.info("Adding reader {} to CassandraSplitEnumerator.", subtaskId);
+        assignUnprocessedSplitsToReader(subtaskId);
+    }
+
+    private void assignUnprocessedSplitsToReader(int readerId) {
+        checkReaderRegistered(readerId);
+
+        final Set<CassandraSplit> splitsForReader = state.getSplitsForReader(readerId);

Review Comment:
   ok thanks. This is what I thought



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1089214272


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        this.unprocessedSplits = new HashSet<>();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1127585102


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {

Review Comment:
   > Of course the splits are scoped to the reader. But say, if parallelism=1, then you're reading all records from all splits into memory. Even with a higher parallelism cases I don't see this working very well for larger tables.
   > 
   > While the split reader API docs are a bit unclear on this, you _are_ allowed to exist the fetch call early. If you adjust the splits accordingly fetch() will be called again later on and you can continue where you left off.
   
   Submitted [a doc improvement PR](https://github.com/apache/flink/pull/22111)



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073306704


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/CassandraSplit.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.api.connector.source.SourceSplit;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * Immutable {@link SourceSplit} for Cassandra source. A Cassandra split is just a set of {@link

Review Comment:
   :+1:
   



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073554548


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {

Review Comment:
   See my general comments about Apache Beam Cassandra partitionner



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }
+
+    /**
+     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}

Review Comment:
   See my general comments about Apache Beam Cassandra partitionner



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1073553165


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s by generating {@link RingRange}s based on Cassandra
+ * cluster partitioner and Flink source parallelism.
+ */
+public final class SplitsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+
+    private final String partitioner;
+    private final BigInteger rangeMin;
+    private final BigInteger rangeMax;
+    private final BigInteger rangeSize;
+
+    public SplitsGenerator(String partitioner) {
+        this.partitioner = partitioner;
+        rangeMin = getRangeMin();
+        rangeMax = getRangeMax();
+        rangeSize = getRangeSize();
+    }
+
+    private BigInteger getRangeMin() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.ZERO;
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).negate();
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeMax() {
+        if (partitioner.endsWith("RandomPartitioner")) {
+            return BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE);
+        } else if (partitioner.endsWith("Murmur3Partitioner")) {
+            return BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported partitioner. " + "Only Random and Murmur3 are supported");
+        }
+    }
+
+    private BigInteger getRangeSize() {
+        return rangeMax.subtract(rangeMin).add(BigInteger.ONE);
+    }

Review Comment:
   See my general comments about Apache Beam Cassandra partitionner



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {
+
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private RingRange(BigInteger start, BigInteger end) {
+        this.start = start;
+        this.end = end;
+    }
+
+    public static RingRange of(BigInteger start, BigInteger end) {
+        return new RingRange(start, end);
+    }
+
+    public BigInteger getStart() {
+        return start;
+    }
+
+    public BigInteger getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the size of this range.
+     *
+     * @return size of the range, max - range, in case of wrap
+     */
+    BigInteger span(BigInteger ringSize) {

Review Comment:
   https://docs.datastax.com/en/cassandra-oss/3.x/cassandra/architecture/archPartitionerM3P.html



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1085311136


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set
+        this.unprocessedSplits = ConcurrentHashMap.newKeySet();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();
+    }
+
+    @Override
+    public RecordsWithSplitIds<CassandraRow> fetch() {
+        Map<String, Collection<CassandraRow>> recordsBySplit = new HashMap<>();
+        Set<String> finishedSplits = new HashSet<>();
+        Metadata clusterMetadata = cluster.getMetadata();
+
+        String partitionKey = getPartitionKey(clusterMetadata);
+        String finalQuery = generateRangeQuery(query, partitionKey);
+        PreparedStatement preparedStatement = session.prepare(finalQuery);
+        // Set wakeup to false to start consuming.
+        wakeup.compareAndSet(true, false);
+        for (CassandraSplitState cassandraSplitState : unprocessedSplits) {
+            // allow to interrupt the reading of splits as requested in the API
+            if (wakeup.get()) {
+                break;
+            }
+            if (!cassandraSplitState.isEmpty()) {
+                try {
+                    final Set<RingRange> ringRanges =
+                            cassandraSplitState.getUnprocessedRingRanges();
+                    final String cassandraSplitId = cassandraSplitState.getSplitId();
+
+                    for (RingRange ringRange : ringRanges) {
+                        Token startToken =
+                                clusterMetadata.newToken(ringRange.getStart().toString());
+                        Token endToken = clusterMetadata.newToken(ringRange.getEnd().toString());
+                        if (ringRange.isWrapping()) {
+                            // A wrapping range is one that overlaps from the end of the partitioner
+                            // range and its
+                            // start (ie : when the start token of the split is greater than the end
+                            // token)
+                            // We need to generate two queries here : one that goes from the start
+                            // token to the end
+                            // of
+                            // the partitioner range, and the other from the start of the
+                            // partitioner range to the
+                            // end token of the split.

Review Comment:
   This comes from Beam master Cassandra split. This will be changed with the other split mechanism



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1086431597


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraRecordEmitter.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.api.connector.source.SourceOutput;
+import org.apache.flink.connector.base.source.reader.RecordEmitter;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.streaming.connectors.cassandra.MapperOptions;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.ExecutionInfo;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.mapping.Mapper;
+import com.datastax.driver.mapping.MappingManager;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * {@link RecordEmitter} that converts the {@link CassandraRow} read by the {@link
+ * CassandraSplitReader} to specified POJO and output it while updating splits state. This class
+ * uses the Cassandra driver mapper to map the row to the POJO.
+ *
+ * @param <OUT> type of POJO record to output
+ */
+public class CassandraRecordEmitter<OUT>
+        implements RecordEmitter<CassandraRow, OUT, CassandraSplitState> {
+
+    private final Mapper<OUT> mapper;
+
+    public CassandraRecordEmitter(
+            Class<OUT> pojoClass, ClusterBuilder clusterBuilder, MapperOptions mapperOptions) {
+        // session and cluster are managed at the SplitReader level. So we need to create one
+        // locally here just to me able to create the mapper.
+        final Cluster cluster = clusterBuilder.getCluster();
+        final Session session = cluster.connect();

Review Comment:
   Well, mapping from row to pojo is rather complex and is supported out of the box by Cassandra mapper. Would not it be easier for the user if we guide him in using the mapper rather than leaving him provide a potentially buggy mapping function ? I agree, crating the cluster and session here just for mapper creation is sketchy but I cannot get them in `CassandraRecordEmitter` from the `CassandraSplitReader`. WDYT ?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087607875


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set

Review Comment:
   I've pushed an impl that assigns only a single split to each reader ans this removed the need for cucurrent set



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set

Review Comment:
   I've pushed an impl that assigns only a single split to each reader ans this removed the need for concurrent set



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1087607875


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        // need a thread safe set

Review Comment:
   I'll push an impl that assigns only a single split to each reader.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1088718260


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/RingRange.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+/**
+ * Represents a portion of Cassandra token ring. It is a range between a start token and an end
+ * token.
+ */
+public final class RingRange implements Serializable {

Review Comment:
   removed serializable also of CassandraEnumeratorState (that was the only class left)



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1363328649

   @RyanSkraba thanks for your review. I addressed all your comments PTAL


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
zentol commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1344270870

   >  Is there anything I can do to help the review ?
   
   For better or worse, no :)
   
   Holidays are coming up so I won't review it this year (:scream:) at the very least.
   If nothing happened til January (which I'd like to avoid!) I'll take a look myself to get things moving.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1055281383


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorState.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+/** Sate for {@link CassandraSplitEnumerator} to track the splits yet to assign. */
+public class CassandraEnumeratorState implements Serializable {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraEnumeratorState.class);
+
+    // map readerId to splits
+    private final Map<Integer, Set<CassandraSplit>> unassignedSplits = new HashMap<>();
+
+    public void addNewSplits(Collection<CassandraSplit> newSplits, int numReaders) {
+        for (CassandraSplit split : newSplits) {
+            int ownerReader = getOwnerReader(numReaders, split);
+            unassignedSplits.computeIfAbsent(ownerReader, r -> new HashSet<>()).add(split);
+        }
+    }
+
+    private int getOwnerReader(int numReaders, CassandraSplit split) {
+        // readerId == subTaksId == 0 or 1 if numReaders == 2 so  modulo is fine for ownerReader

Review Comment:
   :+1: I will also make the comment more generic



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101570853


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -41,139 +41,31 @@ public SplitsGenerator(CassandraPartitioner partitioner) {
     }
 
     /**
-     * Given properly ordered list of Cassandra tokens, compute at least {@code totalSplitCount}
-     * splits. Each split can contain several token ranges in order to reduce the overhead of
-     * Cassandra vnodes. Currently, token range grouping is not smart and doesn't check if they
-     * share the same replicas.
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the ring.
      *
-     * @param totalSplitCount requested total amount of splits. This function may generate more
-     *     splits.
-     * @param ringTokens list of all start tokens in Cassandra cluster. They have to be in ring
-     *     order.
-     * @return list containing at least {@code totalSplitCount} CassandraSplits.
+     * @param numSplits requested number of splits
+     * @return list containing {@code numSplits} CassandraSplits.
      */
-    public List<CassandraSplit> generateSplits(long totalSplitCount, List<BigInteger> ringTokens) {
-        if (totalSplitCount == 1) {
-            RingRange totalRingRange = RingRange.of(partitioner.min(), partitioner.max());
+    public List<CassandraSplit> generateSplits(long numSplits) {

Review Comment:
   for the current code yes but I'll add split size evaluation (see my proposal in another comment) So I'd need some state and will keep SplitsGenerator



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1101571880


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/split/SplitsGeneratorTest.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link SplitsGenerator}. */
+public final class SplitsGeneratorTest {

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1100252874


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/reader/CassandraSplitReader.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitState;
+import org.apache.flink.connector.cassandra.source.split.RingRange;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SplitReader} for Cassandra source. This class is responsible for fetching the records as
+ * {@link CassandraRow}. For that, it executes a range query (query that outputs records belonging
+ * to a {@link RingRange}) based on the user specified query. This class manages the Cassandra
+ * cluster and session.
+ */
+public class CassandraSplitReader implements SplitReader<CassandraRow, CassandraSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitReader.class);
+    public static final String SELECT_REGEXP = "(?i)select .+ from (\\w+)\\.(\\w+).*;$";
+
+    private final Cluster cluster;
+    private final Session session;
+    private final Set<CassandraSplitState> unprocessedSplits;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final String query;
+
+    public CassandraSplitReader(ClusterBuilder clusterBuilder, String query) {
+        this.unprocessedSplits = new HashSet<>();
+        this.query = query;
+        cluster = clusterBuilder.getCluster();
+        session = cluster.connect();

Review Comment:
   Yes I thought about that but the problem I had was when to close the session/cluster. What I can do is override `SourceReaderBase#close()` and close them there + create a static factory to initialize cluster and session at the creation of SourceReader before the super(...) calll. Also, I like the design of passing a map function to the emitter from the source reader. 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1104184317


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.CassandraSource;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** tests for query generation and query sanity checks. */
+class CassandraQueryTest {
+
+    @Test
+    public void testKeySpaceTableExtractionRegexp() {
+        final Pattern pattern = Pattern.compile(CassandraSplitReader.SELECT_REGEXP);
+        Matcher matcher;
+        matcher = pattern.matcher("SELECT field FROM keyspace.table where field = value;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("SELECT * FROM keyspace.table;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1, field2 from keyspace.table;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1, field2 from keyspace.table LIMIT(1000);");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1 from keyspace.table ;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1 from keyspace.table where field1=1;");
+        assertThat(matcher.matches()).isTrue();
+        assertThat(matcher.group(1)).isEqualTo("keyspace");
+        assertThat(matcher.group(2)).isEqualTo("table");
+
+        matcher = pattern.matcher("select field1 from table;"); // missing keyspace
+        assertThat(matcher.matches()).isFalse();
+
+        matcher = pattern.matcher("select field1 from keyspace.table"); // missing ";"
+        assertThat(matcher.matches()).isFalse();
+    }
+
+    @Test
+    public void testProhibitedClauses() {
+        assertThatThrownBy(
+                        () ->
+                                CassandraSource.checkQueryValidity(
+                                        "SELECT COUNT(*) from flink.table;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");
+        assertThatThrownBy(
+                        () -> CassandraSource.checkQueryValidity("SELECT AVG(*) from flink.table;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");
+
+        assertThatThrownBy(
+                        () -> CassandraSource.checkQueryValidity("SELECT MIN(*) from flink.table;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");
+        assertThatThrownBy(
+                        () -> CassandraSource.checkQueryValidity("SELECT MAX(*) from flink.table;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");
+        assertThatThrownBy(
+                        () -> CassandraSource.checkQueryValidity("SELECT SUM(*) from flink.table;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");
+        assertThatThrownBy(
+                        () ->
+                                CassandraSource.checkQueryValidity(
+                                        "SELECT field1, field2 from flink.table ORDER BY field1;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");
+        assertThatThrownBy(
+                        () ->
+                                CassandraSource.checkQueryValidity(
+                                        "SELECT field1, field2 from flink.table GROUP BY field1;"))
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessageContaining("query must not contain aggregate or order clauses ");

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1105844502


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -57,7 +57,22 @@ public CassandraSplitEnumerator(
 
     @Override
     public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
-        assignUnprocessedSplitToReader(subtaskId);
+        checkReaderRegistered(subtaskId);
+        final CassandraSplit cassandraSplit = state.getASplit();
+        if (cassandraSplit != null) {
+            LOG.info("Assigning splits to reader {}", subtaskId);
+            enumeratorContext.assignSplit(cassandraSplit, subtaskId);
+        } else {
+            LOG.info(
+                    "No split assigned to reader {} because the enumerator has no unassigned split left",
+                    subtaskId);
+        }
+        if (!state.hasMoreSplits()) {
+            LOG.info(
+                    "No more CassandraSplits to assign. Sending NoMoreSplitsEvent to reader {}.",
+                    subtaskId);
+            enumeratorContext.signalNoMoreSplits(subtaskId);
+        }

Review Comment:
   sketchy thing, not ready for review. But yes you're right



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137361425


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.CassandraSource;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** tests for query generation and query sanity checks. */
+class CassandraQueryTest {
+
+    private static final Pattern SELECT_PATTERN = Pattern.compile(CassandraSource.SELECT_REGEXP);
+
+    @Test
+    public void testKeySpaceTableExtractionRegexp() {
+        Arrays.asList(
+                        "select field FROM keyspace.table where field = value;",
+                        "select * FROM keyspace.table;",
+                        "select field1, field2 from keyspace.table;",
+                        "select field1, field2 from keyspace.table LIMIT(1000);",
+                        "select field1 from keyspace.table ;",
+                        "select field1 from keyspace.table where field1=1;")
+                .forEach(CassandraQueryTest::assertQueryFormatCorrect);
+
+        Arrays.asList(
+                        "select field1 from table;", // missing keyspace
+                        "select field1 from keyspace.table" // missing ";"
+                        )

Review Comment:
   I figured that since you're testing for missing `;` that you're also trying to catch user errors.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137369940


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class generates {@link CassandraSplit}s based on Cassandra cluster partitioner and cluster
+ * statistics. It estimates the total size of the table using Cassandra system table
+ * system.size_estimates. But there is no way to estimate the size of the data with the optional SQL
+ * filters without reading the data. So the splits can be smaller than {@param maxSplitMemorySize}
+ * when the query is executed.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    private static final int ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO = 10;
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Split Cassandra tokens ring into {@link CassandraSplit}s containing each a range of the
+     * Cassandra ring of {@param maxSplitMemorySize}. If {@param maxSplitMemorySize} is not defined,
+     * or is too high or too low compared to the task parallelism, then it generates as many {@link
+     * CassandraSplit}s as the task parallelism.
+     *
+     * @return list containing {@code numSplits} CassandraSplits.
+     */
+    public List<CassandraSplit> generateSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            final long estimateTableSize = estimateTableSize();
+            LOG.debug("Estimated table size for table {} is {} bytes", table, estimateTableSize);
+            numSplits = estimateTableSize / maxSplitMemorySize;
+            if (numSplits == 0 // estimateTableSize can be null in some cases (see javadoc)
+                    || numSplits < parallelism / ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO // too low
+                    || numSplits
+                            > (long) parallelism
+                                    * ACCEPTABLE_NB_SPLIT_PARALLELISM_RATIO) { // too high
+                LOG.info(
+                        "maxSplitMemorySize set value leads to {} splits with a task parallelism of {}. Creating as many splits as parallelism",
+                        numSplits,
+                        parallelism);
+                numSplits = parallelism;
+            }
+        } else { // not defined
+            LOG.info("maxSplitMemorySize not set. Creating as many splits as parallelism");
+            numSplits = parallelism;
+        }
+
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken = partitioner.minToken;
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            BigInteger endToken = startToken.add(increment);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken;
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+            startToken = endToken;
+        }
+        LOG.debug("Generated {} splits : {}", splits.size(), splits);
+        return splits;
+    }
+
+    /**
+     * Estimates the size of the table in bytes. Cassandra size estimates can be 0 if the data was
+     * just inserted and the amount of data in the table was small. This is very common situation
+     * during tests.
+     */
+    @VisibleForTesting
+    public long estimateTableSize() {
+        List<TokenRange> tokenRanges = getTokenRangesOfTable();
+        long size = 0L;
+        for (TokenRange tokenRange : tokenRanges) {
+            size += tokenRange.meanPartitionSize * tokenRange.partitionCount;
+        }
+        return Math.round(size / getRingFraction(tokenRanges));
+    }
+
+    /**
+     * The values that we get from system.size_estimates are for one node. We need to extrapolate to
+     * the whole cluster. This method estimates the percentage, the node represents in the cluster.
+     *
+     * @param tokenRanges The list of {@link TokenRange} to estimate
+     * @return The percentage the node represent in the whole cluster
+     */
+    private float getRingFraction(List<TokenRange> tokenRanges) {
+        BigInteger addressedTokens = BigInteger.ZERO;
+        for (TokenRange tokenRange : tokenRanges) {
+            addressedTokens =
+                    addressedTokens.add(distance(tokenRange.rangeStart, tokenRange.rangeEnd));
+        }
+        // it is < 1 because it is a percentage
+        return Float.valueOf(addressedTokens.divide(partitioner.ringSize).toString());

Review Comment:
   I edited my comment to clarify what scenario I'm thinking of (not related to `ringSize`)



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137367392


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/reader/CassandraQueryTest.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.flink.connector.cassandra.source.reader;
+
+import org.apache.flink.connector.cassandra.source.CassandraSource;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** tests for query generation and query sanity checks. */
+class CassandraQueryTest {
+
+    private static final Pattern SELECT_PATTERN = Pattern.compile(CassandraSource.SELECT_REGEXP);
+
+    @Test
+    public void testKeySpaceTableExtractionRegexp() {
+        Arrays.asList(
+                        "select field FROM keyspace.table where field = value;",
+                        "select * FROM keyspace.table;",
+                        "select field1, field2 from keyspace.table;",
+                        "select field1, field2 from keyspace.table LIMIT(1000);",
+                        "select field1 from keyspace.table ;",
+                        "select field1 from keyspace.table where field1=1;")
+                .forEach(CassandraQueryTest::assertQueryFormatCorrect);
+
+        Arrays.asList(
+                        "select field1 from table;", // missing keyspace
+                        "select field1 from keyspace.table" // missing ";"
+                        )

Review Comment:
   yes but `;` is kind of optional (tolerance) depending on the client CQL tool that sent the request so it is a common mistake



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1477662012

   @echauchot I've made some final adjustments, PTAL.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1142227595


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This class prepares the generation of {@link CassandraSplit}s based on Cassandra cluster
+ * partitioner and cluster statistics. It estimates the total size of the table using Cassandra
+ * system table system.size_estimates.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    @VisibleForTesting public Long minSplitMemorySize = 10_000_000L; // 10 MB
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Prepare the {@param CassandraEnumeratorState} for lazy generation of {@link CassandraSplit}s:
+     * calculate {@code numSplitsToGenerate} based on estimated target table size and provided
+     * {@code maxSplitMemorySize} and calculate {@code increment} which is the size of a split in
+     * tokens.
+     */
+    public CassandraEnumeratorState prepareSplits() {
+        final long numSplitsToGenerate = decideOnNumSplits();
+        final BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplitsToGenerate)));
+        final BigInteger startToken = partitioner.minToken;
+        return new CassandraEnumeratorState(
+                numSplitsToGenerate,
+                increment,
+                startToken,
+                partitioner.maxToken,
+                new ArrayDeque<>());
+    }
+
+    /**
+     * Determine {@code numSplits} based on the estimation of the target table size and user defined
+     * {@code maxSplitMemorySize}. Provide fallbacks when table size is unavailable, too few or too
+     * many splits are calculated.
+     */
+    private long decideOnNumSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            checkState(
+                    maxSplitMemorySize >= minSplitMemorySize,
+                    "Defined maxSplitMemorySize (%s) is below minimum (%s)",
+                    maxSplitMemorySize,
+                    minSplitMemorySize);
+            final long estimateTableSize = estimateTableSize();
+            if (estimateTableSize == 0) { // size estimates unavailable
+                LOG.info(
+                        "Cassandra size estimates are not available for {}.{} table. Creating as many splits as parallelism ({})",
+                        keyspace,
+                        table,
+                        parallelism);
+                numSplits = parallelism;
+            } else {
+                LOG.debug(
+                        "Estimated size for {}.{} table is {} bytes",
+                        keyspace,
+                        table,
+                        estimateTableSize);
+                numSplits =
+                        estimateTableSize / maxSplitMemorySize == 0
+                                ? parallelism // estimateTableSize under sizes maxSplitMemorySize
+                                // creating as many splits as parallelism
+                                : estimateTableSize / maxSplitMemorySize;
+                LOG.info(
+                        "maxSplitMemorySize set value ({}) leads to the creation of {} splits",
+                        maxSplitMemorySize,
+                        numSplits);
+            }
+        } else { // maxSplitMemorySize not defined

Review Comment:
   fair enough



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1475934338

   @zentol I addressed all the points PTAL.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1141957092


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -75,54 +77,68 @@ public SplitsGenerator(
      * @return list containing {@code numSplits} CassandraSplits.
      */
     public List<CassandraSplit> generateSplits() {
+        long numSplits = decideOnNumSplits();
+        List<CassandraSplit> splits = new ArrayList<>();
+        BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplits)));
+
+        BigInteger startToken = partitioner.minToken;
+        for (int splitCount = 1; splitCount <= numSplits; splitCount++) {
+            BigInteger endToken = startToken.add(increment);
+            if (splitCount == numSplits) {
+                endToken = partitioner.maxToken;
+            }
+            splits.add(new CassandraSplit(startToken, endToken));
+            startToken = endToken;
+        }
+        LOG.debug("Generated {} splits : {}", splits.size(), splits);
+        return splits;
+    }
+
+    /**
+     * Determine {@code numSplits} based on the estimation of the target table size and user defined
+     * {@code maxSplitMemorySize}. Add fallbacks when table size is unavailable, too few or too many
+     * splits are calculated.
+     */
+    private long decideOnNumSplits() {
         long numSplits;
         if (maxSplitMemorySize != null) {
+            checkState(

Review Comment:
   This should be checked in the constructor of the `CassandraSource` to fail as early as possible.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private CassandraEnumeratorState state;
+    private final Cluster cluster;
+    private final Long maxSplitMemorySize;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder,
+            Long maxSplitMemorySize,
+            String keyspace,
+            String table) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+        this.maxSplitMemorySize = maxSplitMemorySize;
+        this.session = cluster.newSession();
+        this.keyspace = keyspace;
+        this.table = table;
+    }
+
+    @Override
+    public void start() {
+        enumeratorContext.callAsync(
+                this::prepareSplits,
+                (preparedState, throwable) -> {
+                    LOG.debug("Initialized CassandraEnumeratorState: {}", preparedState.toString());
+                    state = preparedState;
+                });
+    }
+
+    private CassandraEnumeratorState prepareSplits() {
+        final int parallelism = enumeratorContext.currentParallelism();
+        final String partitionerName = cluster.getMetadata().getPartitioner();
+        final SplitsGenerator.CassandraPartitioner partitioner =
+                partitionerName.contains(MURMUR3PARTITIONER.getClassName())
+                        ? MURMUR3PARTITIONER
+                        : RANDOMPARTITIONER;
+        final SplitsGenerator splitsGenerator =
+                new SplitsGenerator(
+                        partitioner, session, keyspace, table, parallelism, maxSplitMemorySize);
+        return splitsGenerator.prepareSplits();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        checkReaderRegistered(subtaskId);
+        final CassandraSplit cassandraSplit = state.getNextSplit();
+        if (cassandraSplit != null) {
+            LOG.info("Assigning splits to reader {}", subtaskId);
+            enumeratorContext.assignSplit(cassandraSplit, subtaskId);
+        } else {
+            LOG.info(
+                    "No split assigned to reader {} because the enumerator has no unassigned split left. Sending NoMoreSplitsEvent to reader",
+                    subtaskId);
+            enumeratorContext.signalNoMoreSplits(subtaskId);
+        }
+    }
+
+    @Override
+    public void addSplitsBack(List<CassandraSplit> splits, int subtaskId) {
+        // splits that were assigned to a failed reader and that were not part of a checkpoint, so
+        // after
+        // restoration, they need to be reassigned.

Review Comment:
   ```suggestion
           // after restoration, they need to be reassigned.
   ```



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraEnumeratorStateSerializer.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.math.BigInteger;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+/** Serializer for {@link CassandraEnumeratorState}. */
+public class CassandraEnumeratorStateSerializer
+        implements SimpleVersionedSerializer<CassandraEnumeratorState> {
+
+    public static final CassandraEnumeratorStateSerializer INSTANCE =
+            new CassandraEnumeratorStateSerializer();
+    public static final int CURRENT_VERSION = 0;
+
+    private CassandraEnumeratorStateSerializer() {}
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(CassandraEnumeratorState cassandraEnumeratorState) throws IOException {
+        try (final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                final ObjectOutputStream objectOutputStream =
+                        new ObjectOutputStream(byteArrayOutputStream)) {
+            final Queue<CassandraSplit> splitsToReassign =
+                    cassandraEnumeratorState.getSplitsToReassign();
+            objectOutputStream.writeInt(splitsToReassign.size());
+            for (CassandraSplit cassandraSplit : splitsToReassign) {
+                final byte[] serializedSplit =
+                        CassandraSplitSerializer.INSTANCE.serialize(cassandraSplit);
+                objectOutputStream.writeInt(serializedSplit.length);
+                objectOutputStream.write(serializedSplit);
+            }
+
+            objectOutputStream.writeLong(cassandraEnumeratorState.getNumSplitsLeftToGenerate());
+
+            final byte[] increment = cassandraEnumeratorState.getIncrement().toByteArray();
+            objectOutputStream.writeInt(increment.length);
+            objectOutputStream.write(increment);
+
+            final byte[] startToken = cassandraEnumeratorState.getStartToken().toByteArray();
+            objectOutputStream.writeInt(startToken.length);
+            objectOutputStream.write(startToken);
+
+            final byte[] maxToken = cassandraEnumeratorState.getMaxToken().toByteArray();
+            objectOutputStream.writeInt(maxToken.length);
+            objectOutputStream.write(maxToken);
+
+            objectOutputStream.flush();
+            return byteArrayOutputStream.toByteArray();
+        }
+    }
+
+    @Override
+    public CassandraEnumeratorState deserialize(int version, byte[] serialized) throws IOException {
+        try (final ByteArrayInputStream byteArrayInputStream =
+                        new ByteArrayInputStream(serialized);
+                final ObjectInputStream objectInputStream =
+                        new ObjectInputStream(byteArrayInputStream)) {
+            final Queue<CassandraSplit> splitsToReassign = new ArrayDeque<>();
+            final int splitsToReassignSize = objectInputStream.readInt();
+            for (int i = 0; i < splitsToReassignSize; i++) {
+                final int splitSize = objectInputStream.readInt();
+                final byte[] splitBytes = new byte[splitSize];
+                if (objectInputStream.read(splitBytes) == -1) {
+                    throw new IOException(
+                            "EOF received while deserializing CassandraEnumeratorState.splitsToReassign");
+                }
+                final CassandraSplit split =
+                        CassandraSplitSerializer.INSTANCE.deserialize(
+                                CassandraSplitSerializer.CURRENT_VERSION, splitBytes);
+                splitsToReassign.add(split);
+            }
+
+            final long numSplitsLeftToGenerate = objectInputStream.readLong();
+
+            final int incrementSize = objectInputStream.readInt();
+            final byte[] incrementBytes = new byte[incrementSize];
+            if (objectInputStream.read(incrementBytes) == -1) {
+                throw new IOException(
+                        "EOF received while deserializing cassandraEnumeratorState.increment");
+            }
+            final BigInteger increment = new BigInteger(incrementBytes);
+
+            final int startTokenSize = objectInputStream.readInt();
+            final byte[] startTokenBytes = new byte[startTokenSize];
+            if (objectInputStream.read(startTokenBytes) == -1) {
+                throw new IOException(
+                        "EOF received while deserializing cassandraEnumeratorState.startToken");
+            }
+            final BigInteger startToken = new BigInteger(startTokenBytes);
+
+            final int maxTokenSize = objectInputStream.readInt();
+            final byte[] maxTokenBytes = new byte[maxTokenSize];
+            if (objectInputStream.read(maxTokenBytes) == -1) {
+                throw new IOException(
+                        "EOF received while deserializing cassandraEnumeratorState.maxToken");
+            }
+            final BigInteger maxToken = new BigInteger(maxTokenBytes);

Review Comment:
   Could add a utility method for reading/writing a BigInteger, avoiding the code duplication.



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This class prepares the generation of {@link CassandraSplit}s based on Cassandra cluster
+ * partitioner and cluster statistics. It estimates the total size of the table using Cassandra
+ * system table system.size_estimates.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    @VisibleForTesting public Long minSplitMemorySize = 10_000_000L; // 10 MB
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Prepare the {@param CassandraEnumeratorState} for lazy generation of {@link CassandraSplit}s:
+     * calculate {@code numSplitsToGenerate} based on estimated target table size and provided
+     * {@code maxSplitMemorySize} and calculate {@code increment} which is the size of a split in
+     * tokens.
+     */
+    public CassandraEnumeratorState prepareSplits() {
+        final long numSplitsToGenerate = decideOnNumSplits();
+        final BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplitsToGenerate)));
+        final BigInteger startToken = partitioner.minToken;
+        return new CassandraEnumeratorState(
+                numSplitsToGenerate,
+                increment,
+                startToken,
+                partitioner.maxToken,
+                new ArrayDeque<>());
+    }
+
+    /**
+     * Determine {@code numSplits} based on the estimation of the target table size and user defined
+     * {@code maxSplitMemorySize}. Provide fallbacks when table size is unavailable, too few or too
+     * many splits are calculated.
+     */
+    private long decideOnNumSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            checkState(
+                    maxSplitMemorySize >= minSplitMemorySize,
+                    "Defined maxSplitMemorySize (%s) is below minimum (%s)",
+                    maxSplitMemorySize,
+                    minSplitMemorySize);
+            final long estimateTableSize = estimateTableSize();
+            if (estimateTableSize == 0) { // size estimates unavailable
+                LOG.info(
+                        "Cassandra size estimates are not available for {}.{} table. Creating as many splits as parallelism ({})",
+                        keyspace,
+                        table,
+                        parallelism);
+                numSplits = parallelism;
+            } else {
+                LOG.debug(
+                        "Estimated size for {}.{} table is {} bytes",
+                        keyspace,
+                        table,
+                        estimateTableSize);
+                numSplits =
+                        estimateTableSize / maxSplitMemorySize == 0
+                                ? parallelism // estimateTableSize under sizes maxSplitMemorySize
+                                // creating as many splits as parallelism

Review Comment:
   This comment line is a bit confusing. Placement wise it could apply to both branches, and it seems to obvious for the `?` branch?
   ```suggestion
   ```



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This class prepares the generation of {@link CassandraSplit}s based on Cassandra cluster
+ * partitioner and cluster statistics. It estimates the total size of the table using Cassandra
+ * system table system.size_estimates.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    @VisibleForTesting public Long minSplitMemorySize = 10_000_000L; // 10 MB

Review Comment:
   ```suggestion
       @VisibleForTesting public Long minSplitMemorySize = MemorySize.ofMebiBytes(10).getBytes();
   ```



##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/split/SplitsGenerator.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.flink.connector.cassandra.source.split;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.cassandra.source.enumerator.CassandraEnumeratorState;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This class prepares the generation of {@link CassandraSplit}s based on Cassandra cluster
+ * partitioner and cluster statistics. It estimates the total size of the table using Cassandra
+ * system table system.size_estimates.
+ */
+public final class SplitsGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SplitsGenerator.class);
+    @VisibleForTesting public Long minSplitMemorySize = 10_000_000L; // 10 MB
+
+    private final CassandraPartitioner partitioner;
+    private final Session session;
+    private final String keyspace;
+    private final String table;
+    private final int parallelism;
+    @Nullable private final Long maxSplitMemorySize;
+
+    public SplitsGenerator(
+            CassandraPartitioner partitioner,
+            Session session,
+            String keyspace,
+            String table,
+            int parallelism,
+            Long maxSplitMemorySize) {
+        this.partitioner = partitioner;
+        this.session = session;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.parallelism = parallelism;
+        this.maxSplitMemorySize = maxSplitMemorySize;
+    }
+
+    /**
+     * Prepare the {@param CassandraEnumeratorState} for lazy generation of {@link CassandraSplit}s:
+     * calculate {@code numSplitsToGenerate} based on estimated target table size and provided
+     * {@code maxSplitMemorySize} and calculate {@code increment} which is the size of a split in
+     * tokens.
+     */
+    public CassandraEnumeratorState prepareSplits() {
+        final long numSplitsToGenerate = decideOnNumSplits();
+        final BigInteger increment =
+                (partitioner.ringSize).divide(new BigInteger(String.valueOf(numSplitsToGenerate)));
+        final BigInteger startToken = partitioner.minToken;
+        return new CassandraEnumeratorState(
+                numSplitsToGenerate,
+                increment,
+                startToken,
+                partitioner.maxToken,
+                new ArrayDeque<>());
+    }
+
+    /**
+     * Determine {@code numSplits} based on the estimation of the target table size and user defined
+     * {@code maxSplitMemorySize}. Provide fallbacks when table size is unavailable, too few or too
+     * many splits are calculated.
+     */
+    private long decideOnNumSplits() {
+        long numSplits;
+        if (maxSplitMemorySize != null) {
+            checkState(
+                    maxSplitMemorySize >= minSplitMemorySize,
+                    "Defined maxSplitMemorySize (%s) is below minimum (%s)",
+                    maxSplitMemorySize,
+                    minSplitMemorySize);
+            final long estimateTableSize = estimateTableSize();
+            if (estimateTableSize == 0) { // size estimates unavailable
+                LOG.info(
+                        "Cassandra size estimates are not available for {}.{} table. Creating as many splits as parallelism ({})",
+                        keyspace,
+                        table,
+                        parallelism);
+                numSplits = parallelism;
+            } else {
+                LOG.debug(
+                        "Estimated size for {}.{} table is {} bytes",
+                        keyspace,
+                        table,
+                        estimateTableSize);
+                numSplits =
+                        estimateTableSize / maxSplitMemorySize == 0
+                                ? parallelism // estimateTableSize under sizes maxSplitMemorySize
+                                // creating as many splits as parallelism
+                                : estimateTableSize / maxSplitMemorySize;
+                LOG.info(
+                        "maxSplitMemorySize set value ({}) leads to the creation of {} splits",
+                        maxSplitMemorySize,
+                        numSplits);
+            }
+        } else { // maxSplitMemorySize not defined

Review Comment:
   should we maybe have a default maximum? 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#issuecomment-1468398944

   > (Note: This review is specifically about the JMX parts; nothing else)
   
   OK. Tell me if you have comments on the size-based split.
   


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138422310


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        checkReaderRegistered(subtaskId);
+        final CassandraSplit cassandraSplit = state.getASplit();
+        if (cassandraSplit != null) {
+            LOG.info("Assigning splits to reader {}", subtaskId);
+            enumeratorContext.assignSplit(cassandraSplit, subtaskId);
+        } else {
+            LOG.info(
+                    "No split assigned to reader {} because the enumerator has no unassigned split left",
+                    subtaskId);
+        }
+        if (!state.hasMoreSplits()) {
+            LOG.info(
+                    "No more CassandraSplits to assign. Sending NoMoreSplitsEvent to reader {}.",
+                    subtaskId);
+            enumeratorContext.signalNoMoreSplits(subtaskId);
+        }
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits);
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitionerName = clusterMetadata.getPartitioner();
+        final SplitsGenerator.CassandraPartitioner partitioner =
+                partitionerName.contains(MURMUR3PARTITIONER.className())
+                        ? MURMUR3PARTITIONER
+                        : RANDOMPARTITIONER;
+        return new SplitsGenerator(partitioner).generateSplits(numberOfSplits);

Review Comment:
   cf https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138381568



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1137158043


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);

Review Comment:
   :+1: 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] echauchot commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "echauchot (via GitHub)" <gi...@apache.org>.
echauchot commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1138327313


##########
flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/source/CassandraSourceITCase.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.flink.connector.cassandra.source;
+
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.connector.testframe.environment.ClusterControllable;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
+import org.apache.flink.connector.testframe.utils.CollectIteratorAssertions;
+import org.apache.flink.connectors.cassandra.utils.Pojo;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+
+import java.util.List;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.flink.connector.cassandra.source.CassandraTestContext.CassandraTestContextFactory;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for the Cassandra source. */
+class CassandraSourceITCase extends SourceTestSuiteBase<Pojo> {
+
+    @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment();
+
+    @TestExternalSystem
+    CassandraTestEnvironment cassandraTestEnvironment = new CassandraTestEnvironment();
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    @TestContext
+    CassandraTestContextFactory contextFactory =
+            new CassandraTestContextFactory(cassandraTestEnvironment);
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with MURMUR3PARTITIONER (default Cassandra partitioner)")
+    public void testGenerateSplitsMurMur3Partitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(-9223372036854775808,0)");
+        assertThat(splits.get(1).splitId()).isEqualTo("(0,9223372036854775807)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test basic splitting with RANDOMPARTITIONER")
+    public void testGenerateSplitsRandomPartitioner(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic) {
+        final int parallelism = 2;
+        final SplitsGenerator generator =
+                new SplitsGenerator(
+                        RANDOMPARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        null);
+        List<CassandraSplit> splits = generator.generateSplits();
+
+        // no maxSplitMemorySize specified falling back number of splits = parallelism
+        assertThat(splits.size()).isEqualTo(parallelism);
+        assertThat(splits.get(0).splitId()).isEqualTo("(0,85070591730234615865843651857942052864)");
+        assertThat(splits.get(1).splitId())
+                .isEqualTo(
+                        "(85070591730234615865843651857942052864,170141183460469231731687303715884105727)");
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a correct split size set")
+    public void testGenerateSplitsWithCorrectSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 2;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        10000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // nb splits = tableSize / maxSplitMemorySize
+        assertThat(splits.size()).isEqualTo(3);
+    }
+
+    @TestTemplate
+    @DisplayName("Test splitting with a too big split size set")
+    public void testGenerateSplitsWithTooBigSize(
+            TestEnvironment testEnv,
+            DataStreamSourceExternalContext<Pojo> externalContext,
+            CheckpointingMode semantic)
+            throws Exception {
+        final int parallelism = 20;
+        SplitsGenerator generator =
+                new SplitsGenerator(
+                        MURMUR3PARTITIONER,
+                        cassandraTestEnvironment.getSession(),
+                        CassandraTestEnvironment.KEYSPACE,
+                        CassandraTestEnvironment.SPLITS_TABLE,
+                        parallelism,
+                        100_000_000L);
+        assertThat(generator.estimateTableSize()).isEqualTo(35840L);
+        List<CassandraSplit> splits = generator.generateSplits();
+        // tableSize / maxSplitMemorySize is too little compared to parallelism falling back to
+        // number of splits = parallelism

Review Comment:
   fair enough



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-cassandra] zentol commented on a diff in pull request #3: [FLINK-26822] Add Cassandra Source

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #3:
URL: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1135506290


##########
flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/source/enumerator/CassandraSplitEnumerator.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.flink.connector.cassandra.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.cassandra.source.split.CassandraSplit;
+import org.apache.flink.connector.cassandra.source.split.SplitsGenerator;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Metadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.MURMUR3PARTITIONER;
+import static org.apache.flink.connector.cassandra.source.split.SplitsGenerator.CassandraPartitioner.RANDOMPARTITIONER;
+
+/** {@link SplitEnumerator} that splits Cassandra cluster into {@link CassandraSplit}s. */
+public final class CassandraSplitEnumerator
+        implements SplitEnumerator<CassandraSplit, CassandraEnumeratorState> {
+    private static final Logger LOG = LoggerFactory.getLogger(CassandraSplitEnumerator.class);
+
+    private final SplitEnumeratorContext<CassandraSplit> enumeratorContext;
+    private final CassandraEnumeratorState state;
+    private final Cluster cluster;
+
+    public CassandraSplitEnumerator(
+            SplitEnumeratorContext<CassandraSplit> enumeratorContext,
+            CassandraEnumeratorState state,
+            ClusterBuilder clusterBuilder) {
+        this.enumeratorContext = enumeratorContext;
+        this.state = state == null ? new CassandraEnumeratorState() : state /* snapshot restore*/;
+        this.cluster = clusterBuilder.getCluster();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        checkReaderRegistered(subtaskId);
+        final CassandraSplit cassandraSplit = state.getASplit();
+        if (cassandraSplit != null) {
+            LOG.info("Assigning splits to reader {}", subtaskId);
+            enumeratorContext.assignSplit(cassandraSplit, subtaskId);
+        } else {
+            LOG.info(
+                    "No split assigned to reader {} because the enumerator has no unassigned split left",
+                    subtaskId);
+        }
+        if (!state.hasMoreSplits()) {
+            LOG.info(
+                    "No more CassandraSplits to assign. Sending NoMoreSplitsEvent to reader {}.",
+                    subtaskId);
+            enumeratorContext.signalNoMoreSplits(subtaskId);
+        }
+    }
+
+    @Override
+    public void start() {
+        // discover the splits and update unprocessed splits and then assign them.
+        // There is only an initial splits discovery, no periodic discovery.
+        enumeratorContext.callAsync(
+                this::discoverSplits,
+                (splits, throwable) -> {
+                    LOG.info("Add {} splits to CassandraSplitEnumerator.", splits.size());
+                    state.addNewSplits(splits);
+                });
+    }
+
+    private List<CassandraSplit> discoverSplits() {
+        final int numberOfSplits = enumeratorContext.currentParallelism();
+        final Metadata clusterMetadata = cluster.getMetadata();
+        final String partitionerName = clusterMetadata.getPartitioner();
+        final SplitsGenerator.CassandraPartitioner partitioner =
+                partitionerName.contains(MURMUR3PARTITIONER.className())
+                        ? MURMUR3PARTITIONER
+                        : RANDOMPARTITIONER;
+        return new SplitsGenerator(partitioner).generateSplits(numberOfSplits);

Review Comment:
   Related to: https://github.com/apache/flink-connector-cassandra/pull/3#discussion_r1136940960
   
   Looking at SplitsGenerator#generateaSplits it should be straight-forward to lazily generate splits, no?
   The start-/endToken, increment and numSplits would be your state.
   
   This seems safer than to add safeguards for split counts, which'd be difficult to asses imo as to how effective/dangerous they are.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org