Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id ED1622009DC for ; Tue, 2 May 2017 17:53:04 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id EB7D8160BA1; Tue, 2 May 2017 15:53:04 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id E2F66160B9B for ; Tue, 2 May 2017 17:53:03 +0200 (CEST) Received: (qmail 56328 invoked by uid 500); 2 May 2017 15:53: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 56319 invoked by uid 99); 2 May 2017 15:53: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; Tue, 02 May 2017 15:53: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 A7604192B90 for ; Tue, 2 May 2017 15:53:02 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -3.22 X-Spam-Level: X-Spam-Status: No, score=-3.22 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id V6jrVvATd3tY for ; Tue, 2 May 2017 15:52:59 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id 46B4A5F5F9 for ; Tue, 2 May 2017 15:52:58 +0000 (UTC) Received: (qmail 56283 invoked by uid 99); 2 May 2017 15:52:57 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 02 May 2017 15:52:57 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 603C9DF9FD; Tue, 2 May 2017 15:52:57 +0000 (UTC) From: aljoscha To: issues@flink.incubator.apache.org Reply-To: issues@flink.incubator.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility... Content-Type: text/plain Message-Id: <20170502155257.603C9DF9FD@git1-us-west.apache.org> Date: Tue, 2 May 2017 15:52:57 +0000 (UTC) archived-at: Tue, 02 May 2017 15:53:05 -0000 Github user aljoscha commented on a diff in the pull request: https://github.com/apache/flink/pull/3778#discussion_r114353390 --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java --- @@ -0,0 +1,491 @@ +/* + * 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.connectors.kafka; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyMapOf; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles; +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OperatorSnapshotUtil; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.util.SerializedValue; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * Tests for checking whether {@link FlinkKafkaConsumerBase} can restore from snapshots that were + * done using the Flink 1.2 {@link FlinkKafkaConsumerBase}. + * + *

