/** * Get the {@link ConfigurationKeys#TASK_FAILURE_EXCEPTION_KEY} if it exists, else return {@link Optional#absent()}. */ public Optional<String> getTaskFailureException() { return Optional.fromNullable(this.getProp(ConfigurationKeys.TASK_FAILURE_EXCEPTION_KEY)); }
/** * Convert this {@link TaskState} to a json document. * * @param jsonWriter a {@link com.google.gson.stream.JsonWriter} used to write the json document * @throws IOException */ public void toJson(JsonWriter jsonWriter, boolean keepConfig) throws IOException { jsonWriter.beginObject(); jsonWriter.name("task id").value(this.getTaskId()).name("task state").value(this.getWorkingState().name()) .name("start time").value(this.getStartTime()).name("end time").value(this.getEndTime()).name("duration") .value(this.getTaskDuration()).name("retry count") .value(this.getPropAsInt(ConfigurationKeys.TASK_RETRIES_KEY, 0)); // Also add failure exception information if it exists. This information is useful even in the // case that the task finally succeeds so we know what happened in the course of task execution. if (getTaskFailureException().isPresent()) { jsonWriter.name("exception").value(getTaskFailureException().get()); } if (keepConfig) { jsonWriter.name("properties"); jsonWriter.beginObject(); for (String key : this.getPropertyNames()) { jsonWriter.name(key).value(this.getProp(key)); } jsonWriter.endObject(); } jsonWriter.endObject(); }
public AsynchronousFork(TaskContext taskContext, Object schema, int branches, int index, ExecutionModel executionModel) throws Exception { super(taskContext, schema, branches, index, executionModel); TaskState taskState = taskContext.getTaskState(); this.recordQueue = BoundedBlockingRecordQueue.newBuilder() .hasCapacity(taskState.getPropAsInt( ConfigurationKeys.FORK_RECORD_QUEUE_CAPACITY_KEY, ConfigurationKeys.DEFAULT_FORK_RECORD_QUEUE_CAPACITY)) .useTimeout(taskState.getPropAsLong( ConfigurationKeys.FORK_RECORD_QUEUE_TIMEOUT_KEY, ConfigurationKeys.DEFAULT_FORK_RECORD_QUEUE_TIMEOUT)) .useTimeoutTimeUnit(TimeUnit.valueOf(taskState.getProp( ConfigurationKeys.FORK_RECORD_QUEUE_TIMEOUT_UNIT_KEY, ConfigurationKeys.DEFAULT_FORK_RECORD_QUEUE_TIMEOUT_UNIT))) .collectStats() .build(); }
private void verifyJobState(int run) throws IOException { List<JobState.DatasetState> datasetStateList = this.datasetStateStore.getAll(JOB_NAME, "current.jst"); Assert.assertEquals(datasetStateList.size(), 1); JobState jobState = datasetStateList.get(0); Assert.assertEquals(jobState.getState(), JobState.RunningState.COMMITTED); Assert.assertEquals(jobState.getTaskStates().size(), DummySource.NUM_WORK_UNITS); for (TaskState taskState : jobState.getTaskStates()) { Assert.assertEquals(taskState.getWorkingState(), WorkUnitState.WorkingState.COMMITTED); Assert.assertEquals(taskState.getProp(FOO), BAR); // Check if the low watermark is properly kept track of int expectedLowWatermark = (run - 1) * DummySource.NUM_WORK_UNITS * DummySource.NUM_RECORDS_TO_EXTRACT_PER_EXTRACTOR + taskState.getPropAsInt(WORK_UNIT_INDEX_KEY) * DummySource.NUM_RECORDS_TO_EXTRACT_PER_EXTRACTOR + 1; Assert.assertEquals(taskState.getPropAsInt(ConfigurationKeys.WORK_UNIT_LOW_WATER_MARK_KEY), expectedLowWatermark); // Check if the high watermark is properly kept track of int expectedHighWatermark = expectedLowWatermark + DummySource.NUM_RECORDS_TO_EXTRACT_PER_EXTRACTOR - 1; Assert.assertEquals(taskState.getPropAsInt(ConfigurationKeys.WORK_UNIT_HIGH_WATER_MARK_KEY), expectedHighWatermark); Assert.assertEquals(taskState.getPropAsInt(LAST_READ_RECORD_KEY), expectedHighWatermark); } }
@SuppressWarnings("unchecked") private Class<? extends DataPublisher> getTaskPublisherClass() throws ReflectiveOperationException { if (this.taskState.contains(ConfigurationKeys.TASK_DATA_PUBLISHER_TYPE)) { return (Class<? extends DataPublisher>) Class .forName(this.taskState.getProp(ConfigurationKeys.TASK_DATA_PUBLISHER_TYPE)); } return (Class<? extends DataPublisher>) Class.forName( this.taskState.getProp(ConfigurationKeys.DATA_PUBLISHER_TYPE, ConfigurationKeys.DEFAULT_DATA_PUBLISHER_TYPE)); }
/** * Get a {@link Source} instance used to get a list of {@link WorkUnit}s. * * @return the {@link Source} used to get the {@link WorkUnit}, <em>null</em> * if it fails to instantiate a {@link Source} object of the given class. */ public Source getSource() { try { return Source.class.cast(Class.forName(this.taskState.getProp(ConfigurationKeys.SOURCE_CLASS_KEY)).newInstance()); } catch (ClassNotFoundException cnfe) { throw new RuntimeException(cnfe); } catch (InstantiationException ie) { throw new RuntimeException(ie); } catch (IllegalAccessException iae) { throw new RuntimeException(iae); } }
/** * Append values for the given key from all {@link TaskState}s * @param sb a {@link StringBuffer} to hold the output * @param key the key of the values to retrieve */ private static void appendTaskStateValues(List<TaskState> taskStates, StringBuffer sb, String key) { // Add task failure messages in a group followed by task failure exceptions for (TaskState taskState : taskStates) { if (taskState.contains(key)) { if (sb.length() != 0) { sb.append(","); } sb.append(taskState.getProp(key)); } } } }
private String createDatasetUrn(Map<String, DatasetState> datasetStatesByUrns, TaskState taskState) { String datasetUrn = taskState.getProp(ConfigurationKeys.DATASET_URN_KEY, ConfigurationKeys.DEFAULT_DATASET_URN); if (!datasetStatesByUrns.containsKey(datasetUrn)) { DatasetState datasetState = newDatasetState(false); datasetState.setDatasetUrn(datasetUrn); datasetStatesByUrns.put(datasetUrn, datasetState); } return datasetUrn; }
/** * Get the {@link ForkOperator} to be applied to converted input schema and data record. * * @return {@link ForkOperator} to be used or <code>null</code> if none is specified */ @SuppressWarnings("unchecked") public ForkOperator getForkOperator() { try { ForkOperator fork = ForkOperator.class.cast(Class.forName(this.taskState.getProp(ConfigurationKeys.FORK_OPERATOR_CLASS_KEY, ConfigurationKeys.DEFAULT_FORK_OPERATOR_CLASS)).newInstance()); return new InstrumentedForkOperatorDecorator<>(fork); } catch (ClassNotFoundException cnfe) { throw new RuntimeException(cnfe); } catch (InstantiationException ie) { throw new RuntimeException(ie); } catch (IllegalAccessException iae) { throw new RuntimeException(iae); } }
protected boolean areSingleBranchTasksSynchronous(TaskContext taskContext) { return BooleanUtils.toBoolean(taskContext.getTaskState() .getProp(TaskConfigurationKeys.TASK_IS_SINGLE_BRANCH_SYNCHRONOUS, TaskConfigurationKeys.DEFAULT_TASK_IS_SINGLE_BRANCH_SYNCHRONOUS)); }
/** * Get the output format of the writer of type {@link WriterOutputFormat}. * * @param branches number of forked branches * @param index branch index * @return output format of the writer */ public WriterOutputFormat getWriterOutputFormat(int branches, int index) { String writerOutputFormatValue = this.taskState.getProp( ForkOperatorUtils.getPropertyNameForBranch(ConfigurationKeys.WRITER_OUTPUT_FORMAT_KEY, branches, index), WriterOutputFormat.OTHER.name()); log.debug("Found writer output format value = {}", writerOutputFormatValue); WriterOutputFormat wof = Enums.getIfPresent(WriterOutputFormat.class, writerOutputFormatValue.toUpperCase()) .or(WriterOutputFormat.OTHER); log.debug("Returning writer output format = {}", wof); return wof; }
protected static List<Tag<?>> tagsForTask(TaskState taskState) { List<Tag<?>> tags = Lists.newArrayList(); tags.add(new Tag<>(TaskEvent.METADATA_TASK_ID, taskState.getTaskId())); tags.add(new Tag<>(TaskEvent.METADATA_TASK_ATTEMPT_ID, taskState.getTaskAttemptId().or(""))); tags.add(new Tag<>(ConfigurationKeys.DATASET_URN_KEY, taskState.getProp(ConfigurationKeys.DATASET_URN_KEY, ConfigurationKeys.DEFAULT_DATASET_URN))); tags.addAll(getCustomTagsFromState(taskState)); return tags; }
/** * Get the writer {@link Destination.DestinationType}. * * @param branches number of forked branches * @param index branch index * @return writer {@link Destination.DestinationType} */ public Destination.DestinationType getDestinationType(int branches, int index) { return Destination.DestinationType.valueOf(this.taskState.getProp( ForkOperatorUtils.getPropertyNameForBranch(ConfigurationKeys.WRITER_DESTINATION_TYPE_KEY, branches, index), Destination.DestinationType.HDFS.name())); }
public TaskState(TaskState taskState) { super(taskState.getWorkunit(), taskState.getJobState(), taskState.getTaskBrokerNullable()); addAll(taskState); this.jobId = taskState.getProp(ConfigurationKeys.JOB_ID_KEY); this.taskId = taskState.getProp(ConfigurationKeys.TASK_ID_KEY); this.taskAttemptId = taskState.getTaskAttemptId(); this.setId(this.taskId); }
private static MetricContext parentContextForTask(TaskState taskState) { return JobMetrics.get(taskState.getProp(ConfigurationKeys.JOB_NAME_KEY), taskState.getJobId()).getMetricContext(); }
public CompactionFailedTask (TaskContext taskContext) { super(taskContext); this.suite = CompactionSuiteUtils.getCompactionSuiteFactory (taskContext.getTaskState()). createSuite(taskContext.getTaskState()); this.dataset = this.suite.load(taskContext.getTaskState()); this.failedReason = taskContext.getTaskState().getProp(CompactionVerifier.COMPACTION_VERIFICATION_FAIL_REASON); }
@Test(dependsOnMethods = {"testSetAndGet"}) public void testSerDe() throws IOException { Closer closer = Closer.create(); try { ByteArrayOutputStream baos = closer.register(new ByteArrayOutputStream()); DataOutputStream dos = closer.register(new DataOutputStream(baos)); this.taskState.write(dos); ByteArrayInputStream bais = closer.register((new ByteArrayInputStream(baos.toByteArray()))); DataInputStream dis = closer.register((new DataInputStream(bais))); TaskState newTaskState = new TaskState(); newTaskState.readFields(dis); Assert.assertEquals(newTaskState.getJobId(), "Job-1"); Assert.assertEquals(newTaskState.getTaskId(), "Task-1"); Assert.assertEquals(this.taskState.getHighWaterMark(), 2000); Assert.assertEquals(newTaskState.getStartTime(), this.startTime); Assert.assertEquals(newTaskState.getEndTime(), this.startTime + 1000); Assert.assertEquals(newTaskState.getTaskDuration(), 1000); Assert.assertEquals(newTaskState.getWorkingState(), WorkUnitState.WorkingState.COMMITTED); Assert.assertEquals(newTaskState.getProp("foo"), "bar"); } catch (Throwable t) { throw closer.rethrow(t); } finally { closer.close(); } }
@Override public DataPublisher createDataPublisher(JobState.DatasetState datasetState) { EventBus eventBus = null; if (datasetState.getTaskStates().get(0).contains(EVENTBUS_ID_KEY)) { eventBus = TestingEventBuses.getEventBus(datasetState.getTaskStates().get(0).getProp(EVENTBUS_ID_KEY)); } return new Publisher(datasetState, eventBus); }
@Override public TaskIFace createTask(TaskContext taskContext) { String taskId = taskContext.getTaskState().getProp(TASK_ID_KEY); EventBus eventBus = null; if (taskContext.getTaskState().contains(EVENTBUS_ID_KEY)) { String eventbusId = taskContext.getTaskState().getProp(EVENTBUS_ID_KEY); eventBus = TestingEventBuses.getEventBus(eventbusId); } return new Task(taskContext, taskId, eventBus); }
@Test public void testSetAndGet() { this.taskState.setId("Task-1"); this.taskState.setHighWaterMark(2000); this.startTime = System.currentTimeMillis(); this.taskState.setStartTime(this.startTime); this.taskState.setEndTime(this.startTime + 1000); this.taskState.setTaskDuration(1000); this.taskState.setWorkingState(WorkUnitState.WorkingState.COMMITTED); this.taskState.setProp("foo", "bar"); Assert.assertEquals(this.taskState.getJobId(), "Job-1"); Assert.assertEquals(this.taskState.getTaskId(), "Task-1"); Assert.assertEquals(this.taskState.getId(), "Task-1"); Assert.assertEquals(this.taskState.getHighWaterMark(), 2000); Assert.assertEquals(this.taskState.getStartTime(), this.startTime); Assert.assertEquals(this.taskState.getEndTime(), this.startTime + 1000); Assert.assertEquals(this.taskState.getTaskDuration(), 1000); Assert.assertEquals(this.taskState.getWorkingState(), WorkUnitState.WorkingState.COMMITTED); Assert.assertEquals(this.taskState.getProp("foo"), "bar"); }