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 2019/05/21 06:11:15 UTC

[GitHub] [flink] dianfu commented on a change in pull request #8488: [FLINK-12308][python] Add FileSystem Connector with CSV format support in Python Table API

dianfu commented on a change in pull request #8488: [FLINK-12308][python] Add FileSystem Connector with CSV format support in Python Table API
URL: https://github.com/apache/flink/pull/8488#discussion_r285859502
 
 

 ##########
 File path: flink-python/pyflink/table/table_descriptor.py
 ##########
 @@ -0,0 +1,546 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+import sys
+from abc import ABCMeta
+
+from py4j.java_gateway import get_method
+
+from pyflink.java_gateway import get_gateway
+from pyflink.util.type_utils import to_java_type
+
+if sys.version >= '3':
+    unicode = str
+
+
+class Descriptor(object):
+    """
+    Base class of the descriptors that adds a set of string-based, normalized properties for
+    describing DDL information.
+
+    Typical characteristics of a descriptor are:
+    - descriptors have a default constructor
+    - descriptors themselves contain very little logic
+    - corresponding validators validate the correctness (goal: have a single point of validation)
+
+    A descriptor is similar to a builder in a builder pattern, thus, mutable for building
+    properties.
+    """
+
+    __metaclass__ = ABCMeta
+
+    def __init__(self, j_descriptor):
+        self._j_descriptor = j_descriptor
+
+    def to_properties(self):
+        """
+        Converts this descriptor into a dict of properties.
+
+        :return: Dict object contains all of current properties.
+        """
+        return dict(self._j_descriptor.toProperties())
+
+
+class WatermarkStrategy(Descriptor):
+    """
+    Provides a strategy to generate watermarks for a rowtime attribute.
+    """
+
+    __metaclass__ = ABCMeta
+
+    def __init__(self, j_strategy):
+        self._j_strategy = j_strategy
+        super(WatermarkStrategy, self).__init__(self._j_strategy)
+
+
+class PreserveWatermarks(WatermarkStrategy):
+    """
+    A strategy which indicates the watermarks should be preserved from the underlying datastream.
+    """
+
+    def __init__(self):
+        gateway = get_gateway()
+        super(PreserveWatermarks, self).__init__(gateway.jvm.PreserveWatermarks())
+
+
+class AscendingTimestamps(WatermarkStrategy):
+    """
+    A watermark strategy for ascending rowtime attributes.
+
+    Emits a watermark of the maximum observed timestamp so far minus 1. Rows that have a timestamp
+    equal to the max timestamp are not late.
+    """
+
+    def __init__(self):
+        gateway = get_gateway()
+        super(AscendingTimestamps, self).__init__(gateway.jvm.AscendingTimestamps())
+
+
+class BoundedOutOfOrderTimestamps(WatermarkStrategy):
+    """
+    A watermark strategy for rowtime attributes which are out-of-order by a bounded time interval.
+
+    Emits watermarks which are the maximum observed timestamp minus the specified delay.
+    """
+
+    def __init__(self, delay):
+        """
+        :param delay: The delay in milliseconds by which watermarks are behind the maximum observed
+                      timestamp.
+        """
+        gateway = get_gateway()
+        super(BoundedOutOfOrderTimestamps, self).__init__(
+            gateway.jvm.BoundedOutOfOrderTimestamps(delay))
+
+
+class Rowtime(Descriptor):
+    """
+    Rowtime descriptor for describing an event time attribute in the schema.
+    """
+
+    def __init__(self):
+        gateway = get_gateway()
+        self._j_rowtime = gateway.jvm.Rowtime()
+        super(Rowtime, self).__init__(self._j_rowtime)
+
+    def timestamps_from_field(self, field_name):
+        """
+        Sets a built-in timestamp extractor that converts an existing LONG or TIMESTAMP field into
+        the rowtime attribute.
+
+        :param field_name: The field to convert into a rowtime attribute.
+        :return: This rowtime descriptor.
+        """
+        self._j_rowtime = self._j_rowtime.timestampsFromField(field_name)
+        return self
+
+    def timestamps_from_source(self):
+        """
+        Sets a built-in timestamp extractor that converts the assigned timestamps from a DataStream
+        API record into the rowtime attribute and thus preserves the assigned timestamps from the
+        source.
+
+        ..note::
+            This extractor only works in streaming environments.
+
+        :return: This rowtime descriptor.
+        """
+        self._j_rowtime = self._j_rowtime.timestampsFromSource()
+        return self
+
+    def timestamps_from_extractor(self, extractor):
+        """
+        Sets a custom timestamp extractor to be used for the rowtime attribute.
+
+        :param extractor: The java canonical class name of the TimestampExtractor to extract the
+                          rowtime attribute from the physical type. The TimestampExtractor must
+                          have a public no-argument constructor and can be founded by
+                          ``Class.forName`` in current Java classpath.
+        :return: This rowtime descriptor.
+        """
+        gateway = get_gateway()
+        self._j_rowtime = self._j_rowtime.timestampsFromExtractor(
+            gateway.jvm.Class.forName(extractor).newInstance())
+        return self
+
+    def watermarks_periodic_ascending(self):
+        """
+        Sets a built-in watermark strategy for ascending rowtime attributes.
+
+        Emits a watermark of the maximum observed timestamp so far minus 1. Rows that have a
+        timestamp equal to the max timestamp are not late.
+
+        :return: This rowtime descriptor.
+        """
+        self._j_rowtime = self._j_rowtime.watermarksPeriodicAscending()
+        return self
+
+    def watermarks_periodic_bounded(self, delay):
+        """
+        Sets a built-in watermark strategy for rowtime attributes which are out-of-order by a
+        bounded time interval.
+
+        Emits watermarks which are the maximum observed timestamp minus the specified delay.
+
+        :param delay: Delay in milliseconds.
+        :return: This rowtime descriptor.
+        """
+        self._j_rowtime = self._j_rowtime.watermarksPeriodicBounded(delay)
+        return self
+
+    def watermarks_from_source(self):
+        """
+        Sets a built-in watermark strategy which indicates the watermarks should be preserved from
+        the underlying DataStream API and thus preserves the assigned watermarks from the source.
+
+        :return: This rowtime descriptor.
+        """
+        self._j_rowtime = self._j_rowtime.watermarksFromSource()
+        return self
+
+    def watermarks_from_strategy(self, strategy):
+        """
+        Sets a custom watermark strategy to be used for the rowtime attribute.
+
+        :param strategy: Watermark strategy object.
+        :return: This rowtime descriptor.
+        """
+        self._j_rowtime = self._j_rowtime.watermarksFromStrategy(strategy._j_strategy)
+        return self
+
+
+class Schema(Descriptor):
+    """
+    Describes a schema of a table.
+
+    ..note::
+        Field names are matched by the exact name by default (case sensitive).
+    """
+
+    def __init__(self):
+        gateway = get_gateway()
+        self._j_schema = gateway.jvm.Schema()
+        super(Schema, self).__init__(self._j_schema)
+
+    def field(self, field_name, field_type):
+        """
+        Adds a field with the field name and the data type or type string. Required.
+        This method can be called multiple times. The call order of this method defines
+        also the order of the fields in a row. Here is a document that introduces the type strings:
+        https://ci.apache.org/projects/flink/flink-docs-master/dev/table/connect.html#type-strings
+
+        :param field_name: The field name.
+        :param field_type: The data type or type string of the field.
+        :return: This schema object.
+        """
+        if isinstance(field_type, str) or isinstance(field_type, unicode):
+            self._j_schema = self._j_schema.field(field_name, field_type)
+        else:
+            self._j_schema = self._j_schema.field(field_name, to_java_type(field_type))
+        return self
+
+    def from_origin_field(self, origin_field_name):
 
 Review comment:
   Why use `from_origin_field` instead of `from`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services