Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion bin/kafka-run-class.sh
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ do
CLASSPATH=$CLASSPATH:$dir/*
done

for cc_pkg in "api" "runtime" "file" "json"
for cc_pkg in "api" "runtime" "file" "json" "tools"
do
for file in $base_dir/connect/${cc_pkg}/build/libs/connect-${cc_pkg}*.jar;
do
Expand Down
65 changes: 64 additions & 1 deletion build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -230,7 +230,7 @@ for ( sv in ['2_10_5', '2_11_7'] ) {
}
}

def connectPkgs = ['connect:api', 'connect:runtime', 'connect:json', 'connect:file']
def connectPkgs = ['connect:api', 'connect:runtime', 'connect:json', 'connect:file', 'connect:tools']
def pkgs = ['clients', 'examples', 'log4j-appender', 'tools', 'streams'] + connectPkgs

tasks.create(name: "jarConnect", dependsOn: connectPkgs.collect { it + ":jar" }) {}
Expand Down Expand Up @@ -350,6 +350,8 @@ project(':core') {
from(project(':connect:json').configurations.runtime) { into("libs/") }
from(project(':connect:file').jar) { into("libs/") }
from(project(':connect:file').configurations.runtime) { into("libs/") }
from(project(':connect:tools').jar) { into("libs/") }
from(project(':connect:tools').configurations.runtime) { into("libs/") }
}

jar {
Expand Down Expand Up @@ -887,3 +889,64 @@ project(':connect:file') {
}
test.dependsOn('checkstyleMain', 'checkstyleTest')
}

project(':connect:tools') {
apply plugin: 'checkstyle'
archivesBaseName = "connect-tools"

dependencies {
compile project(':connect:api')
compile "$slf4japi"
compile "com.fasterxml.jackson.core:jackson-databind:$jackson_version"

testCompile "$junit"
testCompile "$easymock"
testCompile "$powermock"
testCompile "$powermock_easymock"
testRuntime "$slf4jlog4j"
}

task testJar(type: Jar) {
classifier = 'test'
from sourceSets.test.output
}

test {
testLogging {
events "passed", "skipped", "failed"
exceptionFormat = 'full'
}
}

javadoc {
include "**/org/apache/kafka/connect/*"
}

tasks.create(name: "copyDependantLibs", type: Copy) {
from (configurations.testRuntime) {
include('slf4j-log4j12*')
}
from (configurations.runtime) {
exclude('kafka-clients*')
exclude('connect-*')
}
into "$buildDir/dependant-libs"
}

jar {
dependsOn copyDependantLibs
}

artifacts {
archives testJar
}

configurations {
archives.extendsFrom(testCompile)
}

checkstyle {
configFile = new File(rootDir, "checkstyle/checkstyle.xml")
}
test.dependsOn('checkstyleMain', 'checkstyleTest')
}
4 changes: 4 additions & 0 deletions checkstyle/import-control.xml
Original file line number Diff line number Diff line change
Expand Up @@ -205,6 +205,10 @@
<allow pkg="org.powermock" />
</subpackage>

<subpackage name="tools">
<allow pkg="org.apache.kafka.connect" />
<allow pkg="com.fasterxml.jackson" />
</subpackage>
</subpackage>

</import-control>
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.kafka.tools;
package org.apache.kafka.common.utils;


/**
Expand Down Expand Up @@ -46,6 +46,7 @@ public class ThroughputThrottler {
long sleepDeficitNs = 0;
long targetThroughput = -1;
long startMs;
private boolean wakeup = false;

/**
* @param targetThroughput Can be messages/sec or bytes/sec
Expand Down Expand Up @@ -83,7 +84,11 @@ public boolean shouldThrottle(long amountSoFar, long sendStartMs) {
public void throttle() {
if (targetThroughput == 0) {
try {
Thread.sleep(Long.MAX_VALUE);
synchronized (this) {
while (!wakeup) {
this.wait();
}
}
} catch (InterruptedException e) {
// do nothing
}
Expand All @@ -96,12 +101,21 @@ public void throttle() {

// If enough sleep deficit has accumulated, sleep a little
if (sleepDeficitNs >= MIN_SLEEP_NS) {
long sleepMs = sleepDeficitNs / 1000000;
long sleepNs = sleepDeficitNs - sleepMs * 1000000;

long sleepStartNs = System.nanoTime();
long currentTimeNs = sleepStartNs;
try {
Thread.sleep(sleepMs, (int) sleepNs);
synchronized (this) {
long elapsed = currentTimeNs - sleepStartNs;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this correct? currentTimeNs is never updated and hence always the same as sleepStartNs.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, it was not correct. Good catch! I've fixed it and validated correctness by running ProducerPerformance manually.

long remaining = sleepDeficitNs - elapsed;
while (!wakeup && remaining > 0) {
long sleepMs = remaining / 1000000;
long sleepNs = remaining - sleepMs * 1000000;
this.wait(sleepMs, (int) sleepNs);
elapsed = System.nanoTime() - sleepStartNs;
remaining = sleepDeficitNs - elapsed;
}
wakeup = false;
}
sleepDeficitNs = 0;
} catch (InterruptedException e) {
// If sleep is cut short, reduce deficit by the amount of
Expand All @@ -113,5 +127,15 @@ public void throttle() {
}
}
}

/**
* Wakeup the throttler if its sleeping.
*/
public void wakeup() {
synchronized (this) {
wakeup = true;
this.notifyAll();
}
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,11 @@ public SourceRecord(Map<String, ?> sourcePartition, Map<String, ?> sourceOffset,
this(sourcePartition, sourceOffset, topic, null, null, null, valueSchema, value);
}

public SourceRecord(Map<String, ?> sourcePartition, Map<String, ?> sourceOffset,
String topic, Schema keySchema, Object key, Schema valueSchema, Object value) {
this(sourcePartition, sourceOffset, topic, null, keySchema, key, valueSchema, value);
}

public SourceRecord(Map<String, ?> sourcePartition, Map<String, ?> sourceOffset,
String topic, Integer partition,
Schema keySchema, Object key, Schema valueSchema, Object value) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -230,6 +230,13 @@ public WorkerConfig workerConfig() {
return workerConfig;
}

@Override
public String toString() {
return "WorkerSinkTask{" +
"id=" + id +
'}';
}

private KafkaConsumer<byte[], byte[]> createConsumer() {
// Include any unknown worker configs so consumer configs can be set globally on the worker
// and through to the task
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ public void iteration() {
long commitTimeout = commitStarted + task.workerConfig().getLong(
WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG);
if (committing && now >= commitTimeout) {
log.warn("Commit of {} offsets timed out", this);
log.warn("Commit of {} offsets timed out", task);
commitFailures++;
committing = false;
}
Expand All @@ -98,11 +98,11 @@ public void onCommitCompleted(Throwable error, long seqno) {
seqno, commitSeqno);
} else {
if (error != null) {
log.error("Commit of {} offsets threw an unexpected exception: ", this, error);
log.error("Commit of {} offsets threw an unexpected exception: ", task, error);
commitFailures++;
} else {
log.debug("Finished {} offset commit successfully in {} ms",
this, task.time().milliseconds() - commitStarted);
task, task.time().milliseconds() - commitStarted);
commitFailures = 0;
}
committing = false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -178,6 +178,8 @@ private synchronized void recordSent(final ProducerRecord<byte[], byte[]> record
public boolean commitOffsets() {
long commitTimeoutMs = workerConfig.getLong(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG);

log.debug("{} Committing offsets", this);

long started = time.milliseconds();
long timeout = started + commitTimeoutMs;

Expand Down Expand Up @@ -259,7 +261,7 @@ public void onCompletion(Throwable error, Void result) {
}

finishSuccessfulFlush();
log.debug("Finished {} commitOffsets successfully in {} ms",
log.info("Finished {} commitOffsets successfully in {} ms",
this, time.milliseconds() - started);
return true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,8 @@ public void start(Herder herder) {
}

public void stop() {
log.info("Stopping REST server");

try {
jettyServer.stop();
jettyServer.join();
Expand All @@ -141,6 +143,8 @@ public void stop() {
} finally {
jettyServer.destroy();
}

log.info("REST server stopped");
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@
import org.apache.kafka.connect.runtime.rest.entities.TaskInfo;
import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException;
import org.apache.kafka.connect.util.FutureCallback;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.servlet.ServletContext;
import javax.ws.rs.Consumes;
Expand All @@ -51,6 +53,8 @@
@Produces(MediaType.APPLICATION_JSON)
@Consumes(MediaType.APPLICATION_JSON)
public class ConnectorsResource {
private static final Logger log = LoggerFactory.getLogger(ConnectorsResource.class);

// TODO: This should not be so long. However, due to potentially long rebalances that may have to wait a full
// session timeout to complete, during which we cannot serve some requests. Ideally we could reduce this, but
// we need to consider all possible scenarios this could fail. It might be ok to fail with a timeout in rare cases,
Expand Down Expand Up @@ -159,7 +163,9 @@ private <T, U> T completeOrForwardRequest(
} catch (ExecutionException e) {
if (e.getCause() instanceof NotLeaderException) {
NotLeaderException notLeaderError = (NotLeaderException) e.getCause();
return translator.translate(RestServer.httpRequest(RestServer.urlJoin(notLeaderError.leaderUrl(), path), method, body, resultType));
String forwardUrl = RestServer.urlJoin(notLeaderError.leaderUrl(), path);
log.debug("Forwarding request to leader: {} {} {}", forwardUrl, method, body);
return translator.translate(RestServer.httpRequest(forwardUrl, method, body, resultType));
}

throw e.getCause();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,7 @@ public void stop() {
* @param callback the callback to invoke once the end of the log has been reached.
*/
public void readToEnd(Callback<Void> callback) {
log.trace("Starting read to end log for topic {}", topic);
producer.flush();
synchronized (this) {
readLogEndOffsetCallbacks.add(callback);
Expand Down Expand Up @@ -286,6 +287,10 @@ private void readToLogEnd() {


private class WorkThread extends Thread {
public WorkThread() {
super("KafkaBasedLog Work Thread - " + topic);
}

@Override
public void run() {
try {
Expand All @@ -300,6 +305,7 @@ public void run() {
if (numCallbacks > 0) {
try {
readToLogEnd();
log.trace("Finished read to end log for topic {}", topic);
} catch (WakeupException e) {
// Either received another get() call and need to retry reading to end of log or stop() was
// called. Both are handled by restarting this loop.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/**
* 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.kafka.connect.tools;

import org.apache.kafka.common.utils.AppInfoParser;
import org.apache.kafka.connect.connector.Task;
import org.apache.kafka.connect.source.SourceConnector;

import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

/**
* @see VerifiableSinkTask
*/
public class VerifiableSinkConnector extends SourceConnector {
private Map<String, String> config;

@Override
public String version() {
return AppInfoParser.getVersion();
}

@Override
public void start(Map<String, String> props) {
this.config = props;
}

@Override
public Class<? extends Task> taskClass() {
return VerifiableSinkTask.class;
}

@Override
public List<Map<String, String>> taskConfigs(int maxTasks) {
ArrayList<Map<String, String>> configs = new ArrayList<>();
for (Integer i = 0; i < maxTasks; i++) {
Map<String, String> props = new HashMap<>(config);
props.put(VerifiableSinkTask.ID_CONFIG, i.toString());
configs.add(props);
}
return configs;
}

@Override
public void stop() {
}
}
Loading