For regenerating the binary snapshot files run {@link #writeSnapshot()} on the Flink 1.2 + * branch. + */ +public class FlinkKafkaConsumerBaseFrom12MigrationTest { + + + /** + * Manually run this to write binary snapshot data. + */ + @Ignore + @Test + public void writeSnapshot() throws Exception { + final HashMap state = new HashMap<>(); + state.put(new KafkaTopicPartition("abc", 13), 16768L); + state.put(new KafkaTopicPartition("def", 7), 987654321L); + writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", state); + + final HashMap emptyState = new HashMap<>(); + writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot-empty-state", emptyState); + } + + private void writeSnapshot(String path, HashMap state) throws Exception { + + final OneShotLatch latch = new OneShotLatch(); + final AbstractFetcher fetcher = mock(AbstractFetcher.class); + + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + latch.trigger(); + return null; + } + }).when(fetcher).runFetchLoop(); + + when(fetcher.snapshotCurrentState()).thenReturn(state); + + final List partitions = new ArrayList<>(); + partitions.add(new KafkaTopicPartition("abc", 13)); + partitions.add(new KafkaTopicPartition("def", 7)); + + final DummyFlinkKafkaConsumer consumerFunction = new DummyFlinkKafkaConsumer<>( + new FetcherFactory() { + private static final long serialVersionUID = -2803131905656983619L; + + @Override + public AbstractFetcher createFetcher() { + return fetcher; + } + }, + partitions); + + StreamSource> consumerOperator = + new StreamSource<>(consumerFunction); + + + final AbstractStreamOperatorTestHarness testHarness = + new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0); + + testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); + + testHarness.setup(); + testHarness.open(); + + final Throwable[] error = new Throwable[1]; + + // run the source asynchronously + Thread runner = new Thread() { + @Override + public void run() { + try { + consumerFunction.run(new DummySourceContext() { + @Override + public void collect(String element) { + latch.trigger(); + } + }); + } + catch (Throwable t) { + t.printStackTrace(); + error[0] = t; + } + } + }; + runner.start(); + + if (!latch.isTriggered()) { + latch.await(); + } + + final OperatorStateHandles snapshot; + synchronized (testHarness.getCheckpointLock()) { + snapshot = testHarness.snapshot(0L, 0L); + } + + OperatorSnapshotUtil.writeStateHandle(snapshot, path); + + consumerOperator.close(); + runner.join(); + } + + @Test + public void testRestoreWithEmptyStateNoPartitions() throws Exception { + // -------------------------------------------------------------------- + // prepare fake states + // -------------------------------------------------------------------- + + final OneShotLatch latch = new OneShotLatch(); + final AbstractFetcher fetcher = mock(AbstractFetcher.class); + + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) throws Throwable { + latch.trigger(); + Assert.fail("This should never be called"); + return null; + } + }).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class)); + + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + latch.trigger(); + Assert.fail("This should never be called"); + return null; + } + }).when(fetcher).runFetchLoop(); + + final DummyFlinkKafkaConsumer consumerFunction = new DummyFlinkKafkaConsumer<>( + new FetcherFactory() { + private static final long serialVersionUID = -2803131905656983619L; + + @Override + public AbstractFetcher createFetcher() { + return fetcher; + } + }, + Collections.emptyList()); + + StreamSource> consumerOperator = + new StreamSource<>(consumerFunction); + + final AbstractStreamOperatorTestHarness testHarness = + new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0); + + testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); + + testHarness.setup(); + testHarness.initializeState( + OperatorSnapshotUtil.readStateHandle( + OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state"))); + testHarness.open(); + + final Throwable[] error = new Throwable[1]; + + // run the source asynchronously + Thread runner = new Thread() { + @Override + public void run() { + try { + consumerFunction.run(new DummySourceContext() { + @Override + public void collect(String element) { + latch.trigger(); + Assert.fail("This should never be called."); + } + + @Override + public void emitWatermark(Watermark mark) { + latch.trigger(); + assertEquals(Long.MAX_VALUE, mark.getTimestamp()); + } + }); + } + catch (Throwable t) { + t.printStackTrace(); + error[0] = t; + } + } + }; + runner.start(); + + if (!latch.isTriggered()) { + latch.await(); + } + + consumerOperator.cancel(); + consumerOperator.close(); + + runner.interrupt(); + runner.join(); + + assertNull(error[0]); + } + + @Test + public void testRestoreWithEmptyStateWithPartitions() throws Exception { + final OneShotLatch latch = new OneShotLatch(); + final AbstractFetcher fetcher = mock(AbstractFetcher.class); + + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) throws Throwable { + latch.trigger(); + Assert.fail("This should never be called"); + return null; + } + }).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class)); + + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + latch.trigger(); + return null; + } + }).when(fetcher).runFetchLoop(); + + final List partitions = new ArrayList<>(); + partitions.add(new KafkaTopicPartition("abc", 13)); + partitions.add(new KafkaTopicPartition("def", 7)); + + final DummyFlinkKafkaConsumer consumerFunction = new DummyFlinkKafkaConsumer<>( + new FetcherFactory() { + private static final long serialVersionUID = -2803131905656983619L; + + @Override + public AbstractFetcher createFetcher() { + return fetcher; + } + }, + partitions); + + StreamSource> consumerOperator = + new StreamSource<>(consumerFunction); + + final AbstractStreamOperatorTestHarness testHarness = + new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0); + + testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); + + testHarness.setup(); + testHarness.initializeState( + OperatorSnapshotUtil.readStateHandle( + OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state"))); + testHarness.open(); + + final Throwable[] error = new Throwable[1]; + + // run the source asynchronously + Thread runner = new Thread() { + @Override + public void run() { + try { + consumerFunction.run(new DummySourceContext() { + @Override + public void collect(String element) { + latch.trigger(); + Assert.fail("This should never be called."); + } + + @Override + public void emitWatermark(Watermark mark) { + latch.trigger(); + assertEquals(Long.MAX_VALUE, mark.getTimestamp()); + } + }); + } + catch (Throwable t) { + t.printStackTrace(); + error[0] = t; + } + } + }; + runner.start(); + + if (!latch.isTriggered()) { + latch.await(); + } + + consumerOperator.close(); + runner.interrupt(); + runner.join(); + + assertNull(error[0]); + } + + @Test + public void testRestore() throws Exception { + // -------------------------------------------------------------------- + // prepare fake states + // -------------------------------------------------------------------- + + final HashMap state1 = new HashMap<>(); + state1.put(new KafkaTopicPartition("abc", 13), 16768L); + state1.put(new KafkaTopicPartition("def", 7), 987654321L); + + final boolean[] verifiedState = new boolean[1]; + + final OneShotLatch latch = new OneShotLatch(); + final AbstractFetcher fetcher = mock(AbstractFetcher.class); + + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) throws Throwable { + Map map = (HashMap) invocationOnMock.getArguments()[0]; + + latch.trigger(); + assertEquals(state1, map); + verifiedState[0] = true; + return null; + } + }).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class)); + + + final List partitions = new ArrayList<>(); + partitions.add(new KafkaTopicPartition("abc", 13)); + partitions.add(new KafkaTopicPartition("def", 7)); + + final DummyFlinkKafkaConsumer consumerFunction = new DummyFlinkKafkaConsumer<>( + new FetcherFactory() { + private static final long serialVersionUID = -2803131905656983619L; + + @Override + public AbstractFetcher createFetcher() { + return fetcher; + } + }, + partitions); + + StreamSource> consumerOperator = + new StreamSource<>(consumerFunction); + + final AbstractStreamOperatorTestHarness testHarness = + new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0); + + testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); + + testHarness.setup(); + testHarness.initializeState( + OperatorSnapshotUtil.readStateHandle( + OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot"))); + testHarness.open(); + + final Throwable[] error = new Throwable[1]; + + // run the source asynchronously + Thread runner = new Thread() { + @Override + public void run() { + try { + consumerFunction.run(new DummySourceContext() { + @Override + public void collect(String element) { + //latch.trigger(); --- End diff -- Fixing --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastructure@apache.org or file a JIRA ticket with INFRA. ---