StefanRRichter commented on a change in pull request #7674: [FLINK-10043] 
[State Backends] Refactor RocksDBKeyedStateBackend object 
construction/initialization/restore code
URL: https://github.com/apache/flink/pull/7674#discussion_r257630419
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackendBuilder.java
 ##########
 @@ -0,0 +1,68 @@
+/*
+ * 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.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
+
+import java.util.Collection;
+
+/**
+ * An abstract base implementation of the {@link StateBackendBuilder} 
interface.
+ */
+public abstract class AbstractKeyedStateBackendBuilder<K>
+       implements StateBackendBuilder<AbstractKeyedStateBackend, 
BackendBuildingException> {
+       protected final TaskKvStateRegistry kvStateRegistry;
+       protected final TypeSerializer<K> keySerializer;
+       protected final StateSerializerProvider<K> keySerializerProvider;
+       protected final ClassLoader userCodeClassLoader;
+       protected final int numberOfKeyGroups;
+       protected final KeyGroupRange keyGroupRange;
+       protected final ExecutionConfig executionConfig;
+       protected final TtlTimeProvider ttlTimeProvider;
+       protected final StreamCompressionDecorator keyGroupCompressionDecorator;
+       protected final Collection<KeyedStateHandle> restoreStateHandles;
+
+       public AbstractKeyedStateBackendBuilder(
+               TaskKvStateRegistry kvStateRegistry,
+               TypeSerializer<K> keySerializer,
+               ClassLoader userCodeClassLoader,
+               int numberOfKeyGroups,
+               KeyGroupRange keyGroupRange,
+               ExecutionConfig executionConfig,
+               TtlTimeProvider ttlTimeProvider,
+               Collection<KeyedStateHandle> stateHandles) {
+               this.kvStateRegistry = kvStateRegistry;
+               this.keySerializer = keySerializer;
+               this.keySerializerProvider = 
StateSerializerProvider.fromNewRegisteredSerializer(keySerializer);
+               this.userCodeClassLoader = userCodeClassLoader;
+               this.numberOfKeyGroups = numberOfKeyGroups;
+               this.keyGroupRange = keyGroupRange;
+               this.executionConfig = executionConfig;
+               this.ttlTimeProvider = ttlTimeProvider;
+               if (executionConfig != null && 
executionConfig.isUseSnapshotCompression()) {
 
 Review comment:
   I would suggest to keep such logic out of the constructor. A better and more 
testable approach would be to check the flags outside this method and just pass 
in whatever compression decorator was chosen. This way, tests can also pass in 
arbitrary compression decorators and it becomes easier to add new compression 
schemes in the future.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to