Skip to content

Commit

Permalink
More error reporting and stats for ingestion tasks (apache#5418)
Browse files Browse the repository at this point in the history
* Add more indexing task status and error reporting

* PR comments, add support in AppenderatorDriverRealtimeIndexTask

* Use TaskReport instead of metrics/context

* Fix tests

* Use TaskReport uploads

* Refactor fire department metrics retrieval

* Refactor input row serde in hadoop task

* Refactor hadoop task loader names

* Truncate error message in TaskStatus, add errorMsg to task report

* PR comments
  • Loading branch information
jon-wei authored and Surekha Saharan committed Apr 6, 2018
1 parent ac401c5 commit 9e786e8
Show file tree
Hide file tree
Showing 87 changed files with 4,085 additions and 615 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ public List<InputRow> parseBatch(Map<String, Object> theMap)
}
}
catch (Exception e) {
throw new ParseException(e, "Unparseable timestamp found!");
throw new ParseException(e, "Unparseable timestamp found! Event: %s", theMap);
}

return ImmutableList.of(new MapBasedInputRow(timestamp.getMillis(), dimensions, theMap));
Expand Down
28 changes: 28 additions & 0 deletions api/src/main/java/io/druid/indexer/IngestionState.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.indexer;

public enum IngestionState
{
NOT_STARTED,
DETERMINE_PARTITIONS,
BUILD_SEGMENTS,
COMPLETED
}
29 changes: 29 additions & 0 deletions api/src/main/java/io/druid/indexer/TaskMetricsGetter.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.indexer;

import java.util.List;
import java.util.Map;

public interface TaskMetricsGetter
{
List<String> getKeys();
Map<String, Number> getTotalMetrics();
}
47 changes: 47 additions & 0 deletions api/src/main/java/io/druid/indexer/TaskMetricsUtils.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.indexer;

import com.google.common.collect.Maps;

import java.util.Map;

public class TaskMetricsUtils
{
public static final String ROWS_PROCESSED = "rowsProcessed";
public static final String ROWS_PROCESSED_WITH_ERRORS = "rowsProcessedWithErrors";
public static final String ROWS_UNPARSEABLE = "rowsUnparseable";
public static final String ROWS_THROWN_AWAY = "rowsThrownAway";

public static Map<String, Object> makeIngestionRowMetrics(
long rowsProcessed,
long rowsProcessedWithErrors,
long rowsUnparseable,
long rowsThrownAway
)
{
Map<String, Object> metricsMap = Maps.newHashMap();
metricsMap.put(ROWS_PROCESSED, rowsProcessed);
metricsMap.put(ROWS_PROCESSED_WITH_ERRORS, rowsProcessedWithErrors);
metricsMap.put(ROWS_UNPARSEABLE, rowsUnparseable);
metricsMap.put(ROWS_THROWN_AWAY, rowsThrownAway);
return metricsMap;
}
}
71 changes: 40 additions & 31 deletions api/src/main/java/io/druid/indexer/TaskStatusPlus.java
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,9 @@ public class TaskStatusPlus
private final TaskLocation location;
private final String dataSource;

@Nullable
private final String errorMsg;

@JsonCreator
public TaskStatusPlus(
@JsonProperty("id") String id,
Expand All @@ -47,7 +50,8 @@ public TaskStatusPlus(
@JsonProperty("statusCode") @Nullable TaskState state,
@JsonProperty("duration") @Nullable Long duration,
@JsonProperty("location") TaskLocation location,
@JsonProperty("dataSource") String dataSource
@JsonProperty("dataSource") String dataSource,
@JsonProperty("errorMsg") String errorMsg
)
{
if (state != null && state.isComplete()) {
Expand All @@ -61,6 +65,7 @@ public TaskStatusPlus(
this.duration = duration;
this.location = Preconditions.checkNotNull(location, "location");
this.dataSource = dataSource;
this.errorMsg = errorMsg;
}

@JsonProperty
Expand Down Expand Up @@ -108,49 +113,53 @@ public TaskLocation getLocation()
return location;
}

@JsonProperty
public String getDataSource()
{
return dataSource;
}

@Nullable
@JsonProperty("errorMsg")
public String getErrorMsg()
{
return errorMsg;
}

@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}

if (o == null || getClass() != o.getClass()) {
return false;
}

final TaskStatusPlus that = (TaskStatusPlus) o;
if (!id.equals(that.id)) {
return false;
}
if (!type.equals(that.type)) {
return false;
}
if (!createdTime.equals(that.createdTime)) {
return false;
}
if (!queueInsertionTime.equals(that.queueInsertionTime)) {
return false;
}
if (!Objects.equals(state, that.state)) {
return false;
}
if (!Objects.equals(duration, that.duration)) {
return false;
}
return location.equals(that.location);
TaskStatusPlus that = (TaskStatusPlus) o;
return Objects.equals(getId(), that.getId()) &&
Objects.equals(getType(), that.getType()) &&
Objects.equals(getCreatedTime(), that.getCreatedTime()) &&
Objects.equals(getQueueInsertionTime(), that.getQueueInsertionTime()) &&
getState() == that.getState() &&
Objects.equals(getDuration(), that.getDuration()) &&
Objects.equals(getLocation(), that.getLocation()) &&
Objects.equals(getDataSource(), that.getDataSource()) &&
Objects.equals(getErrorMsg(), that.getErrorMsg());
}

