workUnit.removeProp(ConfigurationKeys.WORK_UNIT_LOW_WATER_MARK_KEY); workUnit.removeProp(ConfigurationKeys.WORK_UNIT_HIGH_WATER_MARK_KEY); workUnit.setWatermarkInterval(interval); workUnit.setProp(KafkaUtils.getPartitionPropName(KafkaSource.PREVIOUS_START_FETCH_EPOCH_TIME, index), wu.getProp(KafkaSource.PREVIOUS_START_FETCH_EPOCH_TIME)); workUnit.setProp(KafkaUtils.getPartitionPropName(KafkaSource.PREVIOUS_STOP_FETCH_EPOCH_TIME, index), wu.getProp(KafkaSource.PREVIOUS_STOP_FETCH_EPOCH_TIME)); workUnit.setProp(KafkaUtils.getPartitionPropName(KafkaSource.PREVIOUS_LOW_WATERMARK, index), wu.getProp(KafkaSource.PREVIOUS_LOW_WATERMARK)); workUnit.setProp(KafkaUtils.getPartitionPropName(KafkaSource.PREVIOUS_HIGH_WATERMARK, index), wu.getProp(KafkaSource.PREVIOUS_HIGH_WATERMARK)); workUnit.setProp(KafkaUtils.getPartitionPropName(KafkaSource.PREVIOUS_OFFSET_FETCH_EPOCH_TIME, index), wu.getProp(KafkaSource.PREVIOUS_OFFSET_FETCH_EPOCH_TIME)); workUnit.setProp(KafkaUtils.getPartitionPropName(KafkaSource.OFFSET_FETCH_EPOCH_TIME, index), wu.getProp(KafkaSource.OFFSET_FETCH_EPOCH_TIME)); workUnit.setProp(KafkaUtils.getPartitionPropName(KafkaSource.PREVIOUS_LATEST_OFFSET, index), wu.getProp(KafkaSource.PREVIOUS_LATEST_OFFSET)); index++; workUnit.removeProp(KafkaSource.PREVIOUS_START_FETCH_EPOCH_TIME); workUnit.removeProp(KafkaSource.PREVIOUS_STOP_FETCH_EPOCH_TIME); workUnit.removeProp(KafkaSource.PREVIOUS_LOW_WATERMARK); workUnit.removeProp(KafkaSource.PREVIOUS_HIGH_WATERMARK); workUnit.removeProp(KafkaSource.PREVIOUS_OFFSET_FETCH_EPOCH_TIME); workUnit.removeProp(KafkaSource.OFFSET_FETCH_EPOCH_TIME); workUnit.removeProp(KafkaSource.PREVIOUS_LATEST_OFFSET); workUnit.removeProp(KafkaSource.PARTITION_ID);
public static Partition deserialize(WorkUnit workUnit) { long lowWatermark = ConfigurationKeys.DEFAULT_WATERMARK_VALUE; long highWatermark = ConfigurationKeys.DEFAULT_WATERMARK_VALUE; if (workUnit.getProp(ConfigurationKeys.WATERMARK_INTERVAL_VALUE_KEY) != null) { lowWatermark = workUnit.getLowWatermark(LongWatermark.class).getValue(); highWatermark = workUnit.getExpectedHighWatermark(LongWatermark.class).getValue(); } return new Partition(lowWatermark, highWatermark, workUnit.getPropAsBoolean(Partition.IS_LAST_PARTIITON), workUnit.getPropAsBoolean(Partition.HAS_USER_SPECIFIED_HIGH_WATERMARK)); } }
@Override public void readFields(DataInput in) throws IOException { int numWorkUnits = in.readInt(); for (int i = 0; i < numWorkUnits; i++) { WorkUnit workUnit = WorkUnit.createEmpty(); workUnit.readFields(in); this.workUnits.add(workUnit); } super.readFields(in); }
protected static double getWorkUnitEstSize(WorkUnit workUnit) { Preconditions.checkArgument(workUnit.contains(ESTIMATED_WORKUNIT_SIZE)); return workUnit.getPropAsDouble(ESTIMATED_WORKUNIT_SIZE); }
@Override public List<WorkUnit> getWorkunits(SourceState state) { List<WorkUnit> workUnits = Lists.newArrayList(); if (!state.contains(ConfigurationKeys.SOURCE_FILEBASED_FILES_TO_PULL)) { return workUnits; } // Create a single snapshot-type extract for all files Extract extract = new Extract(Extract.TableType.SNAPSHOT_ONLY, state.getProp(ConfigurationKeys.EXTRACT_NAMESPACE_NAME_KEY, "ExampleNamespace"), "ExampleTable"); String filesToPull = state.getProp(ConfigurationKeys.SOURCE_FILEBASED_FILES_TO_PULL); for (String file : Splitter.on(',').omitEmptyStrings().split(filesToPull)) { // Create one work unit for each file to pull WorkUnit workUnit = WorkUnit.create(extract); workUnit.setProp(SOURCE_FILE_KEY, file); workUnits.add(workUnit); } return workUnits; }
LongWatermark expectedWatermark = new LongWatermark(watermark.getValue() + numRecordsPerExtract); WatermarkInterval watermarkInterval = new WatermarkInterval(watermark, expectedWatermark); workUnit = WorkUnit.create(newExtract(tableType, namespace, table), watermarkInterval); log.debug("Will be setting watermark interval to " + watermarkInterval.toJson()); workUnit.setProp(WORK_UNIT_INDEX, workUnitState.getWorkunit().getProp(WORK_UNIT_INDEX)); LongWatermark watermark = workUnitState.getWorkunit().getLowWatermark(LongWatermark.class); LongWatermark expectedWatermark = new LongWatermark(watermark.getValue() + numRecordsPerExtract); WatermarkInterval watermarkInterval = new WatermarkInterval(watermark, expectedWatermark); workUnit = WorkUnit.create(newExtract(tableType, namespace, table), watermarkInterval); log.debug("Will be setting watermark interval to " + watermarkInterval.toJson()); workUnit.setProp(WORK_UNIT_INDEX, workUnitState.getWorkunit().getProp(WORK_UNIT_INDEX));
if (previousWorkunits.get(0).getWorkunit().contains(ConfigurationKeys.SOURCE_FILEBASED_FS_SNAPSHOT)) { prevFsSnapshot.addAll(previousWorkunits.get(0).getWorkunit().getPropAsSet(ConfigurationKeys.SOURCE_FILEBASED_FS_SNAPSHOT)); } else if (state.getPropAsBoolean(ConfigurationKeys.SOURCE_FILEBASED_FS_PRIOR_SNAPSHOT_REQUIRED, ConfigurationKeys.DEFAULT_SOURCE_FILEBASED_FS_PRIOR_SNAPSHOT_REQUIRED)) { log.info("Total number of work units from the previous failed runs: " + previousWorkUnitsForRetry.size()); for (WorkUnit previousWorkUnitForRetry : previousWorkUnitsForRetry) { prevFsSnapshot.addAll(previousWorkUnitForRetry.getPropAsSet(ConfigurationKeys.SOURCE_FILEBASED_FILES_TO_PULL)); workUnits.add(previousWorkUnitForRetry); workUnit.setProp(ConfigurationKeys.SOURCE_FILEBASED_FS_SNAPSHOT, StringUtils.join(effectiveSnapshot, ",")); WorkUnit workUnit = WorkUnit.create(extract); workUnit.setProp(ConfigurationKeys.SOURCE_FILEBASED_FS_SNAPSHOT, StringUtils.join(effectiveSnapshot, ",")); workUnit.setProp(ConfigurationKeys.SOURCE_FILEBASED_FILES_TO_PULL, StringUtils.join(partitionFilesToPull, ",")); if (state.getPropAsBoolean(ConfigurationKeys.SOURCE_FILEBASED_PRESERVE_FILE_NAME, false)) { throw new RuntimeException("Cannot preserve the file name if a workunit is given multiple files"); workUnit.setProp(ConfigurationKeys.DATA_PUBLISHER_FINAL_DIR, workUnit.getProp(ConfigurationKeys.SOURCE_FILEBASED_FILES_TO_PULL));
@BeforeClass public void setUp() throws Exception { WorkUnit workUnit = WorkUnit.createEmpty(); Properties properties = new Properties(); properties.load(new StringReader(TEST_JOB_CONFIG)); workUnit.addAll(properties); workUnit.setProp(ConfigurationKeys.JOB_ID_KEY, JobLauncherUtils.newJobId("GobblinTest1")); workUnit.setProp(ConfigurationKeys.TASK_ID_KEY, JobLauncherUtils.newTaskId(workUnit.getProp(ConfigurationKeys.JOB_ID_KEY), 0)); this.taskContext = new TaskContext(new WorkUnitState(workUnit)); }
/** * Get the low {@link Watermark} as a {@link JsonElement}. * * @return a {@link JsonElement} representing the low {@link Watermark} or * {@code null} if the low {@link Watermark} is not set. */ public JsonElement getLowWatermark() { if (!contains(ConfigurationKeys.WATERMARK_INTERVAL_VALUE_KEY)) { return null; } return JSON_PARSER.parse(getProp(ConfigurationKeys.WATERMARK_INTERVAL_VALUE_KEY)).getAsJsonObject() .get(WatermarkInterval.LOW_WATERMARK_TO_JSON_KEY); }
@Override public String getConnectionUrl() { String urlPrefix = "jdbc:teradata://"; String host = this.workUnit.getProp(ConfigurationKeys.SOURCE_CONN_HOST_NAME); checkArgument(!isNullOrEmpty(host), "Connectionn host cannot be null or empty at %s", ConfigurationKeys.SOURCE_CONN_HOST_NAME); String port = this.workUnit.getProp(ConfigurationKeys.SOURCE_CONN_PORT,"1025"); String database = this.workUnit.getProp(ConfigurationKeys.SOURCE_QUERYBASED_SCHEMA); String defaultUrl = urlPrefix + host.trim() + "/TYPE=FASTEXPORT,DATABASE=" + database.trim() + ",DBS_PORT=" + port.trim() ; // use custom url from source.conn.host if Teradata jdbc url available return host.contains(urlPrefix) ? host.trim() : defaultUrl; }
@Test public void testSerializeState() throws IOException { WorkUnit workUnit1 = WorkUnit.createEmpty(); workUnit1.setProp("foo", "bar"); workUnit1.setProp("a", 10); SerializationUtils.serializeState(this.fs, new Path(this.outputPath, "wu1"), workUnit1); WorkUnit workUnit2 = WorkUnit.createEmpty(); workUnit2.setProp("foo", "baz"); workUnit2.setProp("b", 20); SerializationUtils.serializeState(this.fs, new Path(this.outputPath, "wu2"), workUnit2); }
private List<WorkUnit> initialWorkUnits() { List<WorkUnit> workUnits = Lists.newArrayList(); for (int i=0; i < num_parallelism; i++) { WorkUnit workUnit = WorkUnit.create(newExtract(Extract.TableType.APPEND_ONLY, namespace, table)); LongWatermark lowWatermark = new LongWatermark(i * numRecordsPerExtract + 1); LongWatermark expectedHighWatermark = new LongWatermark((i + 1) * numRecordsPerExtract); workUnit.setWatermarkInterval(new WatermarkInterval(lowWatermark, expectedHighWatermark)); workUnit.setProp(WORK_UNIT_INDEX, i); workUnits.add(workUnit); } return workUnits; }
@Test(dependsOnMethods = "testSerializeState") public void testDeserializeState() throws IOException { WorkUnit workUnit1 = WorkUnit.createEmpty(); WorkUnit workUnit2 = WorkUnit.createEmpty(); SerializationUtils.deserializeState(this.fs, new Path(this.outputPath, "wu1"), workUnit1); SerializationUtils.deserializeState(this.fs, new Path(this.outputPath, "wu2"), workUnit2); Assert.assertEquals(workUnit1.getPropertyNames().size(), 2); Assert.assertEquals(workUnit1.getProp("foo"), "bar"); Assert.assertEquals(workUnit1.getPropAsInt("a"), 10); Assert.assertEquals(workUnit2.getPropertyNames().size(), 2); Assert.assertEquals(workUnit2.getProp("foo"), "baz"); Assert.assertEquals(workUnit2.getPropAsInt("b"), 20); }
/** * Set the specified key, value pair in this {@link MultiWorkUnit} only, but do not propagate it to all the inner * {@link WorkUnit}s. * * @param key property key * @param value property value */ public void setPropExcludeInnerWorkUnits(String key, Object value) { super.setProp(key, value); }
private List<WorkUnit> initializeWorkUnits() { List<WorkUnit> workUnits = Lists.newArrayList(); for (int i = 0; i < NUM_WORK_UNITS; i++) { WorkUnit workUnit = WorkUnit.create(createExtract(Extract.TableType.SNAPSHOT_ONLY, NAMESPACE, TABLE)); workUnit.setLowWaterMark(i * NUM_RECORDS_TO_EXTRACT_PER_EXTRACTOR + 1); workUnit.setHighWaterMark((i + 1) * NUM_RECORDS_TO_EXTRACT_PER_EXTRACTOR); workUnit.setProp(WORK_UNIT_INDEX_KEY, i); workUnits.add(workUnit); } return workUnits; } }
WorkUnit watermarkWorkunit = WorkUnit.createEmpty(); watermarkWorkunit.setProp(IS_WATERMARK_WORKUNIT_KEY, true); watermarkWorkunit.setProp(ConfigurationKeys.DATASET_URN_KEY, tableKey); watermarkWorkunit.setWatermarkInterval(new WatermarkInterval(new MultiKeyValueLongWatermark( this.previousWatermarks.get(tableKey)), new MultiKeyValueLongWatermark(expectedPartitionWatermarks)));
this.topicName = KafkaUtils.getTopicName(state); this.partitions = KafkaUtils.getPartitions(state); this.lowWatermark = state.getWorkunit().getLowWatermark(MultiLongWatermark.class); this.highWatermark = state.getWorkunit().getExpectedHighWatermark(MultiLongWatermark.class); this.nextWatermark = new MultiLongWatermark(this.lowWatermark); this.kafkaConsumerClientResolver = new ClassAliasResolver<>(GobblinKafkaConsumerClientFactory.class);
private void addTopicSpecificPropsToWorkUnit(WorkUnit workUnit, Map<String, State> topicSpecificStateMap) { if (workUnit instanceof MultiWorkUnit) { for (WorkUnit wu : ((MultiWorkUnit) workUnit).getWorkUnits()) { addTopicSpecificPropsToWorkUnit(wu, topicSpecificStateMap); } } else if (!workUnit.contains(TOPIC_NAME)) { return; } else { addDatasetUrnOptionally(workUnit); if (topicSpecificStateMap == null) { return; } else if (!topicSpecificStateMap.containsKey(workUnit.getProp(TOPIC_NAME))) { return; } else { workUnit.addAll(topicSpecificStateMap.get(workUnit.getProp(TOPIC_NAME))); } } }