From issues-return-153520-archive-asf-public=cust-asf.ponee.io@flink.apache.org Fri Feb 16 15:47:05 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 6EE7118067B for ; Fri, 16 Feb 2018 15:47:04 +0100 (CET) Received: (qmail 52797 invoked by uid 500); 16 Feb 2018 14:47:03 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 52786 invoked by uid 99); 16 Feb 2018 14:47:03 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 16 Feb 2018 14:47:03 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 07F5A18033B for ; Fri, 16 Feb 2018 14:47:03 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -110.311 X-Spam-Level: X-Spam-Status: No, score=-110.311 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id uVGTVkubH1ck for ; Fri, 16 Feb 2018 14:47:01 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 230505F4EB for ; Fri, 16 Feb 2018 14:47:01 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id A3F90E015F for ; Fri, 16 Feb 2018 14:47:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 6633F21E5E for ; Fri, 16 Feb 2018 14:47:00 +0000 (UTC) Date: Fri, 16 Feb 2018 14:47:00 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-8360) Implement task-local state recovery MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/FLINK-8360?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16367402#comment-16367402 ] ASF GitHub Bot commented on FLINK-8360: --------------------------------------- Github user sihuazhou commented on a diff in the pull request: https://github.com/apache/flink/pull/5239#discussion_r168774627 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java --- @@ -0,0 +1,265 @@ +/* + * 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.checkpoint; + +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nonnull; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.BiFunction; + +/** + * This class is a wrapper over multiple alternative {@link OperatorSubtaskState} that are (partial) substitutes for + * each other and imposes a priority ordering over all alternatives for the different states which define an order in + * which the operator should attempt to restore the state from them. One OperatorSubtaskState is considered as the + * "ground truth" about which state should be represented. Alternatives may be complete or partial substitutes for + * the "ground truth" with a higher priority (if they had a lower alternative, they would not really be alternatives). + * Substitution is determined on a per-sub-state basis. + */ +public class PrioritizedOperatorSubtaskState { + + /** Singleton instance for an empty, non-restored operator state. */ + private static final PrioritizedOperatorSubtaskState EMPTY_NON_RESTORED_INSTANCE = + new PrioritizedOperatorSubtaskState(new OperatorSubtaskState(), Collections.emptyList(), false); + + /** List of prioritized snapshot alternatives for managed operator state. */ + private final List> prioritizedManagedOperatorState; + + /** List of prioritized snapshot alternatives for raw operator state. */ + private final List> prioritizedRawOperatorState; + + /** List of prioritized snapshot alternatives for managed keyed state. */ + private final List> prioritizedManagedKeyedState; + + /** List of prioritized snapshot alternatives for raw keyed state. */ + private final List> prioritizedRawKeyedState; + + /** Signal flag if this represents state for a restored operator. */ + private final boolean restored; + + public PrioritizedOperatorSubtaskState( + @Nonnull OperatorSubtaskState jobManagerState, + @Nonnull List alternativesByPriority) { + this(jobManagerState, alternativesByPriority, true); + } + + public PrioritizedOperatorSubtaskState( + @Nonnull OperatorSubtaskState jobManagerState, + @Nonnull List alternativesByPriority, + boolean restored) { + + Preconditions.checkNotNull(jobManagerState, "Job manager state is null."); + int size = Preconditions.checkNotNull(alternativesByPriority, "Alternative states are null.").size(); + + this.restored = restored; + + List> managedOperatorAlternatives = new ArrayList<>(size); + List> managedKeyedAlternatives = new ArrayList<>(size); + List> rawOperatorAlternatives = new ArrayList<>(size); + List> rawKeyedAlternatives = new ArrayList<>(size); + + for (OperatorSubtaskState subtaskState : alternativesByPriority) { + + if (subtaskState != null) { + managedKeyedAlternatives.add(subtaskState.getManagedKeyedState()); + rawKeyedAlternatives.add(subtaskState.getRawKeyedState()); + managedOperatorAlternatives.add(subtaskState.getManagedOperatorState()); + rawOperatorAlternatives.add(subtaskState.getRawOperatorState()); + } + } + + // Key-groups should match. + BiFunction keyedStateApprover = + (ref, alt) -> ref.getKeyGroupRange().equals(alt.getKeyGroupRange()); + --- End diff -- Why `ref.getKeyGroupRange()` and `alt. ref.getKeyGroupRange ()` need to be strictly equal? It seems that `alt. ref.getKeyGroupRange ().begin <= ref. ref.getKeyGroupRange ().begin && alt. ref.getKeyGroupRange ().end >= ref ref.getKeyGroupRange().end` is also acceptable. > Implement task-local state recovery > ----------------------------------- > > Key: FLINK-8360 > URL: https://issues.apache.org/jira/browse/FLINK-8360 > Project: Flink > Issue Type: New Feature > Components: State Backends, Checkpointing > Reporter: Stefan Richter > Assignee: Stefan Richter > Priority: Major > Fix For: 1.5.0 > > > This issue tracks the development of recovery from task-local state. The main idea is to have a secondary, local copy of the checkpointed state, while there is still a primary copy in DFS that we report to the checkpoint coordinator. > Recovery can attempt to restore from the secondary local copy, if available, to save network bandwidth. This requires that the assignment from tasks to slots is as sticky is possible. > For starters, we will implement this feature for all managed keyed states and can easily enhance it to all other state types (e.g. operator state) later. -- This message was sent by Atlassian JIRA (v7.6.3#76005)