public static void main(String[] args) throws Exception { if (args.length < 4) { System.err.println( "Usage: PrintSampleStream <consumer-key> <consumer-secret> <access-token> <access-token-secret>"); return; } String consumerKey = args[0]; String consumerSecret = args[1]; String accessToken = args[2]; String accessTokenSecret = args[3]; // keywords start with the 5th parameter String[] keyWords = Arrays.copyOfRange(args, 4, args.length); TopologyBuilder builder = new TopologyBuilder(); builder.setSpout( "twitter", new TwitterSampleSpout( consumerKey, consumerSecret, accessToken, accessTokenSecret, keyWords)); builder.setBolt("print", new PrinterBolt()).shuffleGrouping("twitter"); Config conf = new Config(); final FlinkLocalCluster cluster = FlinkLocalCluster.getLocalCluster(); cluster.submitTopology("Print", conf, FlinkTopology.createTopology(builder)); Utils.sleep(10 * 1000); cluster.shutdown(); }
@SuppressWarnings({"unchecked", "rawtypes"}) static void addStormConfigToTopology(FlinkTopology topology, Map conf) throws ClassNotFoundException { if (conf != null) { ExecutionConfig flinkConfig = topology.getExecutionEnvironment().getConfig(); flinkConfig.setGlobalJobParameters(new StormConfig(conf)); // add all registered types to ExecutionConfig List<?> registeredClasses = (List<?>) conf.get(Config.TOPOLOGY_KRYO_REGISTER); if (registeredClasses != null) { for (Object klass : registeredClasses) { if (klass instanceof String) { flinkConfig.registerKryoType(Class.forName((String) klass)); } else { for (Entry<String, String> register : ((Map<String, String>) klass).entrySet()) { flinkConfig.registerTypeWithKryoSerializer( Class.forName(register.getKey()), (Class<? extends Serializer<?>>) Class.forName(register.getValue())); } } } } } }
@SuppressWarnings("rawtypes") public void submitTopologyWithOpts( final String topologyName, final Map conf, final FlinkTopology topology, final SubmitOptions submitOpts) throws Exception { LOG.info("Running Storm topology on FlinkLocalCluster"); if (conf != null) { topology.getConfig().setGlobalJobParameters(new StormConfig(conf)); } StreamGraph streamGraph = topology.getStreamGraph(); streamGraph.setJobName(topologyName); JobGraph jobGraph = streamGraph.getJobGraph(); this.flink.submitJobDetached(jobGraph); }
/** * Parameter {@code uploadedJarLocation} is actually used to point to the local jar, because Flink * does not support uploading a jar file before hand. Jar files are always uploaded directly when * a program is submitted. */ public void submitTopologyWithOpts( final String name, final String uploadedJarLocation, final FlinkTopology topology) throws AlreadyAliveException, InvalidTopologyException { if (this.getTopologyJobId(name) != null) { throw new AlreadyAliveException(); } final URI uploadedJarUri; final URL uploadedJarUrl; try { uploadedJarUri = new File(uploadedJarLocation).getAbsoluteFile().toURI(); uploadedJarUrl = uploadedJarUri.toURL(); JobWithJars.checkJarFile(uploadedJarUrl); } catch (final IOException e) { throw new RuntimeException("Problem with jar file " + uploadedJarLocation, e); } try { FlinkClient.addStormConfigToTopology(topology, conf); } catch (ClassNotFoundException e) { LOG.error("Could not register class for Kryo serialization.", e); throw new InvalidTopologyException("Could not register class for Kryo serialization."); } final StreamGraph streamGraph = topology.getExecutionEnvironment().getStreamGraph(); streamGraph.setJobName(name); final JobGraph jobGraph = streamGraph.getJobGraph(); jobGraph.addJar(new Path(uploadedJarUri)); final Configuration configuration = jobGraph.getJobConfiguration(); configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerHost); configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort); final Client client; try { client = new Client(configuration); } catch (IOException e) { throw new RuntimeException("Could not establish a connection to the job manager", e); } try { ClassLoader classLoader = JobWithJars.buildUserCodeClassLoader( Lists.newArrayList(uploadedJarUrl), Collections.<URL>emptyList(), this.getClass().getClassLoader()); client.runDetached(jobGraph, classLoader); } catch (final ProgramInvocationException e) { throw new RuntimeException("Cannot execute job due to ProgramInvocationException", e); } }