Skip to content

Commit f7976d2

Browse files
mjsaxewencp
authored andcommitted
KAFKA-4008: Module "tools" should not be dependent on "core"
moved streams application reset tool from tools to core Author: Matthias J. Sax <[email protected]> Reviewers: Ismael Juma <[email protected]>, Damian Guy <[email protected]>, Guozhang Wang <[email protected]>, Ewen Cheslack-Postava <[email protected]> Closes apache#1685 from mjsax/moveResetTool (cherry picked from commit f2405a7) Signed-off-by: Ewen Cheslack-Postava <[email protected]>
1 parent 2a9f7af commit f7976d2

File tree

6 files changed

+21
-15
lines changed

6 files changed

+21
-15
lines changed

bin/kafka-streams-application-reset.sh

+1-1
Original file line numberDiff line numberDiff line change
@@ -18,4 +18,4 @@ if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then
1818
export KAFKA_HEAP_OPTS="-Xmx512M"
1919
fi
2020

21-
exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.StreamsResetter "$@"
21+
exec $(dirname $0)/kafka-run-class.sh kafka.tools.StreamsResetter "$@"

build.gradle

-2
Original file line numberDiff line numberDiff line change
@@ -641,7 +641,6 @@ project(':tools') {
641641
archivesBaseName = "kafka-tools"
642642

643643
dependencies {
644-
compile project(':core')
645644
compile project(':clients')
646645
compile project(':log4j-appender')
647646
compile libs.argparse4j
@@ -690,7 +689,6 @@ project(':streams') {
690689
testCompile project(':clients').sourceSets.test.output
691690
testCompile project(':core')
692691
testCompile project(':core').sourceSets.test.output
693-
testCompile project(':tools')
694692
testCompile libs.junit
695693

696694
testRuntime libs.slf4jlog4j

checkstyle/import-control-core.xml

+2
Original file line numberDiff line numberDiff line change
@@ -53,10 +53,12 @@
5353
</subpackage>
5454

5555
<subpackage name="tools">
56+
<allow pkg="kafka.admin" />
5657
<allow pkg="kafka.javaapi" />
5758
<allow pkg="kafka.producer" />
5859
<allow pkg="kafka.consumer" />
5960
<allow pkg="joptsimple" />
61+
<allow pkg="org.apache.kafka.clients.consumer" />
6062
</subpackage>
6163

6264
<subpackage name="examples">

checkstyle/import-control.xml

+1-3
Original file line numberDiff line numberDiff line change
@@ -123,8 +123,6 @@
123123
<allow pkg="com.fasterxml.jackson" />
124124
<allow pkg="net.sourceforge.argparse4j" />
125125
<allow pkg="org.apache.log4j" />
126-
<allow pkg="joptsimple" />
127-
<allow pkg="kafka" />
128126
</subpackage>
129127

130128
<subpackage name="streams">
@@ -144,14 +142,14 @@
144142
<subpackage name="integration">
145143
<allow pkg="kafka.admin" />
146144
<allow pkg="kafka.server" />
145+
<allow pkg="kafka.tools" />
147146
<allow pkg="kafka.utils" />
148147
<allow pkg="kafka.zk" />
149148
<allow pkg="kafka.log" />
150149
<allow pkg="scala" />
151150
<allow pkg="scala.collection" />
152151
<allow pkg="org.I0Itec.zkclient" />
153152
<allow pkg="org.hamcrest" />
154-
<allow pkg="org.apache.kafka.tools" />
155153
</subpackage>
156154

157155
<subpackage name="state">

tools/src/main/java/org/apache/kafka/tools/StreamsResetter.java core/src/main/scala/kafka/tools/StreamsResetter.java

+16-8
Original file line numberDiff line numberDiff line change
@@ -1,16 +1,20 @@
11
/**
2-
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
3-
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
4-
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
5-
* License. You may obtain a copy of the License at
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
68
* <p>
79
* http://www.apache.org/licenses/LICENSE-2.0
810
* <p>
9-
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
10-
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
11-
* specific language governing permissions and limitations under the License.
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
1216
*/
13-
package org.apache.kafka.tools;
17+
package kafka.tools;
1418

1519
import joptsimple.OptionException;
1620
import joptsimple.OptionParser;
@@ -21,6 +25,7 @@
2125
import org.apache.kafka.clients.consumer.ConsumerConfig;
2226
import org.apache.kafka.clients.consumer.KafkaConsumer;
2327
import org.apache.kafka.common.TopicPartition;
28+
import org.apache.kafka.common.annotation.InterfaceStability;
2429
import org.apache.kafka.common.security.JaasUtils;
2530
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
2631

@@ -34,6 +39,8 @@
3439
/**
3540
* {@link StreamsResetter} resets the processing state of a Kafka Streams application so that, for example, you can reprocess its input from scratch.
3641
* <p>
42+
* <strong>This class is not part of public API. For backward compatibility, use the provided script in "bin/" instead of calling this class directly from your code.</strong>
43+
* <p>
3744
* Resetting the processing state of an application includes the following actions:
3845
* <ol>
3946
* <li>setting the application's consumer offsets for input and internal topics to zero</li>
@@ -50,6 +57,7 @@
5057
* User output topics will not be deleted or modified by this tool.
5158
* If downstream applications consume intermediate or output topics, it is the user's responsibility to adjust those applications manually if required.
5259
*/
60+
@InterfaceStability.Unstable
5361
public class StreamsResetter {
5462
private static final int EXIT_CODE_SUCCESS = 0;
5563
private static final int EXIT_CODE_ERROR = 1;

streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@
1616
*/
1717
package org.apache.kafka.streams.integration;
1818

19+
import kafka.tools.StreamsResetter;
1920
import kafka.utils.ZkUtils;
2021
import org.apache.kafka.clients.consumer.ConsumerConfig;
2122
import org.apache.kafka.clients.producer.ProducerConfig;
@@ -37,7 +38,6 @@
3738
import org.apache.kafka.streams.kstream.TimeWindows;
3839
import org.apache.kafka.streams.kstream.Windowed;
3940
import org.apache.kafka.test.TestUtils;
40-
import org.apache.kafka.tools.StreamsResetter;
4141
import org.junit.Assert;
4242
import org.junit.BeforeClass;
4343
import org.junit.ClassRule;

0 commit comments

Comments
 (0)