This is an automated email from the ASF dual-hosted git repository.
davidlim pushed a commit to branch 0.13.0-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git
The following commit(s) were added to refs/heads/0.13.0-incubating by this push:
new c1d2f55 fix TaskRunnerUtils String formatting issue (#6492)
c1d2f55 is described below
commit c1d2f55240162517c8903bafc085a1d2d090d2e4
Author: Joshua Sun <[email protected]>
AuthorDate: Thu Oct 18 18:16:46 2018 -0700
fix TaskRunnerUtils String formatting issue (#6492)
* fix TaskRunnerUtils String formatting issue
* additional fixes
---
.../druid/indexing/overlord/RemoteTaskRunner.java | 4 ++--
.../druid/indexing/overlord/TaskRunnerUtils.java | 4 ++--
.../indexing/overlord/hrtr/HttpRemoteTaskRunner.java | 19 ++++++++++++-------
3 files changed, 16 insertions(+), 11 deletions(-)
diff --git
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java
index 16b75ed..9480bf3 100644
---
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java
+++
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java
@@ -599,9 +599,9 @@ public class RemoteTaskRunner implements WorkerTaskRunner,
TaskLogStreamer
// Worker is still running this task
final URL url = TaskRunnerUtils.makeWorkerURL(
zkWorker.getWorker(),
- "/druid/worker/v1/task/%s/log?offset=%d",
+ "/druid/worker/v1/task/%s/log?offset=%s",
taskId,
- offset
+ Long.toString(offset)
);
return Optional.of(
new ByteSource()
diff --git
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunnerUtils.java
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunnerUtils.java
index 176a3a1..6352b86 100644
---
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunnerUtils.java
+++
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunnerUtils.java
@@ -99,12 +99,12 @@ public class TaskRunnerUtils
}
}
- public static URL makeWorkerURL(Worker worker, String pathFormat, Object...
pathParams)
+ public static URL makeWorkerURL(Worker worker, String pathFormat, String...
pathParams)
{
Preconditions.checkArgument(pathFormat.startsWith("/"), "path must start
with '/': %s", pathFormat);
final String path = StringUtils.format(
pathFormat,
- Arrays.stream(pathParams).map(s ->
StringUtils.urlEncode(s.toString())).toArray()
+ Arrays.stream(pathParams).map(StringUtils::urlEncode).toArray()
);
try {
diff --git
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java
index 2047d53..f93acbc 100644
---
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java
+++
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java
@@ -106,12 +106,12 @@ import java.util.stream.Collectors;
* A Remote TaskRunner to manage tasks on Middle Manager nodes using
internal-discovery({@link DruidNodeDiscoveryProvider})
* to discover them and Http.
* Middle Managers expose 3 HTTP endpoints
- * 1. POST request for assigning a task
- * 2. POST request for shutting down a task
- * 3. GET request for getting list of assigned, running, completed tasks on
Middle Manager and its enable/disable status.
- * This endpoint is implemented to support long poll and holds the request
till there is a change. This class
- * sends the next request immediately as the previous finishes to keep the
state up-to-date.
- *
+ * 1. POST request for assigning a task
+ * 2. POST request for shutting down a task
+ * 3. GET request for getting list of assigned, running, completed tasks on
Middle Manager and its enable/disable status.
+ * This endpoint is implemented to support long poll and holds the request
till there is a change. This class
+ * sends the next request immediately as the previous finishes to keep the
state up-to-date.
+ * <p>
* ZK_CLEANUP_TODO : As of 0.11.1, it is required to cleanup task status paths
from ZK which are created by the
* workers to support deprecated RemoteTaskRunner. So a method
"scheduleCompletedTaskStatusCleanupFromZk()" is added'
* which should be removed in the release that removes RemoteTaskRunner legacy
ZK updation WorkerTaskMonitor class.
@@ -857,7 +857,12 @@ public class HttpRemoteTaskRunner implements
WorkerTaskRunner, TaskLogStreamer
return Optional.absent();
} else {
// Worker is still running this task
- final URL url = TaskRunnerUtils.makeWorkerURL(worker,
"/druid/worker/v1/task/%s/log?offset=%d", taskId, offset);
+ final URL url = TaskRunnerUtils.makeWorkerURL(
+ worker,
+ "/druid/worker/v1/task/%s/log?offset=%s",
+ taskId,
+ Long.toString(offset)
+ );
return Optional.of(
new ByteSource()
{
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]