blob: 912b43cb6c14dcf5f02ed50f53d413957b657f6c [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.storm.beam.translation;
import org.apache.beam.sdk.io.Read;
import org.apache.beam.sdk.io.UnboundedSource;
import org.apache.storm.beam.StormPipelineOptions;
import org.apache.storm.beam.translation.runtime.UnboundedSourceSpout;
/**
* Translates a Read.Unbounded into a Storm spout.
* @param <T>
*/
public class UnboundedSourceTranslator<T> implements TransformTranslator<Read.Unbounded<T>> {
public void translateNode(Read.Unbounded<T> transform, TranslationContext context) {
UnboundedSource source = transform.getSource();
StormPipelineOptions options = context.getOptions();
UnboundedSourceSpout spout = new UnboundedSourceSpout(source, options);
String name = context.getCurrentTransform().getFullName();
context.addSpout(name, spout);
}
}