From jira-return-10487-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Wed Mar 7 02:50:07 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 5FB15180652 for ; Wed, 7 Mar 2018 02:50:06 +0100 (CET) Received: (qmail 11286 invoked by uid 500); 7 Mar 2018 01:50:05 -0000 Mailing-List: contact jira-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: jira@kafka.apache.org Delivered-To: mailing list jira@kafka.apache.org Received: (qmail 11274 invoked by uid 99); 7 Mar 2018 01:50:05 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 07 Mar 2018 01:50:05 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id CFF2B1A02F1 for ; Wed, 7 Mar 2018 01:50:04 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -101.511 X-Spam-Level: X-Spam-Status: No, score=-101.511 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id IHOSfeZaX9GN for ; Wed, 7 Mar 2018 01:50:02 +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 BBC5A5F588 for ; Wed, 7 Mar 2018 01:50: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 0C919E02F7 for ; Wed, 7 Mar 2018 01:50:01 +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 56E5025402 for ; Wed, 7 Mar 2018 01:50:00 +0000 (UTC) Date: Wed, 7 Mar 2018 01:50:00 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: jira@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (KAFKA-5660) Don't throw TopologyBuilderException during runtime MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/KAFKA-5660?page=3Dcom.atlassian= .jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D1638= 8870#comment-16388870 ]=20 ASF GitHub Bot commented on KAFKA-5660: --------------------------------------- guozhangwang closed pull request #4645: KAFKA-5660 Don't throw TopologyBuil= derException during runtime URL: https://github.com/apache/kafka/pull/4645 =20 =20 =20 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/inter= nals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/str= eams/processor/internals/ProcessorContextImpl.java index 317581a6beb..42d3d70e396 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Pr= ocessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Pr= ocessorContextImpl.java @@ -22,6 +22,7 @@ import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.state.internals.ThreadCache; =20 @@ -54,13 +55,12 @@ public RecordCollector recordCollector() { } =20 /** - * @throws org.apache.kafka.streams.errors.TopologyBuilderException if= an attempt is made to access this state store from an unknown node + * @throws StreamsException if an attempt is made to access this state= store from an unknown node */ - @SuppressWarnings("deprecation") @Override public StateStore getStateStore(final String name) { if (currentNode() =3D=3D null) { - throw new org.apache.kafka.streams.errors.TopologyBuilderExcep= tion("Accessing from an unknown node"); + throw new StreamsException("Accessing from an unknown node"); } =20 final StateStore global =3D stateManager.getGlobalStore(name); @@ -69,7 +69,7 @@ public StateStore getStateStore(final String name) { } =20 if (!currentNode().stateStores.contains(name)) { - throw new org.apache.kafka.streams.errors.TopologyBuilderExcep= tion("Processor " + currentNode().name() + " has no access to StateStore " = + name); + throw new StreamsException("Processor " + currentNode().name()= + " has no access to StateStore " + name); } =20 return stateManager.getStore(name); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/inter= nals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka= /streams/processor/internals/StreamsPartitionAssignor.java index 9aa0e94c8c1..d4e7a5d5c62 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/St= reamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/St= reamsPartitionAssignor.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskAssignmentException; import org.apache.kafka.streams.processor.PartitionGrouper; import org.apache.kafka.streams.processor.TaskId; @@ -644,7 +645,7 @@ private void prepareTopic(final Map topicPartitio continue; } if (numPartitions < 0) { - throw new org.apache.kafka.streams.errors.TopologyBuilderE= xception(String.format("%sTopic [%s] number of partitions not defined", log= Prefix, topic.name())); + throw new StreamsException(String.format("%sTopic [%s] num= ber of partitions not defined", logPrefix, topic.name())); } =20 topic.setNumberOfPartitions(numPartitions); diff --git a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.ja= va b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java index 992ffd818ff..68340910c2b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams; =20 import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.errors.TopologyBuilderException; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; @@ -252,7 +251,7 @@ public void shouldNotAllowToAddStoreWithSameName() { } catch (final TopologyException expected) { } } =20 - @Test(expected =3D TopologyBuilderException.class) + @Test public void shouldThrowOnUnassignedStateStoreAccess() throws Exception= { final String sourceNodeName =3D "source"; final String goodNodeName =3D "goodGuy"; @@ -276,15 +275,12 @@ public void shouldThrowOnUnassignedStateStoreAccess()= throws Exception { =20 try { new ProcessorTopologyTestDriver(streamsConfig, topology.intern= alTopologyBuilder); + fail("Should have thrown StreamsException"); } catch (final StreamsException e) { - final Throwable cause =3D e.getCause(); - if (cause !=3D null - && cause instanceof TopologyBuilderException - && cause.getMessage().equals("Invalid topology building: P= rocessor " + badNodeName + " has no access to StateStore " + LocalMockProce= ssorSupplier.STORE_NAME)) { - throw (TopologyBuilderException) cause; - } else { - throw new RuntimeException("Did expect different exception= . Did catch:", e); - } + final String error =3D e.toString(); + final String expectedMessage =3D "org.apache.kafka.streams.err= ors.StreamsException: failed to initialize processor " + badNodeName; + =20 + assertThat(error, equalTo(expectedMessage)); } } =20 diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/Topol= ogyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/proces= sor/TopologyBuilderTest.java index 7a815944ecd..f67b6341f8d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuil= derTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuil= derTest.java @@ -51,6 +51,7 @@ =20 import static org.apache.kafka.common.utils.Utils.mkList; import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -590,8 +591,7 @@ public void shouldAddInternalTopicConfigForRepartitionT= opics() { assertEquals("appId-foo", topicConfig.name()); } =20 - - @Test(expected =3D TopologyBuilderException.class) + @Test public void shouldThroughOnUnassignedStateStoreAccess() throws Excepti= on { final String sourceNodeName =3D "source"; final String goodNodeName =3D "goodGuy"; @@ -603,27 +603,24 @@ public void shouldThroughOnUnassignedStateStoreAccess= () throws Exception { config.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory= ().getAbsolutePath()); final StreamsConfig streamsConfig =3D new StreamsConfig(config); =20 - try { - final TopologyBuilder builder =3D new TopologyBuilder(); - builder - .addSource(sourceNodeName, "topic") - .addProcessor(goodNodeName, new LocalMockProcessorSupplier= (), sourceNodeName) + final TopologyBuilder builder =3D new TopologyBuilder(); + builder.addSource(sourceNodeName, "topic") + .addProcessor(goodNodeName, new LocalMockProcessorSupplier= (), + sourceNodeName) .addStateStore( - Stores.create(LocalMockProcessorSupplier.STORE_NAME).w= ithStringKeys().withStringValues().inMemory().build(), - goodNodeName) - .addProcessor(badNodeName, new LocalMockProcessorSupplier(= ), sourceNodeName); - + Stores.create(LocalMockProcessorSupplier.STORE_NAM= E) + .withStringKeys().withStringValues().inMem= ory() + .build(), goodNodeName) + .addProcessor(badNodeName, new LocalMockProcessorSupplier(= ), + sourceNodeName); =20 + try { final ProcessorTopologyTestDriver driver =3D new ProcessorTopo= logyTestDriver(streamsConfig, builder.internalTopologyBuilder); - driver.process("topic", null, null); + fail("Should have thrown StreamsException"); } catch (final StreamsException e) { - final Throwable cause =3D e.getCause(); - if (cause !=3D null - && cause instanceof TopologyBuilderException - && cause.getMessage().equals("Invalid topology building: P= rocessor " + badNodeName + " has no access to StateStore " + LocalMockProce= ssorSupplier.STORE_NAME)) { - throw (TopologyBuilderException) cause; - } else { - throw new RuntimeException("Did expect different exception= . Did catch:", e); - } + final String error =3D e.toString(); + final String expectedMessage =3D "org.apache.kafka.streams.err= ors.StreamsException: failed to initialize processor " + badNodeName; + + assertThat(error, equalTo(expectedMessage)); } } =20 diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/inter= nals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/ka= fka/streams/processor/internals/InternalTopologyBuilderTest.java index a39e545f513..901fc4b7e0a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/In= ternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/In= ternalTopologyBuilderTest.java @@ -23,7 +23,6 @@ import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.TopologyDescription; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.errors.TopologyBuilderException; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; @@ -53,7 +52,9 @@ =20 import static org.apache.kafka.common.utils.Utils.mkList; import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.core.IsInstanceOf.instanceOf; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -578,17 +579,15 @@ public void shouldThrowOnUnassignedStateStoreAccess()= { Stores.create(LocalMockProcessorSupplier.STORE_NAME).withStrin= gKeys().withStringValues().inMemory().build(), goodNodeName); builder.addProcessor(badNodeName, new LocalMockProcessorSupplier()= , sourceNodeName); - + =20 try { new ProcessorTopologyTestDriver(streamsConfig, builder); fail("Should have throw StreamsException"); - } catch (final StreamsException expected) { - final Throwable cause =3D expected.getCause(); - if (cause =3D=3D null - || !(cause instanceof TopologyBuilderException) - || !cause.getMessage().equals("Invalid topology building: = Processor " + badNodeName + " has no access to StateStore " + LocalMockProc= essorSupplier.STORE_NAME)) { - throw new RuntimeException("Did expect different exception= . Did catch:", expected); - } + } catch (final StreamsException e) { + final String error =3D e.toString(); + final String expectedMessage =3D "org.apache.kafka.streams.err= ors.StreamsException: failed to initialize processor " + badNodeName; + =20 + assertThat(error, equalTo(expectedMessage)); } } =20 =20 ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. =20 For queries about this service, please contact Infrastructure at: users@infra.apache.org > Don't throw TopologyBuilderException during runtime > --------------------------------------------------- > > Key: KAFKA-5660 > URL: https://issues.apache.org/jira/browse/KAFKA-5660 > Project: Kafka > Issue Type: Task > Components: streams > Affects Versions: 0.11.0.0 > Reporter: Matthias J. Sax > Assignee: Nick Afshartous > Priority: Major > Fix For: 1.2.0 > > > {{TopologyBuilderException}} is a pre-runtime exception that should only = be thrown before=C2=A0{{KafkaStreams#start()}} is called. > However, we do throw {{TopologyBuilderException}} within > - `SourceNodeFactory#getTopics` > - `ProcessorContextImpl#getStateStore` > - `StreamPartitionAssignor#prepareTopic ` > (and maybe somewhere else: we should double check if there are other plac= es in the code like those). > We should replace those exception with either {{StreamsException}} or wit= h a new exception type. -- This message was sent by Atlassian JIRA (v7.6.3#76005)