zchovan commented on code in PR #8:
URL: 
https://github.com/apache/flink-connector-kudu/pull/8#discussion_r1967293096


##########
flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/source/enumerator/KuduSourceEnumeratorStateSerializer.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kudu.source.enumerator;
+
+import org.apache.flink.connector.kudu.source.split.KuduSourceSplit;
+import org.apache.flink.connector.kudu.source.split.KuduSourceSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/** The class that serializes and deserializes {@link 
KuduSourceEnumeratorState}. */
+public class KuduSourceEnumeratorStateSerializer
+        implements SimpleVersionedSerializer<KuduSourceEnumeratorState> {
+
+    private static final int VERSION = 1;
+    private final KuduSourceSplitSerializer splitSerializer = new 
KuduSourceSplitSerializer();
+
+    @Override
+    public int getVersion() {
+        return VERSION;
+    }
+
+    @Override
+    public byte[] serialize(KuduSourceEnumeratorState state) throws 
IOException {
+        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+                DataOutputStream out = new DataOutputStream(baos)) {
+
+            out.writeLong(state.getLastEndTimestamp());
+            serializeSplits(out, state.getUnassigned());
+            serializeSplits(out, state.getPending());
+
+            return baos.toByteArray();
+        }
+    }
+
+    @Override
+    public KuduSourceEnumeratorState deserialize(int version, byte[] 
serialized)
+            throws IOException {
+        if (version != VERSION) {
+            throw new IOException("Unsupported version: " + version);

Review Comment:
   is IOException the best fit for this?



##########
flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/source/enumerator/KuduSourceEnumeratorTest.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kudu.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.kudu.connector.KuduTableInfo;
+import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig;
+import org.apache.flink.connector.kudu.source.split.KuduSourceSplit;
+import 
org.apache.flink.connector.testutils.source.reader.TestingSplitEnumeratorContext;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+// Basic test to verify that when a registered reader requests a split:
+// - An unassigned split moves to the enumerator context's split assignment.
+// - The enumerator tracks this split by adding it to the pending list.

Review Comment:
   comment is duplicated



##########
flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/source/enumerator/KuduSourceEnumeratorStateSerializerTest.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kudu.source.enumerator;
+
+import org.apache.flink.connector.kudu.source.split.KuduSourceSplit;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
+/** Tests for {@link KuduSourceEnumeratorStateSerializer}. */
+public class KuduSourceEnumeratorStateSerializerTest {
+    private final KuduSourceEnumeratorStateSerializer serializer =
+            new KuduSourceEnumeratorStateSerializer();
+
+    @Test
+    public void testSerializeDeserialize() throws IOException {
+        byte[] token1 = {1, 2, 3};
+        byte[] token2 = {4, 5, 6};
+        byte[] token3 = {7, 8, 9};
+        byte[] token4 = {10, 11, 12};
+
+        List<KuduSourceSplit> unassigned =
+                Arrays.asList(new KuduSourceSplit(token1), new 
KuduSourceSplit(token2));
+        List<KuduSourceSplit> pending =
+                Arrays.asList(new KuduSourceSplit(token3), new 
KuduSourceSplit(token4));
+        KuduSourceEnumeratorState state =
+                new KuduSourceEnumeratorState(12345L, unassigned, pending);
+
+        byte[] serialized = serializer.serialize(state);
+        KuduSourceEnumeratorState deserialized =
+                serializer.deserialize(serializer.getVersion(), serialized);
+
+        
assertThat(state.getLastEndTimestamp()).isEqualTo(deserialized.getLastEndTimestamp());
+        
assertThat(state.getUnassigned().size()).isEqualTo(deserialized.getUnassigned().size());
+        
assertThat(state.getPending().size()).isEqualTo(deserialized.getPending().size());
+
+        for (int i = 0; i < unassigned.size(); i++) {

Review Comment:
   change this to for-each? e.g. for(KuduSourceSplit split : unassigned) {}
   same for the next for block



##########
flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/source/split/KuduSourceSplitSerializer.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.kudu.source.split;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.IOException;
+
+/** The class that serializes and deserializes {@link KuduSourceSplit}. */
+public class KuduSourceSplitSerializer implements 
SimpleVersionedSerializer<KuduSourceSplit> {
+
+    private static final int VERSION = 1; // Versioning for future changes
+
+    @Override
+    public int getVersion() {
+        return VERSION;
+    }
+
+    @Override
+    public byte[] serialize(KuduSourceSplit obj) throws IOException {
+        if (obj == null || obj.getSerializedScanToken() == null) {
+            throw new IOException("KuduSourceSplit or serializedScanToken is 
null.");
+        }
+
+        return obj.getSerializedScanToken(); // Directly return the byte array
+    }
+
+    @Override
+    public KuduSourceSplit deserialize(int version, byte[] serialized) throws 
IOException {
+        if (version != VERSION) {
+            throw new IOException("Unsupported version: " + version);

Review Comment:
   is IOException the best fit for this?



##########
flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/source/enumerator/KuduSourceEnumeratorStateSerializerTest.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kudu.source.enumerator;
+
+import org.apache.flink.connector.kudu.source.split.KuduSourceSplit;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
+/** Tests for {@link KuduSourceEnumeratorStateSerializer}. */
+public class KuduSourceEnumeratorStateSerializerTest {
+    private final KuduSourceEnumeratorStateSerializer serializer =
+            new KuduSourceEnumeratorStateSerializer();
+
+    @Test
+    public void testSerializeDeserialize() throws IOException {
+        byte[] token1 = {1, 2, 3};
+        byte[] token2 = {4, 5, 6};
+        byte[] token3 = {7, 8, 9};
+        byte[] token4 = {10, 11, 12};
+
+        List<KuduSourceSplit> unassigned =
+                Arrays.asList(new KuduSourceSplit(token1), new 
KuduSourceSplit(token2));
+        List<KuduSourceSplit> pending =
+                Arrays.asList(new KuduSourceSplit(token3), new 
KuduSourceSplit(token4));
+        KuduSourceEnumeratorState state =
+                new KuduSourceEnumeratorState(12345L, unassigned, pending);
+
+        byte[] serialized = serializer.serialize(state);
+        KuduSourceEnumeratorState deserialized =
+                serializer.deserialize(serializer.getVersion(), serialized);
+
+        
assertThat(state.getLastEndTimestamp()).isEqualTo(deserialized.getLastEndTimestamp());
+        
assertThat(state.getUnassigned().size()).isEqualTo(deserialized.getUnassigned().size());
+        
assertThat(state.getPending().size()).isEqualTo(deserialized.getPending().size());
+
+        for (int i = 0; i < unassigned.size(); i++) {

Review Comment:
   replace with for-each? eg. for(KuduSourceSplit split : unassigned) {} 
   same for the next for block



##########
flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/source/enumerator/KuduSourceEnumeratorTest.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kudu.source.enumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.kudu.connector.KuduTableInfo;
+import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig;
+import org.apache.flink.connector.kudu.source.split.KuduSourceSplit;
+import 
org.apache.flink.connector.testutils.source.reader.TestingSplitEnumeratorContext;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+// Basic test to verify that when a registered reader requests a split:

Review Comment:
   comment is duplicated



##########
flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/source/enumerator/KuduSplitGenerator.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.kudu.source.enumerator;
+
+import org.apache.flink.connector.kudu.connector.KuduTableInfo;
+import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig;
+import org.apache.flink.connector.kudu.source.split.KuduSourceSplit;
+
+import org.apache.kudu.client.AsyncKuduScanner;
+import org.apache.kudu.client.KuduClient;
+import org.apache.kudu.client.KuduScanToken;
+import org.apache.kudu.client.KuduTable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The class responsible for producing scan tokens for given timestamps and 
returning them in the
+ * form of {@link KuduSourceSplit}.
+ */
+public class KuduSplitGenerator {
+    private final KuduTableInfo tableInfo;
+    private final KuduClient kuduClient;
+
+    public KuduSplitGenerator(KuduReaderConfig readerConfig, KuduTableInfo 
tableInfo) {
+        this.tableInfo = tableInfo;
+        this.kuduClient = new 
KuduClient.KuduClientBuilder(readerConfig.getMasters()).build();
+    }
+
+    public List<KuduSourceSplit> generateFullScanSplits(long 
snapshotTimestamp) {
+        try {
+            KuduTable table = kuduClient.openTable(tableInfo.getName());
+            List<KuduScanToken> tokens =
+                    kuduClient
+                            .newScanTokenBuilder(table)
+                            .snapshotTimestampRaw(snapshotTimestamp)
+                            
.readMode(AsyncKuduScanner.ReadMode.READ_AT_SNAPSHOT)
+                            .build();
+
+            return serializeTokens(tokens);
+        } catch (Exception e) {
+            throw new RuntimeException("Error during full snapshot scan", e);

Review Comment:
   is there a way to extract more information out of this exception? In the 
related tests every scenario tests for the general error message "Error during 
full snapshot scan", would it be possible to change this to be a bit more 
user-friendly?



-- 
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