[
https://issues.apache.org/jira/browse/FLINK-5715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15899686#comment-15899686
]
ASF GitHub Bot commented on FLINK-5715:
---------------------------------------
Github user aljoscha commented on a diff in the pull request:
https://github.com/apache/flink/pull/3466#discussion_r104675456
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedMapsStateTable.java
---
@@ -0,0 +1,319 @@
+/*
+ * 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.heap;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
+import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * This implementation of {@link StateTable} is based on the Flink 1.2
implementation, using nested {@link HashMap}
--- End diff --
I think it's not necessary to mention Flink 1.2. The description you gave
should be enough.
> Asynchronous snapshotting for HeapKeyedStateBackend
> ---------------------------------------------------
>
> Key: FLINK-5715
> URL: https://issues.apache.org/jira/browse/FLINK-5715
> Project: Flink
> Issue Type: New Feature
> Components: State Backends, Checkpointing
> Affects Versions: 1.3.0
> Reporter: Stefan Richter
> Assignee: Stefan Richter
>
> Blocking snapshots render the HeapKeyedStateBackend practically unusable for
> many user in productions. Their jobs can not tolerate stopped processing for
> the time it takes to write gigabytes of data from memory to disk.
> Asynchronous snapshots would be a solution to this problem. The challenge for
> the implementation is coming up with a copy-on-write scheme for the in-memory
> hash maps that build the foundation of this backend. After taking a closer
> look, this problem is twofold. First, providing CoW semantics for the hashmap
> itself, as a mutible structure, thereby avoiding costly locking or blocking
> where possible. Second, CoW for the mutable value objects, e.g. through
> cloning via serializers.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)