Skip to content

Commit

Permalink
[hotfix][tests] Extract AcknowledgeStreamMockEnvironment
Browse files Browse the repository at this point in the history
  • Loading branch information
pnowojski authored and aljoscha committed Oct 24, 2017
1 parent 5bebef1 commit 4379d0d
Show file tree
Hide file tree
Showing 3 changed files with 69 additions and 79 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.testutils.OneShotLatch;
import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
import org.apache.flink.runtime.checkpoint.CheckpointOptions;
import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
import org.apache.flink.runtime.execution.Environment;
Expand All @@ -54,11 +53,11 @@
import org.apache.flink.streaming.api.operators.async.queue.StreamRecordQueueEntry;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.AcknowledgeStreamMockEnvironment;
import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
Expand Down Expand Up @@ -597,43 +596,6 @@ public void testStateSnapshotAndRestore() throws Exception {
restoredTaskHarness.getOutput());
}

private static class AcknowledgeStreamMockEnvironment extends StreamMockEnvironment {
private volatile long checkpointId;
private volatile TaskStateSnapshot checkpointStateHandles;

private final OneShotLatch checkpointLatch = new OneShotLatch();

public long getCheckpointId() {
return checkpointId;
}

AcknowledgeStreamMockEnvironment(
Configuration jobConfig, Configuration taskConfig,
ExecutionConfig executionConfig, long memorySize,
MockInputSplitProvider inputSplitProvider, int bufferSize) {
super(jobConfig, taskConfig, executionConfig, memorySize, inputSplitProvider, bufferSize);
}

@Override
public void acknowledgeCheckpoint(
long checkpointId,
CheckpointMetrics checkpointMetrics,
TaskStateSnapshot checkpointStateHandles) {

this.checkpointId = checkpointId;
this.checkpointStateHandles = checkpointStateHandles;
checkpointLatch.trigger();
}

public OneShotLatch getCheckpointLatch() {
return checkpointLatch;
}

public TaskStateSnapshot getCheckpointStateHandles() {
return checkpointStateHandles;
}
}

@Test
public void testAsyncTimeout() throws Exception {
final long timeout = 10L;
Expand Down
Original file line number Diff line number Diff line change
@@ -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
*
* https://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.runtime.tasks;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.testutils.OneShotLatch;
import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;

/**
* Stream environment that allows to wait for checkpoint acknowledgement.
*/
public class AcknowledgeStreamMockEnvironment extends StreamMockEnvironment {
private final OneShotLatch checkpointLatch = new OneShotLatch();
private volatile long checkpointId;
private volatile TaskStateSnapshot checkpointStateHandles;

public AcknowledgeStreamMockEnvironment(
Configuration jobConfig,
Configuration taskConfig,
ExecutionConfig executionConfig,
long memorySize,
MockInputSplitProvider inputSplitProvider,
int bufferSize) {
super(jobConfig, taskConfig, executionConfig, memorySize, inputSplitProvider, bufferSize);
}

public long getCheckpointId() {
return checkpointId;
}

@Override
public void acknowledgeCheckpoint(
long checkpointId,
CheckpointMetrics checkpointMetrics,
TaskStateSnapshot checkpointStateHandles) {

this.checkpointId = checkpointId;
this.checkpointStateHandles = checkpointStateHandles;
checkpointLatch.trigger();
}

public OneShotLatch getCheckpointLatch() {
return checkpointLatch;
}

public TaskStateSnapshot getCheckpointStateHandles() {
return checkpointStateHandles;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

package org.apache.flink.streaming.runtime.tasks;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.state.ListState;
Expand All @@ -28,9 +27,7 @@
import org.apache.flink.api.common.typeutils.base.StringSerializer;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.testutils.OneShotLatch;
import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
import org.apache.flink.runtime.checkpoint.CheckpointOptions;
import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
Expand Down Expand Up @@ -613,43 +610,6 @@ public IN getKey(IN value) throws Exception {
}
}

private static class AcknowledgeStreamMockEnvironment extends StreamMockEnvironment {
private volatile long checkpointId;
private volatile TaskStateSnapshot checkpointStateHandles;

private final OneShotLatch checkpointLatch = new OneShotLatch();

public long getCheckpointId() {
return checkpointId;
}

AcknowledgeStreamMockEnvironment(
Configuration jobConfig, Configuration taskConfig,
ExecutionConfig executionConfig, long memorySize,
MockInputSplitProvider inputSplitProvider, int bufferSize) {
super(jobConfig, taskConfig, executionConfig, memorySize, inputSplitProvider, bufferSize);
}

@Override
public void acknowledgeCheckpoint(
long checkpointId,
CheckpointMetrics checkpointMetrics,
TaskStateSnapshot checkpointStateHandles) {

this.checkpointId = checkpointId;
this.checkpointStateHandles = checkpointStateHandles;
checkpointLatch.trigger();
}

public OneShotLatch getCheckpointLatch() {
return checkpointLatch;
}

public TaskStateSnapshot getCheckpointStateHandles() {
return checkpointStateHandles;
}
}

private static class TestingStreamOperator<IN, OUT>
extends AbstractStreamOperator<OUT>
implements OneInputStreamOperator<IN, OUT> {
Expand Down

0 comments on commit 4379d0d

Please sign in to comment.