Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-21 Thread via GitHub


fsk119 merged PR #24068:
URL: https://github.com/apache/flink/pull/24068


-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-18 Thread via GitHub


xuyangzhong commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1457550115


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/groupwindow/context/WindowContext.java:
##
@@ -0,0 +1,74 @@
+/*
+ * 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.table.runtime.operators.window.groupwindow.context;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.Window;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.InternalWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+
+import java.time.ZoneId;
+import java.util.Collection;
+
+/** A context contains some information used for {@link 
InternalWindowProcessFunction}. */
+public interface WindowContext {
+
+/**
+ * Creates a partitioned state handle, using the state backend configured 
for this task.
+ *
+ * @throws IllegalStateException Thrown, if the key/value state was 
already initialized.
+ * @throws Exception Thrown, if the state backend cannot create the 
key/value state.
+ */
+ S getPartitionedState(StateDescriptor 
stateDescriptor) throws Exception;
+
+/** @return current key of current processed element. */

Review Comment:
   removed changes on it



-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-18 Thread via GitHub


xuyangzhong commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1457549799


##
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java:
##
@@ -96,4 +97,16 @@ protected Transformation translateToPlanInternal(
 inputTransform.getParallelism(),
 false);
 }
+
+private WindowTableFunctionOperator 
createAlignedWindowTableFunctionOperator(
+ExecNodeConfig config) {
+// TODO use WindowAssigner instead of using GroupWindowAssigner

Review Comment:
   removed it



##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/groupwindow/context/WindowContext.java:
##
@@ -0,0 +1,74 @@
+/*
+ * 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.table.runtime.operators.window.groupwindow.context;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.Window;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.InternalWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+
+import java.time.ZoneId;
+import java.util.Collection;
+
+/** A context contains some information used for {@link 
InternalWindowProcessFunction}. */
+public interface WindowContext {
+
+/**
+ * Creates a partitioned state handle, using the state backend configured 
for this task.
+ *
+ * @throws IllegalStateException Thrown, if the key/value state was 
already initialized.
+ * @throws Exception Thrown, if the state backend cannot create the 
key/value state.
+ */
+ S getPartitionedState(StateDescriptor 
stateDescriptor) throws Exception;

Review Comment:
   removed changes on it



-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-18 Thread via GitHub


xuyangzhong commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1457549471


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/groupwindow/context/WindowContext.java:
##
@@ -0,0 +1,74 @@
+/*
+ * 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.table.runtime.operators.window.groupwindow.context;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.Window;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.InternalWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+
+import java.time.ZoneId;
+import java.util.Collection;
+
+/** A context contains some information used for {@link 
InternalWindowProcessFunction}. */
+public interface WindowContext {

Review Comment:
   removed changes on it



-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-18 Thread via GitHub


fsk119 commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1457395858


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/groupwindow/context/WindowContext.java:
##
@@ -0,0 +1,74 @@
+/*
+ * 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.table.runtime.operators.window.groupwindow.context;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.Window;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.InternalWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+
+import java.time.ZoneId;
+import java.util.Collection;
+
+/** A context contains some information used for {@link 
InternalWindowProcessFunction}. */
+public interface WindowContext {
+
+/**
+ * Creates a partitioned state handle, using the state backend configured 
for this task.
+ *
+ * @throws IllegalStateException Thrown, if the key/value state was 
already initialized.
+ * @throws Exception Thrown, if the state backend cannot create the 
key/value state.
+ */
+ S getPartitionedState(StateDescriptor 
stateDescriptor) throws Exception;
+
+/** @return current key of current processed element. */

Review Comment:
   Return



##
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java:
##
@@ -96,4 +97,16 @@ protected Transformation translateToPlanInternal(
 inputTransform.getParallelism(),
 false);
 }
+
+private WindowTableFunctionOperator 
createAlignedWindowTableFunctionOperator(
+ExecNodeConfig config) {
+// TODO use WindowAssigner instead of using GroupWindowAssigner

Review Comment:
   Add issue id



##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/groupwindow/context/WindowContext.java:
##
@@ -0,0 +1,74 @@
+/*
+ * 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.table.runtime.operators.window.groupwindow.context;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.Window;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.InternalWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+
+import java.time.ZoneId;
+import java.util.Collection;
+
+/** A context contains some information used for {@link 
InternalWindowProcessFunction}. */
+public interface WindowContext {
+
+/**
+ * Creates a partitioned state handle, using the state backend configured 
for this task.
+ *
+ * @throws IllegalStateException Thrown, if the key/value state was 
already initialized.
+ * @throws Exception Thrown, if the state backend cannot create the 
key/value state.
+ */
+ S getPartitionedState(StateDescriptor 
stateDescriptor) throws Exception;

Review Comment:
   0.0 Why context can provide state?



-- 
This is an automated message from the Apache Git 

Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-18 Thread via GitHub


xuyangzhong commented on PR #24068:
URL: https://github.com/apache/flink/pull/24068#issuecomment-1898142251

   @flinkbot run azure


-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-17 Thread via GitHub


xuyangzhong commented on PR #24068:
URL: https://github.com/apache/flink/pull/24068#issuecomment-1897862786

   @flinkbot run azure


-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-17 Thread via GitHub


xuyangzhong commented on PR #24068:
URL: https://github.com/apache/flink/pull/24068#issuecomment-1897831028

   Hi, @fsk119 . Thank you for your review. 
   I think some of the questions you raised are valuable and meaningful. I have 
created several independent JIRA to follow up and further optimize the old code 
that was previously implemented. For this PR, the changes are purely 
refactoring. No changes to the nature of the original implementation will be 
introduced.


-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-17 Thread via GitHub


xuyangzhong commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1456843664


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/windowtvf/common/WindowAssigner.java:
##
@@ -0,0 +1,43 @@
+/*
+ * 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.table.runtime.operators.window.windowtvf.common;
+
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.GroupWindowAssigner;
+
+import java.io.Serializable;
+
+/**
+ * WindowAssigner is used to assign windows to elements.
+ *
+ * The differences between {@link WindowAssigner} and {@link 
GroupWindowAssigner} is that, this
+ * window assigner is translated from the new window TVF syntax, but the other 
is from the legacy
+ * GROUP WINDOW FUNCTION syntax. In the long future, {@link 
GroupWindowAssigner} will be dropped.
+ *
+ * See more details in {@link AbstractWindowOperator}.
+ *
+ * TODO support UnsliceAssigner.
+ */
+public interface WindowAssigner extends Serializable {
+
+/**
+ * Returns {@code true} if elements are assigned to windows based on event 
time, {@code false}
+ * based on processing time.
+ */
+boolean isEventTime();

Review Comment:
   Agree it! I create a new JIRA for it. 
https://issues.apache.org/jira/browse/FLINK-34139.



-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-17 Thread via GitHub


xuyangzhong commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1456843334


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/groupwindow/context/WindowContext.java:
##
@@ -0,0 +1,74 @@
+/*
+ * 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.table.runtime.operators.window.groupwindow.context;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.Window;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.InternalWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+
+import java.time.ZoneId;
+import java.util.Collection;
+
+/** A context contains some information used for {@link 
InternalWindowProcessFunction}. */
+public interface WindowContext {
+
+/**
+ * Creates a partitioned state handle, using the state backend configured 
for this task.
+ *
+ * @throws IllegalStateException Thrown, if the key/value state was 
already initialized.
+ * @throws Exception Thrown, if the state backend cannot create the 
key/value state.
+ */
+ S getPartitionedState(StateDescriptor 
stateDescriptor) throws Exception;
+
+/** @return current key of current processed element. */
+K currentKey();
+
+/** Returns the current processing time. */
+long currentProcessingTime();
+
+/** Returns the current event-time watermark. */
+long currentWatermark();
+
+/** Returns the shifted timezone of the window. */
+ZoneId getShiftTimeZone();
+
+/** Gets the accumulators of the given window. */
+RowData getWindowAccumulators(W window) throws Exception;
+
+/** Sets the accumulators of the given window. */
+void setWindowAccumulators(W window, RowData acc) throws Exception;
+
+/** Clear window state of the given window. */
+void clearWindowState(W window) throws Exception;
+
+/** Clear previous agg state (used for retraction) of the given window. */
+void clearPreviousState(W window) throws Exception;
+
+/** Call {@link Trigger#clear(Window)}} on trigger. */
+void clearTrigger(W window) throws Exception;

Review Comment:
   Agree! I create a new JIRA for it. 
https://issues.apache.org/jira/browse/FLINK-34140.



-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-17 Thread via GitHub


xuyangzhong commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1456843334


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/groupwindow/context/WindowContext.java:
##
@@ -0,0 +1,74 @@
+/*
+ * 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.table.runtime.operators.window.groupwindow.context;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.Window;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.InternalWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+
+import java.time.ZoneId;
+import java.util.Collection;
+
+/** A context contains some information used for {@link 
InternalWindowProcessFunction}. */
+public interface WindowContext {
+
+/**
+ * Creates a partitioned state handle, using the state backend configured 
for this task.
+ *
+ * @throws IllegalStateException Thrown, if the key/value state was 
already initialized.
+ * @throws Exception Thrown, if the state backend cannot create the 
key/value state.
+ */
+ S getPartitionedState(StateDescriptor 
stateDescriptor) throws Exception;
+
+/** @return current key of current processed element. */
+K currentKey();
+
+/** Returns the current processing time. */
+long currentProcessingTime();
+
+/** Returns the current event-time watermark. */
+long currentWatermark();
+
+/** Returns the shifted timezone of the window. */
+ZoneId getShiftTimeZone();
+
+/** Gets the accumulators of the given window. */
+RowData getWindowAccumulators(W window) throws Exception;
+
+/** Sets the accumulators of the given window. */
+void setWindowAccumulators(W window, RowData acc) throws Exception;
+
+/** Clear window state of the given window. */
+void clearWindowState(W window) throws Exception;
+
+/** Clear previous agg state (used for retraction) of the given window. */
+void clearPreviousState(W window) throws Exception;
+
+/** Call {@link Trigger#clear(Window)}} on trigger. */
+void clearTrigger(W window) throws Exception;

Review Comment:
   Agree! I create a new JIRA for it. 
https://issues.apache.org/jira/browse/FLINK-34139.



-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-17 Thread via GitHub


xuyangzhong commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1456842979


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/groupwindow/context/WindowContext.java:
##
@@ -0,0 +1,74 @@
+/*
+ * 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.table.runtime.operators.window.groupwindow.context;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.Window;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.InternalWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+
+import java.time.ZoneId;
+import java.util.Collection;
+
+/** A context contains some information used for {@link 
InternalWindowProcessFunction}. */
+public interface WindowContext {

Review Comment:
   I'm afraid so, as in some places where the window context is used (such as 
InternalWindowProcessFunction), a more generic type parameter K is employed 
instead of the specific RowData type. I believe maintaining the most generic 
key type is not necessarily a bad thing.
   Of course, if you think it needs to be changed to the specific RowData type, 
I can introduce a separate JIRA to address it.



-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-17 Thread via GitHub


xuyangzhong commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1456822860


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/MergeCallback.java:
##
@@ -0,0 +1,43 @@
+/*
+ * 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.table.runtime.operators.window;
+
+import javax.annotation.Nullable;
+
+/**
+ * Callback to be used in when merging slices or windows for specifying which 
slices or windows
+ * should be merged.
+ *
+ * @param  The type {@link Window} for windows or the type {@link Long} for 
slices that this
+ * callback used to merge.
+ * @param  The result type like {@link java.util.Collection} or {@link 
Iterable} to specify which
+ * slices or windows should be merged. TODO use {@link 
java.util.Collection} uniformly.
+ */
+public interface MergeCallback {
+
+/**
+ * Specifies that states of the given windows or slices should be merged 
into the result window
+ * or slice.
+ *
+ * @param mergeResult The resulting merged window or slice, {@code null} 
if it represents a
+ * non-state namespace.
+ * @param toBeMerged Windows or slices that should be merged into one 
window or slice.
+ */
+void merge(@Nullable W mergeResult, R toBeMerged) throws Exception;

Review Comment:
   Agree with you. I create an extra JIRA for this. 
https://issues.apache.org/jira/browse/FLINK-34138



-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-17 Thread via GitHub


xuyangzhong commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1456817705


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/builder/AbstractWindowAggOperatorBuilder.java:
##
@@ -0,0 +1,98 @@
+/*
+ * 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.table.runtime.operators.aggregate.window.builder;
+
+import org.apache.flink.table.data.RowData;
+import 
org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import 
org.apache.flink.table.runtime.operators.window.windowtvf.common.AbstractWindowOperator;
+import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer;
+import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer;
+
+import java.time.ZoneId;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The {@link AbstractWindowAggOperatorBuilder} is a base class for building 
window aggregate
+ * operators.
+ *
+ * See more details in {@link SlicingWindowAggOperatorBuilder}.
+ *
+ * TODO support UnslicingWindowAggOperatorBuilder.
+ *
+ * @param  The type of the window. {@link Long} for slicing window.
+ * @param  The implementation of the abstract builder.
+ */
+public abstract class AbstractWindowAggOperatorBuilder<

Review Comment:
   Remove it and do this refactor later.



##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/builder/AbstractWindowAggOperatorBuilder.java:
##
@@ -0,0 +1,98 @@
+/*
+ * 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.table.runtime.operators.aggregate.window.builder;
+
+import org.apache.flink.table.data.RowData;
+import 
org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import 
org.apache.flink.table.runtime.operators.window.windowtvf.common.AbstractWindowOperator;
+import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer;
+import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer;
+
+import java.time.ZoneId;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The {@link AbstractWindowAggOperatorBuilder} is a base class for building 
window aggregate
+ * operators.
+ *
+ * See more details in {@link SlicingWindowAggOperatorBuilder}.
+ *
+ * TODO support UnslicingWindowAggOperatorBuilder.
+ *
+ * @param  The type of the window. {@link Long} for slicing window.
+ * @param  The implementation of the abstract builder.
+ */
+public abstract class AbstractWindowAggOperatorBuilder<
+W, T extends AbstractWindowAggOperatorBuilder> {
+
+protected AbstractRowDataSerializer inputSerializer;
+protected PagedTypeSerializer keySerializer;
+protected AbstractRowDataSerializer accSerializer;
+protected GeneratedNamespaceAggsHandleFunction 
generatedAggregateFunction;
+protected GeneratedNamespaceAggsHandleFunction 
localGeneratedAggregateFunction;
+protected GeneratedNamespaceAggsHandleFunction 
globalGeneratedAggregateFunction;
+protected ZoneId shiftTimeZone;
+
+public T inputSerializer(AbstractRowDataSerializer 
inputSerializer) {
+this.inputSerializer = inputSerializer;
+return self();
+}
+
+public T shiftTimeZone(ZoneId shiftTimeZone) {
+this.shiftTimeZone = shiftTimeZone;
+return self();
+}
+
+public T 

Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-17 Thread via GitHub


xuyangzhong commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1456817108


##
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/logical/WindowSpec.java:
##
@@ -31,4 +32,15 @@
 public interface WindowSpec {
 
 String toSummaryString(String windowing);
+
+/**
+ * Return if the window is a aligned window.
+ *
+ * See more details about aligned window and unaligned window in {@link
+ * 
org.apache.flink.table.runtime.operators.window.windowtvf.common.AbstractWindowOperator}.
+ *
+ * TODO introduce unaligned window like session window.

Review Comment:
   Add a jira issue for it.



-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-17 Thread via GitHub


fsk119 commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1455848131


##
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/logical/WindowSpec.java:
##
@@ -31,4 +32,15 @@
 public interface WindowSpec {
 
 String toSummaryString(String windowing);
+
+/**
+ * Return if the window is a aligned window.

Review Comment:
   Return true if xxx



##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/groupwindow/context/WindowContext.java:
##
@@ -0,0 +1,74 @@
+/*
+ * 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.table.runtime.operators.window.groupwindow.context;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.Window;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.InternalWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+
+import java.time.ZoneId;
+import java.util.Collection;
+
+/** A context contains some information used for {@link 
InternalWindowProcessFunction}. */
+public interface WindowContext {
+
+/**
+ * Creates a partitioned state handle, using the state backend configured 
for this task.
+ *
+ * @throws IllegalStateException Thrown, if the key/value state was 
already initialized.
+ * @throws Exception Thrown, if the state backend cannot create the 
key/value state.
+ */
+ S getPartitionedState(StateDescriptor 
stateDescriptor) throws Exception;
+
+/** @return current key of current processed element. */
+K currentKey();
+
+/** Returns the current processing time. */
+long currentProcessingTime();
+
+/** Returns the current event-time watermark. */
+long currentWatermark();
+
+/** Returns the shifted timezone of the window. */
+ZoneId getShiftTimeZone();
+
+/** Gets the accumulators of the given window. */
+RowData getWindowAccumulators(W window) throws Exception;
+
+/** Sets the accumulators of the given window. */
+void setWindowAccumulators(W window, RowData acc) throws Exception;
+
+/** Clear window state of the given window. */
+void clearWindowState(W window) throws Exception;
+
+/** Clear previous agg state (used for retraction) of the given window. */
+void clearPreviousState(W window) throws Exception;
+
+/** Call {@link Trigger#clear(Window)}} on trigger. */
+void clearTrigger(W window) throws Exception;

Review Comment:
   BTW, if a context provide some actions, it's better to rename it...



##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/windowtvf/common/AbstractWindowOperator.java:
##
@@ -73,34 +68,18 @@
  * Because aligned windows can be divided into finite number of 
non-overlapping chunks (a.k.a.
  * slices), which can apply efficient processing to share intermediate results.
  *
- * Concept of Slice
- *
- * Dividing a window of aligned windows into a finite number of 
non-overlapping chunks, where the
- * chunks are slices. It has the following properties:
- *
- * 
- *   An element must only belong to a single slice.
- *   Slices are non-overlapping, i.e. S_i and S_j should not have any 
shared elements if i != j.
- *   A window is consist of a finite number of slices.
- * 
- *
- * Abstraction of Slicing Window Operator
- *
- * A slicing window operator is a simple wrap of {@link 
SlicingWindowProcessor}. It delegates all
- * the important methods to the underlying processor, where the processor can 
have different
- * implementation for aggregate and topk or others.
+ * Note: currently, {@link AbstractWindowOperator} doesn't support 
early-fire and late-arrival.
+ * Thus late elements (elements belong to emitted windows) will be simply 
dropped.
  *
- * A {@link SlicingWindowProcessor} usually leverages the {@link 
SliceAssigner} to assign slices
- * and calculate based on the slices. See {@link 
SliceSharedWindowAggProcessor} as 

Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-14 Thread via GitHub


xuyangzhong commented on PR #24068:
URL: https://github.com/apache/flink/pull/24068#issuecomment-1891209495

   The CI failure is caused by https://issues.apache.org/jira/browse/FLINK-34077


-- 
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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-10 Thread via GitHub


flinkbot commented on PR #24068:
URL: https://github.com/apache/flink/pull/24068#issuecomment-1886331556

   
   ## CI report:
   
   * fd074024357c148202e3ff3e218309480ba12569 UNKNOWN
   
   
   Bot commands
 The @flinkbot bot supports the following commands:
   
- `@flinkbot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@flink.apache.org

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



[PR] [FLINK-34049][table]Refactor classes related to window TVF aggregation to prepare for non-aligned windows [flink]

2024-01-10 Thread via GitHub


xuyangzhong opened a new pull request, #24068:
URL: https://github.com/apache/flink/pull/24068

   ## What is the purpose of the change
   
   Currently, the implementation of the window aggregate op with new WINDOW TVF 
syntax is strongly bound to slicing windows such as TUMBLE, HOP and CUMULATE. 
This brings difficulties to the introduction of unslicing windows such as 
SESSION.
   
   This pr aims to refactor some classes about window aggregate op to do the 
preparation for the introduction of unslicing windows.
   
   
   ## Brief change log
   
   *(for example:)*
 - *add two package 'groupwindow' and 'windowtvf' under 
'org.apache.flink.table.runtime.operators.window' to split the different 
implements about the legacy group window agg and the new window tvf agg*
 - *move the classes related into these two new packages*
 - *extract abstract class about slicing window processor and operator*
   
   
   ## Verifying this change
   
   Existent tests can cover this changes.
   
   ## Does this pull request potentially affect one of the following parts:
   
 - Dependencies (does it add or upgrade a dependency): no
 - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: no
 - The serializers: no
 - The runtime per-record code paths (performance sensitive): no
 - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Kubernetes/Yarn, ZooKeeper: no
 - The S3 file system connector: no
   
   ## Documentation
   
 - Does this pull request introduce a new feature? no
 - If yes, how is the feature documented? no
   


-- 
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-unsubscr...@flink.apache.org

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