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/06/13 13:03:22 UTC

[GitHub] [flink] dianfu commented on a diff in pull request #19945: [FLINK-27964][python] Support Cassandra connector in Python DataStream API

dianfu commented on code in PR #19945:
URL: https://github.com/apache/flink/pull/19945#discussion_r895628117


##########
docs/content.zh/docs/connectors/datastream/cassandra.md:
##########
@@ -195,6 +195,28 @@ CassandraSink.addSink(result)
 result.print().setParallelism(1)
 ```
 {{< /tab >}}
+{{< tab "Python" >}}
+```python
+# get the execution environment
+env = StreamExecutionEnvironment.get_execution_environment()
+
+# get input data by connecting to the socket
+text = DataStream(env._j_stream_execution_environment.socketTextStream(hostname, port, "\n"))

Review Comment:
   It would be great to introduce `socketTextStream` in PyFlink. We could do it in a separate PR.



##########
docs/content.zh/docs/connectors/datastream/cassandra.md:
##########
@@ -195,6 +195,28 @@ CassandraSink.addSink(result)
 result.print().setParallelism(1)
 ```
 {{< /tab >}}
+{{< tab "Python" >}}
+```python
+# get the execution environment
+env = StreamExecutionEnvironment.get_execution_environment()
+
+# get input data by connecting to the socket
+text = DataStream(env._j_stream_execution_environment.socketTextStream(hostname, port, "\n"))
+
+# parse the data, group it, window it, and aggregate the counts
+result = text \
+    .flat_map(lambda x: x.lower.split("\\s")) \

Review Comment:
   .filter(lambda x: len(x) > 0)



##########
flink-python/pyflink/datastream/connectors/cassandra.py:
##########
@@ -0,0 +1,253 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+from pyflink.common import Duration
+from pyflink.datastream import DataStream
+from pyflink.datastream.connectors import Sink
+from pyflink.java_gateway import get_gateway
+
+
+class MapperOptions(object):
+    """
+    This class is used to configure a Mapper after deployment.
+    """
+    def __init__(self, j_mapper_options):
+        self._j_mapper_options = j_mapper_options
+
+
+class CassandraFailureHandler(object):
+    """
+    Handle a failed Throwable.
+    """
+    def __init__(self, j_cassandra_failure_handler):
+        self._j_cassandra_failure_handler = j_cassandra_failure_handler
+
+
+class ClusterBuilder(object):
+    """
+    This class is used to configure a Cluster after deployment. The cluster represents the
+    connection that will be established to Cassandra.
+    """
+    def __init__(self, j_cluster_builder):
+        self._j_cluster_builder = j_cluster_builder
+
+
+class CheckpointCommitter(object):
+    """
+    CheckpointCommitter that saves information about completed checkpoints within a separate table
+    in a cassandra database.
+    """
+    def __init__(self, j_checkpoint_committer):
+        self._j_checkpoint_committer = j_checkpoint_committer
+
+    @staticmethod
+    def cassandra_committer(builder: ClusterBuilder, key_space: str = None) \

Review Comment:
   If we rename CheckpointCommitter to CassandraCommitter, then we could rename this method to default_checkpoint_committer



##########
docs/content.zh/docs/connectors/datastream/cassandra.md:
##########
@@ -283,6 +305,51 @@ public class WordCount {
 }
 ```
 {{< /tab >}}
+{{< tab "Python" >}}
+```python
+# get the execution environment
+env = StreamExecutionEnvironment.get_execution_environment()
+
+# get input data by connecting to the socket
+text = DataStream(env._j_stream_execution_environment.socketTextStream(hostname, port, "\n"))
+
+# parse the data, group it, window it, and aggregate the counts
+result = text \
+    .flat_map(lambda x: x.lower.split("\\s")) \
+    .map(lambda x: (x, 1), output_type=Types.TUPLE([WordCount, Types.INT()])) \
+    .key_by(lambda x: x.get_word()) \
+    .window(TumblingProcessingTimeWindows.of(Time.seconds(5))) \
+    .reduce(lambda a, b: WordCount(a.get_word(), a.get_count() + b.get_count()))
+
+CassandraSink.add_sink(result) \
+    .set_host("127.0.0.1") \
+    .set_mapper_options(MapperOptions(JMyMapperOptions)) \

Review Comment:
   This is not executable. Could you make it more concrete?



##########
flink-python/pyflink/datastream/connectors/cassandra.py:
##########
@@ -0,0 +1,253 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+from pyflink.common import Duration
+from pyflink.datastream import DataStream
+from pyflink.datastream.connectors import Sink
+from pyflink.java_gateway import get_gateway
+
+
+class MapperOptions(object):
+    """
+    This class is used to configure a Mapper after deployment.
+    """
+    def __init__(self, j_mapper_options):
+        self._j_mapper_options = j_mapper_options
+
+
+class CassandraFailureHandler(object):
+    """
+    Handle a failed Throwable.
+    """
+    def __init__(self, j_cassandra_failure_handler):
+        self._j_cassandra_failure_handler = j_cassandra_failure_handler
+
+
+class ClusterBuilder(object):
+    """
+    This class is used to configure a Cluster after deployment. The cluster represents the
+    connection that will be established to Cassandra.
+    """
+    def __init__(self, j_cluster_builder):
+        self._j_cluster_builder = j_cluster_builder
+
+
+class CheckpointCommitter(object):
+    """
+    CheckpointCommitter that saves information about completed checkpoints within a separate table
+    in a cassandra database.
+    """
+    def __init__(self, j_checkpoint_committer):
+        self._j_checkpoint_committer = j_checkpoint_committer
+
+    @staticmethod
+    def cassandra_committer(builder: ClusterBuilder, key_space: str = None) \
+            -> 'CheckpointCommitter':
+        """
+        CheckpointCommitter that saves information about completed checkpoints within a separate
+        table in a cassandra database.
+        """
+        if key_space is None:
+            j_checkpoint_committer = get_gateway().jvm.org.apache.flink.streaming. \
+                connectors.cassandra.CassandraCommitter(builder._j_cluster_builder)
+        else:
+            j_checkpoint_committer = get_gateway().jvm.org.apache.flink.streaming. \
+                connectors.cassandra.CassandraCommitter(builder._j_cluster_builder, key_space)
+        return CheckpointCommitter(j_checkpoint_committer)
+
+
+class CassandraSink(Sink):

