rkhachatryan commented on a change in pull request #13912: URL: https://github.com/apache/flink/pull/13912#discussion_r570308788
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java ########## @@ -103,6 +108,14 @@ public static StateBackend loadStateBackendFromConfig( String factoryClassName = backendName; switch (backendName.toLowerCase()) { + case HASHMAP_STATE_BACKEND_NAME: + HashMapStateBackend hashMapStateBackend = + new HashMapStateBackendFactory().createFromConfig(config, classLoader); + if (logger != null) { + logger.info("State backend is set to heap memory {}", hashMapStateBackend); + } + return hashMapStateBackend; + case MEMORY_STATE_BACKEND_NAME: MemoryStateBackend memBackend = new MemoryStateBackendFactory().createFromConfig(config, classLoader); Review comment: Why do we still need to create and return deprecated `MemoryStateBackend`? Why can't we return `HashMapStateBackend` as above? ditto `FS_STATE_BACKEND_NAME` ########## File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackendFactory.java ########## @@ -0,0 +1,33 @@ +/* + * 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.contrib.streaming.state; + +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.runtime.state.StateBackendFactory; + +/** A factory that creates an {@link EmbeddedRocksDBStateBackend} from a configuration. */ +public class EmbeddedRocksDBStateBackendFactory Review comment: Is it `@PublicEvolving`? ditto `HashMapStateBackendFactory` and `HashMapStateBackend` ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java ########## @@ -562,27 +566,30 @@ public CheckpointingMode getCheckpointingMode() { } /** - * Sets the state backend that describes how to store and checkpoint operator state. It defines - * both which data structures hold state during execution (for example hash tables, RockDB, or - * other data stores) as well as where checkpointed data will be persisted. + * Sets the state backend that describes how to store operator. It defines the data structures + * that hold state during execution (for example hash tables, RocksDB, or other data stores). Review comment: I think there is a mistake in this change: shouldn't it be "how to checkpoint operator state"? ditto: `StreamExecutionEnvironment` ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java ########## @@ -131,7 +144,7 @@ public static StateBackend loadStateBackendFromConfig( case ROCKSDB_STATE_BACKEND_NAME: factoryClassName = - "org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory"; + "org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackendFactory"; Review comment: I think this change breaks backwards combatibility (with old configs). Should we fallback to the older factory/name? ########## File path: docs/_includes/generated/checkpointing_configuration.html ########## @@ -60,7 +54,19 @@ <td><h5>state.savepoints.dir</h5></td> <td style="word-wrap: break-word;">(none)</td> <td>String</td> - <td>The default directory for savepoints. Used by the state backends that write savepoints to file systems (MemoryStateBackend, FsStateBackend, RocksDBStateBackend).</td> + <td>The default directory for savepoints. Used by the state backends that write savepoints to file systems (HashMapStateBackend, EmbeddedRocksDBStateBackend).</td> + </tr> + <tr> + <td><h5>state.snapshot.fs.memory-threshold</h5></td> Review comment: How about `state.storage.fs.memory-threshold` instead of `state.snapshot.fs.memory-threshold`? ditto `write-buffer-size` ########## File path: flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java ########## @@ -27,22 +29,70 @@ // general checkpoint and state backend options // ------------------------------------------------------------------------ - /** The state backend to be used to store and checkpoint state. */ + /** + * The checkpoint storage used to store state. + * + * <p>The implementation can be specified either via their shortcut name, or via the class name + * of a {@code StateBackendFactory}. If a StateBackendFactory class name is specified, the + * factory is instantiated (via its zero-argument constructor) and its {@code + * StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)} method is called. + * + * <p>Recognized shortcut names are 'hashmap' and 'rocksdb'. + */ Review comment: I guess this change and the one below were made by mistake. This is actually about `CHECKPOINT_STORAGE`, right? ---------------------------------------------------------------- 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: us...@infra.apache.org