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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to