Review Comment:
   ```suggestion
   class CassandraSink(object):
   ```
   
   CassandraSink could not extends from Sink directly as it doesn't support `datastream.sink_to(cassandra_sink)`



##########
flink-python/pyflink/datastream/connectors/cassandra.py:
##########
@@ -0,0 +1,253 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+from pyflink.common import Duration
+from pyflink.datastream import DataStream
+from pyflink.datastream.connectors import Sink
+from pyflink.java_gateway import get_gateway
+
+
+class MapperOptions(object):
+    """
+    This class is used to configure a Mapper after deployment.
+    """
+    def __init__(self, j_mapper_options):
+        self._j_mapper_options = j_mapper_options
+
+
+class CassandraFailureHandler(object):
+    """
+    Handle a failed Throwable.
+    """
+    def __init__(self, j_cassandra_failure_handler):
+        self._j_cassandra_failure_handler = j_cassandra_failure_handler
+
+
+class ClusterBuilder(object):
+    """
+    This class is used to configure a Cluster after deployment. The cluster represents the
+    connection that will be established to Cassandra.
+    """
+    def __init__(self, j_cluster_builder):
+        self._j_cluster_builder = j_cluster_builder
+
+
+class CheckpointCommitter(object):

Review Comment:
   I guess we could rename it to CassandraCommitter



##########
flink-python/pyflink/datastream/connectors/cassandra.py:
##########
@@ -0,0 +1,253 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+from pyflink.common import Duration
+from pyflink.datastream import DataStream
+from pyflink.datastream.connectors import Sink
+from pyflink.java_gateway import get_gateway
+
+
+class MapperOptions(object):
+    """
+    This class is used to configure a Mapper after deployment.
+    """
+    def __init__(self, j_mapper_options):
+        self._j_mapper_options = j_mapper_options

Review Comment:
   Could we introduce a builder and add the following methods insider the builder to make this easy to use for Python users:
   ```
   def ttl(ttl: int):
   
   def timerstamp(timestamp: int):
   
   def consistency_level(xxx)
   
   def tracing(enabled: bool):
   
   def save_null_fields(enabled: bool):
   ```



##########
flink-python/pyflink/datastream/connectors/cassandra.py:
##########
@@ -0,0 +1,253 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+from pyflink.common import Duration
+from pyflink.datastream import DataStream
+from pyflink.datastream.connectors import Sink
+from pyflink.java_gateway import get_gateway
+
+
+class MapperOptions(object):
+    """
+    This class is used to configure a Mapper after deployment.
+    """
+    def __init__(self, j_mapper_options):
+        self._j_mapper_options = j_mapper_options
+
+
+class CassandraFailureHandler(object):
+    """
+    Handle a failed Throwable.
+    """
+    def __init__(self, j_cassandra_failure_handler):
+        self._j_cassandra_failure_handler = j_cassandra_failure_handler

Review Comment:
   Add a default method which returns NoOpCassandraFailureHandler:
   ```
   def default_failure_handler()
   ```



##########
flink-python/pyflink/datastream/connectors/cassandra.py:
##########
@@ -0,0 +1,253 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+from pyflink.common import Duration
+from pyflink.datastream import DataStream
+from pyflink.datastream.connectors import Sink
+from pyflink.java_gateway import get_gateway
+
+
+class MapperOptions(object):
+    """
+    This class is used to configure a Mapper after deployment.
+    """
+    def __init__(self, j_mapper_options):
+        self._j_mapper_options = j_mapper_options
+
+
+class CassandraFailureHandler(object):
+    """
+    Handle a failed Throwable.
+    """
+    def __init__(self, j_cassandra_failure_handler):
+        self._j_cassandra_failure_handler = j_cassandra_failure_handler
+
+
+class ClusterBuilder(object):
+    """
+    This class is used to configure a Cluster after deployment. The cluster represents the
+    connection that will be established to Cassandra.
+    """
+    def __init__(self, j_cluster_builder):
+        self._j_cluster_builder = j_cluster_builder
+

Review Comment:
   Add a builder with the following methods to make it more usable for Python users:
   ```
   with_cluster_name、with_port、with_port、with_max_schema_agreement_wait_seconds、with_protocol_version, add_contact_point, add_contact_points, with_load_balancing_policy, with_reconnection_policy, with_retry_policy,  etc.
   ```
   
   See the Java Cluster.Builder defined in cassandra for more details.



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