flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From aljoscha <...@git.apache.org>
Subject [GitHub] flink pull request #3479: [FLINK-5929] Allow Access to Per-Window State in P...
Date Mon, 06 Mar 2017 15:00:35 GMT
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3479#discussion_r104432699
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
---
    @@ -629,6 +645,135 @@ protected final boolean isCleanupTime(W window, long time) {
     	}
     
     	/**
    +	 * For now keyed state is not allowed in ProcessWindowFunctions
    +	 */
    +	public class MergingKeyStore implements KeyedStateStore {
    +
    +		protected W window;
    +
    +		@Override
    +		public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties)
{
    +			throw new RuntimeException("keyedState is not allowed in merging windows");
    +		}
    +
    +		@Override
    +		public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties)
{
    +			throw new RuntimeException("keyedState is not allowed in merging windows");
    +		}
    +
    +		@Override
    +		public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T>
stateProperties) {
    +			throw new RuntimeException("keyedState is not allowed in merging windows");
    +		}
    +
    +		@Override
    +		public <T, A> FoldingState<T, A> getFoldingState(FoldingStateDescriptor<T,
A> stateProperties) {
    +			throw new RuntimeException("keyedState is not allowed in merging windows");
    +		}
    +
    +		@Override
    +		public <UK, UV> MapState<UK, UV> getMapState(MapStateDescriptor<UK,
UV> stateProperties) {
    +			throw new RuntimeException("keyedState is not allowed in merging windows");
    +		}
    +	}
    +
    +	public class WindowPaneKeyStore implements KeyedStateStore {
    +
    +		protected W window;
    +
    +		@Override
    +		public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties)
{
    +			try {
    +				return WindowOperator.this.getPartitionedState(window, windowSerializer, stateProperties);
    +			} catch (Exception e) {
    +				throw new RuntimeException("Could not retrieve state", e);
    +			}
    +		}
    +
    +		@Override
    +		public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties)
{
    +			try {
    +				return WindowOperator.this.getPartitionedState(window, windowSerializer, stateProperties);
    +			} catch (Exception e) {
    +				throw new RuntimeException("Could not retrieve state", e);
    +			}
    +		}
    +
    +		@Override
    +		public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T>
stateProperties) {
    +			try {
    +				return WindowOperator.this.getPartitionedState(window, windowSerializer, stateProperties);
    +			} catch (Exception e) {
    +				throw new RuntimeException("Could not retrieve state", e);
    +			}
    +		}
    +
    +		@Override
    +		public <T, ACC> FoldingState<T, ACC> getFoldingState(FoldingStateDescriptor<T,
ACC> stateProperties) {
    +			try {
    +				return WindowOperator.this.getPartitionedState(window, windowSerializer, stateProperties);
    +			} catch (Exception e) {
    +				throw new RuntimeException("Could not retrieve state", e);
    +			}
    +		}
    +
    +		@Override
    +		public <UK, UV> MapState<UK, UV> getMapState(MapStateDescriptor<UK,
UV> stateProperties) {
    +			try {
    +				return WindowOperator.this.getPartitionedState(window, windowSerializer, stateProperties);
    +			} catch (Exception e) {
    +				throw new RuntimeException("Could not retrieve state", e);
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * {@code WindowContext} is a utility for handling {@code ProcessWindowFunction} invocations.
It can be reused
    +	 * by setting the {@code key} and {@code window} fields. No internal state must be kept
in
    +	 * the {@code WindowContext}
    +	 */
    +	public class WindowContext implements InternalWindowFunction.InternalWindowContext {
    +		protected W window;
    +
    +		protected WindowPaneKeyStore windowPaneKeyStore;
    +		protected MergingKeyStore mergingKeyStore;
    +
    +		public WindowContext(W window) {
    +			this.window = window;
    +			this.windowPaneKeyStore = new WindowPaneKeyStore();
    +			this.mergingKeyStore = new MergingKeyStore();
    +		}
    +
    +		@Override
    +		public String toString() {
    +			return "WindowContext{Window = " + window.toString() + "}";
    +		}
    +
    +		public void clear() throws Exception {
    +			userFunction.clear(window, this);
    +		}
    +
    +		@Override
    +		public KeyedStateStore windowState() {
    +			if (windowAssigner instanceof MergingWindowAssigner) {
    --- End diff --
    
    Instead of checking every time you could initialise the `WindowContext` with either a
`WindowStateStore` or the (exception throwing) `MergingStateStore` at the beginning.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

Mime
View raw message