johnyangk commented on a change in pull request #104: [NEMO-183] DAG-centric translation from Beam pipeline to IR DAG URL: https://github.com/apache/incubator-nemo/pull/104#discussion_r210493130
########## File path: compiler/frontend/beam/src/main/java/edu/snu/nemo/compiler/frontend/beam/PipelineTranslator.java ########## @@ -0,0 +1,544 @@ +/* + * Copyright (C) 2018 Seoul National University + * + * Licensed 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 edu.snu.nemo.compiler.frontend.beam; + +import edu.snu.nemo.common.dag.DAG; +import edu.snu.nemo.common.dag.DAGBuilder; +import edu.snu.nemo.common.ir.edge.IREdge; +import edu.snu.nemo.common.ir.edge.executionproperty.*; +import edu.snu.nemo.common.ir.vertex.IRVertex; +import edu.snu.nemo.common.ir.vertex.LoopVertex; +import edu.snu.nemo.common.ir.vertex.OperatorVertex; +import edu.snu.nemo.common.ir.vertex.transform.Transform; +import edu.snu.nemo.compiler.frontend.beam.PipelineVisitor.*; +import edu.snu.nemo.compiler.frontend.beam.coder.BeamDecoderFactory; +import edu.snu.nemo.compiler.frontend.beam.coder.BeamEncoderFactory; +import edu.snu.nemo.compiler.frontend.beam.source.BeamBoundedSourceVertex; +import edu.snu.nemo.compiler.frontend.beam.transform.*; +import org.apache.beam.sdk.coders.*; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.*; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.values.*; + +import java.lang.annotation.*; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Stack; +import java.util.function.BiFunction; + +/** + * Converts DAG of Beam pipeline to Nemo IR DAG. + * For a {@link PrimitiveTransformVertex}, it defines mapping to the corresponding {@link IRVertex}. + * For a {@link CompositeTransformVertex}, it defines how to setup and clear {@link TranslationContext} + * before start translating inner Beam transform hierarchy. + */ +public final class PipelineTranslator + implements BiFunction<CompositeTransformVertex, PipelineOptions, DAG<IRVertex, IREdge>> { + + private static final PipelineTranslator INSTANCE = new PipelineTranslator(); + + private final Map<Class<? extends PTransform>, Method> primitiveTransformToTranslator = new HashMap<>(); + private final Map<Class<? extends PTransform>, Method> compositeTransformToTranslator = new HashMap<>(); + + /** + * Static translator method. + * @param pipeline Top-level Beam transform hierarchy, usually given by {@link PipelineVisitor} + * @param pipelineOptions {@link PipelineOptions} + * @return Nemo IR DAG + */ + public static DAG<IRVertex, IREdge> translate(final CompositeTransformVertex pipeline, + final PipelineOptions pipelineOptions) { + return INSTANCE.apply(pipeline, pipelineOptions); + } + + /** + * Creates the translator, while building a map between {@link PTransform}s and the corresponding translators. + */ + private PipelineTranslator() { + for (final Method translator : getClass().getDeclaredMethods()) { + final PrimitiveTransformTranslator primitive = translator.getAnnotation(PrimitiveTransformTranslator.class); + final CompositeTransformTranslator composite = translator.getAnnotation(CompositeTransformTranslator.class); + if (primitive != null) { + for (final Class<? extends PTransform> transform : primitive.value()) { + if (primitiveTransformToTranslator.containsKey(transform)) { + throw new RuntimeException(String.format("Translator for primitive transform %s is" + + "already registered: %s", transform, primitiveTransformToTranslator.get(transform))); + } + primitiveTransformToTranslator.put(transform, translator); + } + } + if (composite != null) { + for (final Class<? extends PTransform> transform : composite.value()) { + if (compositeTransformToTranslator.containsKey(transform)) { + throw new RuntimeException(String.format("Translator for composite transform %s is" + + "already registered: %s", transform, compositeTransformToTranslator.get(transform))); + } + compositeTransformToTranslator.put(transform, translator); + } + } + } + } + + @PrimitiveTransformTranslator(Read.Bounded.class) + private static void boundedReadTranslator(final TranslationContext ctx, Review comment: +1 on a separate translation method for each type of Beam vertex. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
