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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
