blob: 0edb21594cad5297e182f2ce207ef4f2a3ad4dcb [file] [log] [blame]
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.beam.sdk.extensions.euphoria.core.translate;
import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
import org.apache.beam.sdk.extensions.euphoria.core.client.operator.CompositeOperator;
import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionList;
/**
* A translator that expands {@link CompositeOperator composite operators}.
*
* @param <InputT> input type
* @param <OutputT> output type
* @param <OperatorT> operator type
*/
public class CompositeOperatorTranslator<InputT, OutputT, OperatorT extends Operator>
implements OperatorTranslator<InputT, OutputT, OperatorT> {
@SuppressWarnings("unchecked")
@Override
public PCollection<OutputT> translate(OperatorT operator, PCollectionList<InputT> inputs) {
checkState(operator instanceof CompositeOperator, "Operator is not composite.");
return ((CompositeOperator<InputT, OutputT>) operator).expand(inputs);
}
@Override
public boolean canTranslate(OperatorT operator) {
return operator instanceof CompositeOperator;
}
}