|
27 | 27 |
|
28 | 28 | public class ImportGroupPipeline { |
29 | 29 | private static final Logger LOGGER = LoggerFactory.getLogger(ImportGroupPipeline.class); |
| 30 | + private static final String IMPORT_METADATA_FILE = "import_metadata_mcf.mcf"; |
30 | 31 | private static final Counter nodeInvalidTypeCounter = |
31 | 32 | Metrics.counter(ImportGroupPipeline.class, "mcf_nodes_without_type"); |
32 | 33 | private static final Counter nodeCounter = |
@@ -72,45 +73,67 @@ public static void main(String[] args) { |
72 | 73 |
|
73 | 74 | for (JsonElement element : jsonArray) { |
74 | 75 | JsonElement importElement = element.getAsJsonObject().get("importName"); |
75 | | - JsonElement path = element.getAsJsonObject().get("latestVersion"); |
| 76 | + JsonElement versionElement = element.getAsJsonObject().get("latestVersion"); |
| 77 | + JsonElement pathElement = element.getAsJsonObject().get("graphDataPaths"); |
76 | 78 | if (importElement == null |
77 | | - || path == null |
| 79 | + || versionElement == null |
| 80 | + || pathElement == null |
78 | 81 | || importElement.getAsString().isEmpty() |
79 | | - || path.getAsString().isEmpty()) { |
| 82 | + || versionElement.getAsString().isEmpty()) { |
80 | 83 | LOGGER.error("Invalid import input json: {}", element.toString()); |
81 | 84 | continue; |
82 | 85 | } |
83 | 86 | String importName = importElement.getAsString(); |
| 87 | + String latestVersion = versionElement.getAsString(); |
| 88 | + LOGGER.info("Import: {} Latest version: {}", importName, latestVersion); |
| 89 | + |
| 90 | + // Populate provenance node/edges. |
84 | 91 | String provenance = "dc/base/" + importName; |
85 | | - LOGGER.info("Import {} graph path {}", importName, path.getAsString()); |
| 92 | + PCollection<McfGraph> provenanceMcf = |
| 93 | + PipelineUtils.readMcfFiles(latestVersion + IMPORT_METADATA_FILE, pipeline); |
| 94 | + PCollection<Mutation> provenanceNode = |
| 95 | + GraphReader.graphToNodes( |
| 96 | + provenanceMcf, spannerClient, nodeCounter, nodeInvalidTypeCounter) |
| 97 | + .apply("ExtractProvenanceNode", Values.create()); |
| 98 | + PCollection<Mutation> provenanceEdges = |
| 99 | + GraphReader.graphToEdges(provenanceMcf, provenance, spannerClient, edgeCounter) |
| 100 | + .apply("ExtractProvenanceEdges", Values.create()); |
| 101 | + nodeMutationList.add(provenanceNode); |
| 102 | + edgeMutationList.add(provenanceEdges); |
86 | 103 |
|
87 | 104 | PCollection<Mutation> deleteMutations = |
88 | 105 | GraphReader.getDeleteMutations(importName, provenance, pipeline, spannerClient); |
89 | 106 | deleteMutationList.add(deleteMutations); |
90 | 107 | // Read schema mcf files and combine MCF nodes, and convert to spanner mutations (Node/Edge). |
91 | | - PCollection<McfGraph> nodes = PipelineUtils.readMcfFiles(path.getAsString(), pipeline); |
92 | | - PCollectionTuple graphNodes = PipelineUtils.splitGraph(nodes); |
93 | | - PCollection<McfGraph> observationNodes = graphNodes.get(PipelineUtils.OBSERVATION_NODES_TAG); |
94 | | - PCollection<McfGraph> schemaNodes = graphNodes.get(PipelineUtils.SCHEMA_NODES_TAG); |
95 | | - PCollection<McfGraph> combinedGraph = PipelineUtils.combineGraphNodes(schemaNodes); |
96 | | - PCollection<Mutation> nodeMutations = |
97 | | - GraphReader.graphToNodes( |
98 | | - combinedGraph, spannerClient, nodeCounter, nodeInvalidTypeCounter) |
99 | | - .apply("ExtractNodeMutations", Values.create()); |
100 | | - PCollection<Mutation> edgeMutations = |
101 | | - GraphReader.graphToEdges(combinedGraph, provenance, spannerClient, edgeCounter) |
102 | | - .apply("ExtractEdgeMutations", Values.create()); |
103 | | - nodeMutationList.add(nodeMutations); |
104 | | - edgeMutationList.add(edgeMutations); |
105 | | - |
106 | | - // Read observation mcf files, build optimized graph, and convert to spanner mutations |
107 | | - // (Observation). |
108 | | - PCollection<McfOptimizedGraph> optimizedGraph = |
109 | | - PipelineUtils.buildOptimizedMcfGraph(observationNodes); |
110 | | - PCollection<Mutation> observationMutations = |
111 | | - GraphReader.graphToObservations(optimizedGraph, importName, spannerClient, obsCounter) |
112 | | - .apply("ExtractObsMutations", Values.create()); |
113 | | - obsMutationList.add(observationMutations); |
| 108 | + for (JsonElement path : pathElement.getAsJsonArray()) { |
| 109 | + String graphPath = latestVersion + path.getAsString(); |
| 110 | + PCollection<McfGraph> nodes = |
| 111 | + path.getAsString().contains("tfrecord") |
| 112 | + ? PipelineUtils.readMcfGraph(graphPath, pipeline) |
| 113 | + : PipelineUtils.readMcfFiles(graphPath, pipeline); |
| 114 | + PCollectionTuple graphNodes = PipelineUtils.splitGraph(nodes); |
| 115 | + PCollection<McfGraph> observationNodes = |
| 116 | + graphNodes.get(PipelineUtils.OBSERVATION_NODES_TAG); |
| 117 | + PCollection<McfGraph> schemaNodes = graphNodes.get(PipelineUtils.SCHEMA_NODES_TAG); |
| 118 | + PCollection<McfGraph> combinedGraph = PipelineUtils.combineGraphNodes(schemaNodes); |
| 119 | + PCollection<Mutation> nodeMutations = |
| 120 | + GraphReader.graphToNodes( |
| 121 | + combinedGraph, spannerClient, nodeCounter, nodeInvalidTypeCounter) |
| 122 | + .apply("ExtractNodeMutations", Values.create()); |
| 123 | + PCollection<Mutation> edgeMutations = |
| 124 | + GraphReader.graphToEdges(combinedGraph, provenance, spannerClient, edgeCounter) |
| 125 | + .apply("ExtractEdgeMutations", Values.create()); |
| 126 | + nodeMutationList.add(nodeMutations); |
| 127 | + edgeMutationList.add(edgeMutations); |
| 128 | + // Read observation mcf files, build optimized graph, and convert to spanner mutations |
| 129 | + // (Observation). |
| 130 | + PCollection<McfOptimizedGraph> optimizedGraph = |
| 131 | + PipelineUtils.buildOptimizedMcfGraph(observationNodes); |
| 132 | + PCollection<Mutation> observationMutations = |
| 133 | + GraphReader.graphToObservations(optimizedGraph, importName, spannerClient, obsCounter) |
| 134 | + .apply("ExtractObsMutations", Values.create()); |
| 135 | + obsMutationList.add(observationMutations); |
| 136 | + } |
114 | 137 | } |
115 | 138 | PCollection<Mutation> deleteMutations = |
116 | 139 | PCollectionList.of(deleteMutationList) |
|
0 commit comments