public LineageEventBuilder(String name) { super(name, LIENAGE_EVENT_NAMESPACE); addMetadata(EVENT_TYPE, LINEAGE_EVENT_TYPE); }
/** * Prefix all keys with {@link LineageEventBuilder#LIENAGE_EVENT_NAMESPACE} */ private static String getKey(Object... objects) { Object[] args = new Object[objects.length + 1]; args[0] = LineageEventBuilder.LIENAGE_EVENT_NAMESPACE; System.arraycopy(objects, 0, args, 1, objects.length); return LineageEventBuilder.getKey(args); } }
public LineageInfo(Config config) { resolver = getResolver(config.withFallback(FALLBACK)); }
@Test public void testEventForPartitionedDataset() { final String topic = "testTopic"; final String kafka = "kafka"; final String hdfs = "hdfs"; final String path = "/data/tracking/PageViewEvent"; final String partitionName = "hourly/2018/08/15/15"; State state = new State(); LineageInfo lineageInfo = getLineageInfo(); DatasetDescriptor source = new DatasetDescriptor(kafka, topic); lineageInfo.setSource(source, state); DatasetDescriptor destinationDataset = new DatasetDescriptor(hdfs, path); PartitionDescriptor destination = new PartitionDescriptor(partitionName, destinationDataset); lineageInfo.putDestination(destination, 0, state); Map<String, Set<LineageEventBuilder>> events = LineageInfo.load(state); LineageEventBuilder event = first(events.get("0")); verify(event, topic, source, destination); // Verify gobblin tracking event GobblinTrackingEvent trackingEvent = event.build(); Assert.assertEquals(LineageEventBuilder.isLineageEvent(trackingEvent), true); Assert.assertEquals(LineageEventBuilder.fromEvent(trackingEvent), event); }
@Override protected void addLineageSourceInfo(SourceState sourceState, SourceEntity entity, WorkUnit workUnit) { DatasetDescriptor source = new DatasetDescriptor(DatasetConstants.PLATFORM_SALESFORCE, entity.getSourceEntityName()); if (lineageInfo.isPresent()) { lineageInfo.get().setSource(source, workUnit); } }
private void submitLineageEvent(String dataset, Collection<TaskState> states) { Collection<LineageEventBuilder> events = LineageInfo.load(states); // Send events events.forEach(event -> event.submit(metricContext)); log.info(String.format("Submitted %d lineage events for dataset %s", events.size(), dataset)); }
/** * Put a {@link DatasetDescriptor} of a destination dataset to a state * * <p> * Only the {@link org.apache.gobblin.writer.DataWriter} or {@link org.apache.gobblin.publisher.DataPublisher} * is supposed to put the destination dataset information. Since different branches may concurrently put, * the method is implemented to be threadsafe * </p> * * @deprecated Use {@link #putDestination(List, int, State)} */ @Deprecated public void putDestination(Descriptor destination, int branchId, State state) { putDestination(Lists.newArrayList(destination), branchId, state); }
/** * Load all lineage information from {@link State}s of a dataset * * @param states All states which belong to the same dataset * @return A collection of {@link LineageEventBuilder}s put in the state */ public static Collection<LineageEventBuilder> load(Collection<? extends State> states) { Preconditions.checkArgument(states != null && !states.isEmpty()); Set<LineageEventBuilder> allEvents = Sets.newHashSet(); for (State state : states) { Map<String, Set<LineageEventBuilder>> branchedEvents = load(state); branchedEvents.values().forEach(allEvents::addAll); } return allEvents; }
private void verify(LineageEventBuilder event, String name, Descriptor source, Descriptor destination) { Assert.assertEquals(event.getName(), name); Assert.assertEquals(event.getNamespace(), LineageEventBuilder.LIENAGE_EVENT_NAMESPACE); Assert.assertEquals(event.getMetadata().get(GobblinEventBuilder.EVENT_TYPE), LineageEventBuilder.LINEAGE_EVENT_TYPE); Assert.assertTrue(event.getSource().equals(source)); Assert.assertTrue(event.getDestination().equals(destination)); }
@Override public SharedResourceFactoryResponse<LineageInfo> createResource(SharedResourcesBroker<GobblinScopeTypes> broker, ScopedConfigView<GobblinScopeTypes, EmptyKey> config) throws NotConfiguredException { return new ResourceInstance<>(new LineageInfo(config.getConfig())); }
/** * Check if the given state has lineage info */ public static boolean hasLineageInfo(State state) { return state.contains(getKey(NAME_KEY)); }
private static LineageEventBuilder find(Collection<LineageEventBuilder> events, String partitionName) { for (LineageEventBuilder event : events) { if (event.getDestination().getName().equals(partitionName)) { return event; } } return null; }
private static Map<String, Collection<TaskState>> aggregateByLineageEvent(Collection<TaskState> states) { Map<String, Collection<TaskState>> statesByEvents = Maps.newHashMap(); for (TaskState state : states) { String eventName = LineageInfo.getFullEventName(state); Collection<TaskState> statesForEvent = statesByEvents.computeIfAbsent(eventName, k -> Lists.newArrayList()); statesForEvent.add(state); } return statesByEvents; } }
private void addLineageInfo(CopyEntity copyEntity, WorkUnit workUnit) { if (copyEntity instanceof CopyableFile) { CopyableFile copyableFile = (CopyableFile) copyEntity; /* * In Gobblin Distcp, the source and target path info of a CopyableFile are determined by its dataset found by * a DatasetFinder. Consequently, the source and destination dataset for the CopyableFile lineage are expected * to be set by the same logic */ if (lineageInfo.isPresent() && copyableFile.getSourceData() != null && copyableFile.getDestinationData() != null) { lineageInfo.get().setSource(copyableFile.getSourceData(), workUnit); } } } }
@VisibleForTesting public static void setDestLineageInfo(WorkUnitState wus, Optional<LineageInfo> lineageInfo) { HiveWorkUnit hiveWorkUnit = new HiveWorkUnit(wus.getWorkunit()); ConvertibleHiveDataset convertibleHiveDataset = (ConvertibleHiveDataset) hiveWorkUnit.getHiveDataset(); List<DatasetDescriptor> destDatasets = convertibleHiveDataset.getDestDatasets(); for (int i = 0; i < destDatasets.size(); i++) { if (lineageInfo.isPresent()) { lineageInfo.get().putDestination(destDatasets.get(i), i + 1, wus); } } }
/** * Get the full lineage event name from a state */ public static String getFullEventName(State state) { return Joiner.on('.').join(LineageEventBuilder.LIENAGE_EVENT_NAMESPACE, state.getProp(getKey(NAME_KEY))); }
private LineageEventBuilder getLineageEvent(Collection<LineageEventBuilder> events, int branchId, String destinationPlatform) { for (LineageEventBuilder event : events) { DatasetDescriptor descriptor = (DatasetDescriptor) event.getDestination(); if (descriptor.getPlatform().equals(destinationPlatform) && descriptor.getMetadata().get(DatasetConstants.BRANCH).equals(String.valueOf(branchId))) { return event; } } return null; }
@VisibleForTesting public void setSourceLineageInfo(WorkUnit workUnit, Optional<LineageInfo> lineageInfo) { HiveWorkUnit hiveWorkUnit = new HiveWorkUnit(workUnit); ConvertibleHiveDataset convertibleHiveDataset = (ConvertibleHiveDataset) hiveWorkUnit.getHiveDataset(); DatasetDescriptor sourceDataset = convertibleHiveDataset.getSourceDataset(); if (lineageInfo.isPresent()) { lineageInfo.get().setSource(sourceDataset, workUnit); } } }
/** * Set source {@link DatasetDescriptor} of a lineage event * * <p> * Only the {@link org.apache.gobblin.source.Source} or its {@link org.apache.gobblin.source.extractor.Extractor} * is supposed to set the source for a work unit of a dataset * </p> * * @param state state about a {@link org.apache.gobblin.source.workunit.WorkUnit} * */ public void setSource(Descriptor source, State state) { Descriptor descriptor = resolver.resolve(source, state); if (descriptor == null) { return; } state.setProp(getKey(NAME_KEY), descriptor.getName()); state.setProp(getKey(LineageEventBuilder.SOURCE), Descriptor.toJson(descriptor)); }
protected void addLineageSourceInfo(SourceState sourceState, SourceEntity entity, WorkUnit workUnit) { String host = sourceState.getProp(ConfigurationKeys.SOURCE_CONN_HOST_NAME); String port = sourceState.getProp(ConfigurationKeys.SOURCE_CONN_PORT); String database = sourceState.getProp(ConfigurationKeys.SOURCE_QUERYBASED_SCHEMA); String connectionUrl = "jdbc:mysql://" + host.trim() + ":" + port + "/" + database.trim(); DatasetDescriptor source = new DatasetDescriptor(DatasetConstants.PLATFORM_MYSQL, database + "." + entity.getSourceEntityName()); source.addMetadata(DatasetConstants.CONNECTION_URL, connectionUrl); if (lineageInfo.isPresent()) { lineageInfo.get().setSource(source, workUnit); } } }