@Override
public int hashCode()
{
return Objects.hash(id, type, createdTime, queueInsertionTime, state, duration, location);
}

@JsonProperty
public String getDataSource()
{
return dataSource;
return Objects.hash(
getId(),
getType(),
getCreatedTime(),
getQueueInsertionTime(),
getState(),
getDuration(),
getLocation(),
getDataSource(),
getErrorMsg()
);
}

}
92 changes: 92 additions & 0 deletions api/src/main/java/io/druid/utils/CircularBuffer.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.utils;

import com.google.common.base.Preconditions;

/**
* A circular buffer that supports random bidirectional access.
*
* @param <E> Type of object to be stored in the buffer
*/
public class CircularBuffer<E>
{
public E[] getBuffer()
{
return buffer;
}

private final E[] buffer;

private int start = 0;
private int size = 0;

public CircularBuffer(int capacity)
{
Preconditions.checkArgument(capacity > 0, "Capacity must be greater than 0.");
buffer = (E[]) new Object[capacity];
}

public void add(E item)
{
buffer[start++] = item;

if (start >= buffer.length) {
start = 0;
}

if (size < buffer.length) {
size++;
}
}

/**
* Access object at a given index, starting from the latest entry added and moving backwards.
*/
public E getLatest(int index)
{
Preconditions.checkArgument(index >= 0 && index < size, "invalid index");

int bufferIndex = start - index - 1;
if (bufferIndex < 0) {
bufferIndex = buffer.length + bufferIndex;
}
return buffer[bufferIndex];
}

/**
* Access object at a given index, starting from the earliest entry added and moving forward.
*/
public E get(int index)
{
Preconditions.checkArgument(index >= 0 && index < size, "invalid index");

int bufferIndex = (start - size + index) % buffer.length;
if (bufferIndex < 0) {
bufferIndex += buffer.length;
}
return buffer[bufferIndex];
}

public int size()
{
return size;
}
}
3 changes: 2 additions & 1 deletion api/src/test/java/io/druid/indexer/TaskStatusPlusTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,8 @@ public void testSerde() throws IOException
TaskState.RUNNING,
1000L,
TaskLocation.create("testHost", 1010, -1),
"ds_test"
"ds_test",
null
);
final String json = mapper.writeValueAsString(status);
Assert.assertEquals(status, mapper.readValue(json, TaskStatusPlus.class));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -255,7 +255,7 @@ public void ingest(Blackhole blackhole) throws Exception
{
incIndexFilteredAgg = makeIncIndex(filteredMetrics);
for (InputRow row : inputRows) {
int rv = incIndexFilteredAgg.add(row);
int rv = incIndexFilteredAgg.add(row).getRowCount();
blackhole.consume(rv);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,7 @@ public void normalLongs(Blackhole blackhole) throws Exception
{
for (int i = 0; i < maxRows; i++) {
InputRow row = longRows.get(i);
int rv = incIndex.add(row);
int rv = incIndex.add(row).getRowCount();
blackhole.consume(rv);
}
}
Expand All @@ -173,7 +173,7 @@ public void normalFloats(Blackhole blackhole) throws Exception
{
for (int i = 0; i < maxRows; i++) {
InputRow row = floatRows.get(i);
int rv = incFloatIndex.add(row);
int rv = incFloatIndex.add(row).getRowCount();
blackhole.consume(rv);
}
}
Expand All @@ -186,7 +186,7 @@ public void normalStrings(Blackhole blackhole) throws Exception
{
for (int i = 0; i < maxRows; i++) {
InputRow row = stringRows.get(i);
int rv = incStrIndex.add(row);
int rv = incStrIndex.add(row).getRowCount();
blackhole.consume(rv);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ public void addRows(Blackhole blackhole) throws Exception
{
for (int i = 0; i < rowsPerSegment; i++) {
InputRow row = rows.get(i);
int rv = incIndex.add(row);
int rv = incIndex.add(row).getRowCount();
blackhole.consume(rv);
}
}
Expand Down
Loading

0 comments on commit 9e786e8

Please sign in to comment.