You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/09/15 03:22:09 UTC

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #37889: [SPARK-40432][SS][PYTHON] Introduce GroupStateImpl and GroupStateTimeout in PySpark

HeartSaVioR commented on code in PR #37889:
URL: https://github.com/apache/spark/pull/37889#discussion_r971485439


##########
python/pyspark/sql/streaming/state.py:
##########
@@ -0,0 +1,192 @@
+#
+# 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 datetime
+import json
+from typing import Tuple, Optional
+
+from pyspark.sql.types import DateType, Row, StructType
+
+__all__ = ["GroupStateImpl", "GroupStateTimeout"]
+
+
+class GroupStateTimeout:
+    NoTimeout: str = "NoTimeout"
+    ProcessingTimeTimeout: str = "ProcessingTimeTimeout"
+    EventTimeTimeout: str = "EventTimeTimeout"
+
+
+class GroupStateImpl:
+    NO_TIMESTAMP: int = -1
+
+    def __init__(
+            self,
+            # JVM Constructor
+            optionalValue: Row,
+            batchProcessingTimeMs: int,
+            eventTimeWatermarkMs: int,
+            timeoutConf: str,
+            hasTimedOut: bool,
+            watermarkPresent: bool,
+            # JVM internal state.
+            defined: bool,
+            updated: bool,
+            removed: bool,
+            timeoutTimestamp: int,
+            # Python internal state.
+            keyAsUnsafe: bytes,
+            valueSchema: StructType,
+    ) -> None:
+        self._keyAsUnsafe = keyAsUnsafe
+        self._value = optionalValue
+        self._batch_processing_time_ms = batchProcessingTimeMs
+        self._event_time_watermark_ms = eventTimeWatermarkMs
+
+        assert timeoutConf in [
+            GroupStateTimeout.NoTimeout,
+            GroupStateTimeout.ProcessingTimeTimeout,
+            GroupStateTimeout.EventTimeTimeout,
+        ]
+        self._timeout_conf = timeoutConf
+
+        self._has_timed_out = hasTimedOut
+        self._watermark_present = watermarkPresent
+
+        self._defined = defined
+        self._updated = updated
+        self._removed = removed
+        self._timeout_timestamp = timeoutTimestamp
+        # Python internal state.
+        self._old_timeout_timestamp = timeoutTimestamp
+
+        self._value_schema = valueSchema
+
+    @property
+    def exists(self) -> bool:
+        return self._defined
+
+    @property
+    def get(self) -> Tuple:
+        if self.exists:
+            return tuple(self._value)
+        else:
+            raise ValueError("State is either not defined or has already been removed")
+
+    @property
+    def getOption(self) -> Optional[Tuple]:
+        if self.exists:
+            return tuple(self._value)
+        else:
+            return None
+
+    @property
+    def hasTimedOut(self) -> bool:
+        return self._has_timed_out
+
+    # NOTE: this function is only available to PySpark implementation due to underlying
+    # implementation, do not port to Scala implementation!
+    @property
+    def oldTimeoutTimestamp(self) -> int:
+        return self._old_timeout_timestamp
+
+    def update(self, newValue: Tuple) -> None:
+        if newValue is None:
+            raise ValueError("'None' is not a valid state value")
+
+        self._value = Row(*newValue)
+        self._defined = True
+        self._updated = True
+        self._removed = False
+
+    def remove(self) -> None:
+        self._defined = False
+        self._updated = False
+        self._removed = True
+
+    def setTimeoutDuration(self, durationMs: int) -> None:
+        if isinstance(durationMs, str):
+            # TODO(SPARK-XXXXX): Support string representation of durationMs.

Review Comment:
   Nice! I'll update them.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org