flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-8360) Implement task-local state recovery
Date Thu, 11 Jan 2018 14:26:04 GMT

    [ https://issues.apache.org/jira/browse/FLINK-8360?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16322304#comment-16322304
] 

ASF GitHub Bot commented on FLINK-8360:
---------------------------------------

Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5239#discussion_r160962776
  
    --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamTaskStateManagerImplTest.java
---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.streaming.api.operators;
    +
    +import org.apache.flink.api.common.JobID;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.base.IntSerializer;
    +import org.apache.flink.core.fs.CloseableRegistry;
    +import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore;
    +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
    +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
    +import org.apache.flink.runtime.checkpoint.savepoint.CheckpointTestUtils;
    +import org.apache.flink.runtime.execution.Environment;
    +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
    +import org.apache.flink.runtime.jobgraph.OperatorID;
    +import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
    +import org.apache.flink.runtime.query.TaskKvStateRegistry;
    +import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
    +import org.apache.flink.runtime.state.CheckpointStreamFactory;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
    +import org.apache.flink.runtime.state.OperatorStateBackend;
    +import org.apache.flink.runtime.state.OperatorStateHandle;
    +import org.apache.flink.runtime.state.StateBackend;
    +import org.apache.flink.runtime.state.StatePartitionStreamProvider;
    +import org.apache.flink.runtime.state.TaskStateManager;
    +import org.apache.flink.runtime.state.TaskStateManagerImplTest;
    +import org.apache.flink.runtime.state.memory.MemoryStateBackend;
    +import org.apache.flink.runtime.taskmanager.TestCheckpointResponder;
    +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
    +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
    +import org.apache.flink.util.CloseableIterable;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.Random;
    +
    +import static org.mockito.Matchers.any;
    +import static org.mockito.Matchers.anyInt;
    +import static org.mockito.Matchers.eq;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.spy;
    +import static org.mockito.Mockito.verify;
    +import static org.mockito.Mockito.when;
    +
    +/**
    + * Test for {@link StreamTaskStateManagerImpl}.
    + */
    +public class StreamTaskStateManagerImplTest {
    +
    +	@Test
    +	public void testNoRestore() throws Exception {
    +
    +		MemoryStateBackend stateBackend = spy(new MemoryStateBackend(1024));
    +
    +		// No job manager provided state to restore
    +		StreamTaskStateManager streamTaskStateManager = streamTaskStateManager(stateBackend,
null, true);
    +
    +		OperatorID operatorID = new OperatorID(47L, 11L);
    +		AbstractStreamOperator<?> streamOperator = mock(AbstractStreamOperator.class);
    +		when(streamOperator.getOperatorID()).thenReturn(operatorID);
    +
    +		TypeSerializer<?> typeSerializer = new IntSerializer();
    +		CloseableRegistry closeableRegistry = new CloseableRegistry();
    +
    +		StreamOperatorStateContext stateContext = streamTaskStateManager.streamOperatorStateContext(
    +			streamOperator,
    +			typeSerializer,
    +			closeableRegistry);
    +
    +		verify(stateBackend).createKeyedStateBackend(
    +			any(Environment.class),
    +			any(JobID.class),
    +			any(String.class),
    +			eq(typeSerializer),
    +			anyInt(),
    +			any(KeyGroupRange.class),
    +			any(TaskKvStateRegistry.class));
    +
    +		verify(stateBackend).createOperatorStateBackend(
    +			any(Environment.class),
    +			any(String.class));
    +
    +		verify(stateBackend).createStreamFactory(
    +			any(JobID.class),
    +			any(String.class));
    +
    +		OperatorStateBackend operatorStateBackend = stateContext.operatorStateBackend();
    +		AbstractKeyedStateBackend<?> keyedStateBackend = stateContext.keyedStateBackend();
    +		InternalTimeServiceManager<?, ?> timeServiceManager = stateContext.internalTimerServiceManager();
    +		CheckpointStreamFactory streamFactory = stateContext.checkpointStreamFactory();
    +		CloseableIterable<KeyGroupStatePartitionStreamProvider> keyedStateInputs = stateContext.rawKeyedStateInputs();
    +		CloseableIterable<StatePartitionStreamProvider> operatorStateInputs = stateContext.rawOperatorStateInputs();
    +
    +		Assert.assertEquals(false, stateContext.isRestored());
    +		Assert.assertNotNull(operatorStateBackend);
    --- End diff --
    
    Shouldn't you make some additional assertions on those fields besides checking not null?
Or is it enough on this level to have just those not nulls?


> 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
>             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
(v6.4.14#64029)

Mime
View raw message