Re: [PR] [FLINK-35046][state] Introduce AsyncKeyedStateBackend supporting to create StateExecutor [flink]
masteryhx closed pull request #24663: [FLINK-35046][state] Introduce AsyncKeyedStateBackend supporting to create StateExecutor URL: https://github.com/apache/flink/pull/24663 -- 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-35046][state] Introduce AsyncKeyedStateBackend supporting to create StateExecutor [flink]
Zakelly commented on code in PR #24663: URL: https://github.com/apache/flink/pull/24663#discussion_r1566625619 ## flink-runtime/src/main/java/org/apache/flink/runtime/state/AsyncKeyedStateBackend.java: ## @@ -0,0 +1,38 @@ +/* + * 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.runtime.state; + +import org.apache.flink.runtime.asyncprocessing.StateExecutor; +import org.apache.flink.util.Disposable; + +/** + * An async keyed state backend provides methods supporting to access keyed state asynchronously and + * in batch. + */ +public interface AsyncKeyedStateBackend extends Disposable { Review Comment: Better annotate this as `@Internal` -- 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-35046][state] Introduce AsyncKeyedStateBackend supporting to create StateExecutor [flink]
masteryhx commented on code in PR #24663: URL: https://github.com/apache/flink/pull/24663#discussion_r1565377484 ## flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java: ## @@ -104,6 +104,25 @@ default String getName() { CheckpointableKeyedStateBackend createKeyedStateBackend( KeyedStateBackendParameters parameters) throws Exception; +/** + * Creates a new {@link AsyncKeyedStateBackend} which supports to access keyed state + * asynchronously. + * + * Keyed State is state where each value is bound to a key. + * + * @param parameters The arguments bundle for creating {@link AsyncKeyedStateBackend}. + * @param The type of the keys by which the state is organized. + * @return The Async Keyed State Backend for the given job, operator. + * @throws Exception This method may forward all exceptions that occur while instantiating the + * backend. + */ +@Experimental +default AsyncKeyedStateBackend createAsyncKeyedStateBackend( +KeyedStateBackendParameters parameters) throws Exception { +throw new UnsupportedOperationException( +"Don't support createAsyncKeyedStateBackend by default"); +} + Review Comment: Make sense. I just added `supportsAsyncKeyedStateBackend`. The naming is consistent with `supportsNoClaimRestoreMode` and `supportsSavepointFormat`. -- 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-35046][state] Introduce AsyncKeyedStateBackend supporting to create StateExecutor [flink]
Zakelly commented on code in PR #24663: URL: https://github.com/apache/flink/pull/24663#discussion_r1565158902 ## flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java: ## @@ -104,6 +104,25 @@ default String getName() { CheckpointableKeyedStateBackend createKeyedStateBackend( KeyedStateBackendParameters parameters) throws Exception; +/** + * Creates a new {@link AsyncKeyedStateBackend} which supports to access keyed state + * asynchronously. + * + * Keyed State is state where each value is bound to a key. + * + * @param parameters The arguments bundle for creating {@link AsyncKeyedStateBackend}. + * @param The type of the keys by which the state is organized. + * @return The Async Keyed State Backend for the given job, operator. + * @throws Exception This method may forward all exceptions that occur while instantiating the + * backend. + */ +@Experimental +default AsyncKeyedStateBackend createAsyncKeyedStateBackend( +KeyedStateBackendParameters parameters) throws Exception { +throw new UnsupportedOperationException( +"Don't support createAsyncKeyedStateBackend by default"); +} + Review Comment: Should we add another method `isSupportAsyncKeyedStateBackend` within `StateBackend`? -- 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-35046][state] Introduce AsyncKeyedStateBackend supporting to create StateExecutor [flink]
flinkbot commented on PR #24663: URL: https://github.com/apache/flink/pull/24663#issuecomment-2054586620 ## CI report: * b551d919ce3a63501e9ebc6f8769bd56f0c74d0c 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-35046][state] Introduce AsyncKeyedStateBackend supporting to create StateExecutor [flink]
masteryhx opened a new pull request, #24663: URL: https://github.com/apache/flink/pull/24663 ## What is the purpose of the change Introduce AsyncKeyedStateBackend which supports to create StateExecutor ## Brief change log - Introduce a new interface called AsyncKeyedStateBackend - Introduce a new method for StateBackend to create AsyncKeyedStateBackend ## Verifying this change This change modified AsyncExecutionControllerTest to verify the new added interface. ## Does this pull request potentially affect one of the following parts: - Dependencies (does it add or upgrade a dependency): (yes / **no**) - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (**yes** / no) - The serializers: (yes / **no** / don't know) - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know) - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes / **no** / don't know) - The S3 file system connector: (yes / **no** / don't know) ## Documentation - Does this pull request introduce a new feature? (yes / **no**) - If yes, how is the feature documented? (**not applicable** / docs / JavaDocs / not documented) -- 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