removed remaining ruby proxy classes, java proxies now use JRuby API directly

This commit is contained in:
Colin Surprenant 2014-03-02 03:04:01 -05:00
parent 36a74844f7
commit ba19200e6b
25 changed files with 479 additions and 689 deletions

View File

@ -0,0 +1,34 @@
module RedStorm
module DSL
class BatchBolt < Bolt
attr_reader :id
def self.java_proxy; "Java::RedstormStormJruby::JRubyBatchBolt"; end
def self.on_finish_batch(method_name = nil, &on_finish_batch_block)
body = block_given? ? on_finish_batch_block : lambda {self.send((method_name || :on_finish_batch).to_sym)}
define_method(:on_finish_batch, body)
end
def prepare(config, context, collector, id)
@collector = collector
@context = context
@config = config
@id = id
on_init
end
def finish_batch
on_finish_batch
end
private
# default noop optional dsl callbacks
def on_finish_batch; end
end
end
end

View File

@ -0,0 +1,9 @@
module RedStorm
module DSL
class BatchCommitterBolt < BatchBolt
def self.java_proxy; "Java::RedstormStormJruby::JRubyBatchCommitterBolt"; end
end
end
end

View File

@ -0,0 +1,53 @@
module RedStorm
module DSL
class BatchSpout < Spout
def self.java_proxy; "Java::RedstormStormJruby::JRubyBatchSpout"; end
def get_output_fields
Fields.new(self.class.fields)
end
def self.on_emit_batch(*args, &on_emit_batch_block)
options = args.last.is_a?(Hash) ? args.pop : {}
method_name = args.first
self.on_emit_batch_options.merge!(options)
# indirecting through a lambda defers the method lookup at invocation time
# and the performance penalty is negligible
body = block_given? ? on_emit_batch_block : lambda{|batch_id, collector| self.send((method_name || :on_emit_batch).to_sym)}
define_method(:on_emit_batch, body)
end
# Spout proxy interface
#
# note that in batch spout, ack is for the batch id and not the message id as in the base spout.
# TODO maybe rename msg_id to just id in the base spout
#
def emit_batch(batch_id, collector)
# TODO this is a TridentCollector, emit should just work by setting @collector
# report_error need to be hooked?
@collector = collector
on_emit_batch(batch_id, collector)
end
def open(config, context)
@context = context
@config = config
on_init
end
private
def self.on_emit_batch_options
@on_emit_batch_options ||= {}
end
end
end
end

View File

@ -14,6 +14,8 @@ module RedStorm
class Bolt class Bolt
attr_reader :collector, :context, :config attr_reader :collector, :context, :config
def self.java_proxy; "Java::RedstormStormJruby::JRubyBolt"; end
# DSL class methods # DSL class methods
def self.log def self.log

View File

@ -11,6 +11,8 @@ module RedStorm
class Spout class Spout
attr_reader :config, :context, :collector attr_reader :config, :context, :collector
def self.java_proxy; "Java::RedstormStormJruby::JRubySpout"; end
# DSL class methods # DSL class methods
def self.configure(&configure_block) def self.configure(&configure_block)
@ -99,7 +101,7 @@ module RedStorm
unreliable_emit(*values) unreliable_emit(*values)
end end
else else
sleep(0.1) sleep(0.1) # see https://twitter.com/colinsurprenant/status/406445541904494592
end end
end end
end end

View File

@ -46,7 +46,7 @@ module RedStorm
elsif is_java? elsif is_java?
@clazz.new(*constructor_args) @clazz.new(*constructor_args)
else else
JRubySpout.new(@clazz.base_class_path, @clazz.name, @output_fields) Object.module_eval(@clazz.java_proxy).new(@clazz.base_class_path, @clazz.name, @output_fields)
end end
end end
end end
@ -95,7 +95,7 @@ module RedStorm
elsif is_java? elsif is_java?
@clazz.new(*constructor_args) @clazz.new(*constructor_args)
else else
JRubyBolt.new(@clazz.base_class_path, @clazz.name, @output_fields) Object.module_eval(@clazz.java_proxy).new(@clazz.base_class_path, @clazz.name, @output_fields)
end end
end end
end end

View File

@ -23,6 +23,8 @@ class TupleImpl
end end
alias_method :[], :value alias_method :[], :value
alias_method :values, :getValues # wire directly to avoid method missing indirection
def field_index(field) def field_index(field)
fieldIndex(field.to_s) fieldIndex(field.to_s)
end end

View File

@ -1,59 +0,0 @@
require 'java'
java_import 'backtype.storm.coordination.BatchOutputCollector'
java_import 'backtype.storm.task.TopologyContext'
java_import 'backtype.storm.topology.IRichBolt'
java_import 'backtype.storm.coordination.IBatchBolt'
java_import 'backtype.storm.topology.OutputFieldsDeclarer'
java_import 'backtype.storm.tuple.Tuple'
java_import 'java.util.Map'
java_package 'redstorm.proxy'
# the Bolt class is a proxy to the real bolt to avoid having to deal with all the
# Java artifacts when creating a bolt.
#
# The real bolt class implementation must define these methods:
# - prepare(conf, context, collector)
# - execute(tuple)
# - declare_output_fields
#
# and optionnaly:
# - cleanup
#
class BatchBolt
java_implements IBatchBolt
java_signature 'IBatchBolt (String base_class_path, String real_bolt_class_name)'
def initialize(base_class_path, real_bolt_class_name)
@real_bolt = Object.module_eval(real_bolt_class_name).new
rescue NameError
require base_class_path
@real_bolt = Object.module_eval(real_bolt_class_name).new
end
java_signature 'void prepare(Map, TopologyContext, BatchOutputCollector, Object)'
def prepare(conf, context, collector, id)
@real_bolt.prepare(conf, context, collector, id)
end
java_signature 'void execute(Tuple)'
def execute(tuple)
@real_bolt.execute(tuple)
end
java_signature 'void finishBatch()'
def finishBatch
@real_bolt.finish_batch if @real_bolt.respond_to?(:finish_batch)
end
java_signature 'void declareOutputFields(OutputFieldsDeclarer)'
def declareOutputFields(declarer)
@real_bolt.declare_output_fields(declarer)
end
java_signature 'Map<String, Object> getComponentConfiguration()'
def getComponentConfiguration
@real_bolt.get_component_configuration
end
end

View File

@ -1,48 +0,0 @@
require 'java'
java_import 'backtype.storm.coordination.BatchOutputCollector'
java_import 'backtype.storm.task.TopologyContext'
java_import 'backtype.storm.coordination.IBatchBolt'
java_import 'backtype.storm.transactional.ICommitter'
java_import 'backtype.storm.topology.OutputFieldsDeclarer'
java_import 'backtype.storm.tuple.Tuple'
java_import 'java.util.Map'
java_package 'redstorm.proxy'
class BatchCommitterBolt
java_implements 'ICommitter, IBatchBolt'
java_signature 'IBatchCommitterBolt (String base_class_path, String real_bolt_class_name)'
def initialize(base_class_path, real_bolt_class_name)
@real_bolt = Object.module_eval(real_bolt_class_name).new
rescue NameError
require base_class_path
@real_bolt = Object.module_eval(real_bolt_class_name).new
end
java_signature 'void prepare(Map, TopologyContext, BatchOutputCollector, Object)'
def prepare(conf, context, collector, id)
@real_bolt.prepare(conf, context, collector, id)
end
java_signature 'void execute(Tuple)'
def execute(tuple)
@real_bolt.execute(tuple)
end
java_signature 'void finishBatch()'
def finishBatch
@real_bolt.finish_batch if @real_bolt.respond_to?(:finish_batch)
end
java_signature 'void declareOutputFields(OutputFieldsDeclarer)'
def declareOutputFields(declarer)
@real_bolt.declare_output_fields(declarer)
end
java_signature 'Map<String, Object> getComponentConfiguration()'
def getComponentConfiguration
@real_bolt.get_component_configuration
end
end

View File

@ -1,55 +0,0 @@
require 'java'
java_import 'backtype.storm.task.TopologyContext'
java_import 'storm.trident.operation.TridentCollector'
java_import 'storm.trident.spout.IBatchSpout'
java_import 'backtype.storm.tuple.Fields'
java_import 'java.util.Map'
java_package 'redstorm.proxy'
# the Spout class is a proxy to the real spout to avoid having to deal with all the
# Java artifacts when creating a spout.
class BatchSpout
java_implements IBatchSpout
java_signature 'IBatchSpout (String base_class_path, String real_spout_class_name)'
def initialize(base_class_path, real_spout_class_name)
@real_spout = Object.module_eval(real_spout_class_name).new
rescue NameError
require base_class_path
@real_spout = Object.module_eval(real_spout_class_name).new
end
java_signature 'void open(Map, TopologyContext)'
def open(conf, context)
@real_spout.open(conf, context) if @real_spout.respond_to?(:open)
end
java_signature 'void emitBatch(long, TridentCollector)'
def emitBatch(batch_id, collector)
@real_spout.emit_batch(batch_id, collector)
end
java_signature 'void close()'
def close
@real_spout.close if @real_spout.respond_to?(:close)
end
java_signature 'void ack(long)'
def ack(batch_id)
@real_spout.ack(batch_id) if @real_spout.respond_to?(:ack)
end
java_signature 'Fields getOutputFields()'
def getOutputFields()
@real_spout.get_output_fields
end
java_signature 'Map<String, Object> getComponentConfiguration()'
def getComponentConfiguration
@real_spout.get_component_configuration
end
end

View File

@ -1,60 +0,0 @@
require 'java'
java_import 'backtype.storm.task.OutputCollector'
java_import 'backtype.storm.task.TopologyContext'
java_import 'backtype.storm.topology.IRichBolt'
java_import 'backtype.storm.topology.OutputFieldsDeclarer'
java_import 'backtype.storm.tuple.Tuple'
java_import 'backtype.storm.tuple.Fields'
java_import 'backtype.storm.tuple.Values'
java_import 'java.util.Map'
java_package 'redstorm.proxy'
# the Bolt class is a proxy to the real bolt to avoid having to deal with all the
# Java artifacts when creating a bolt.
#
# The real bolt class implementation must define these methods:
# - prepare(conf, context, collector)
# - execute(tuple)
# - declare_output_fields
#
# and optionnaly:
# - cleanup
#
class Bolt
java_implements IRichBolt
java_signature 'IRichBolt (String base_class_path, String real_bolt_class_name)'
def initialize(base_class_path, real_bolt_class_name)
@real_bolt = Object.module_eval(real_bolt_class_name).new
rescue NameError
require base_class_path
@real_bolt = Object.module_eval(real_bolt_class_name).new
end
java_signature 'void prepare(Map, TopologyContext, OutputCollector)'
def prepare(conf, context, collector)
@real_bolt.prepare(conf, context, collector)
end
java_signature 'void execute(Tuple)'
def execute(tuple)
@real_bolt.execute(tuple)
end
java_signature 'void cleanup()'
def cleanup
@real_bolt.cleanup if @real_bolt.respond_to?(:cleanup)
end
java_signature 'void declareOutputFields(OutputFieldsDeclarer)'
def declareOutputFields(declarer)
@real_bolt.declare_output_fields(declarer)
end
java_signature 'Map<String, Object> getComponentConfiguration()'
def getComponentConfiguration
@real_bolt.get_component_configuration
end
end

View File

@ -1,36 +0,0 @@
require 'java'
java_import 'storm.trident.tuple.TridentTuple'
java_import 'storm.trident.operation.TridentCollector'
java_import 'storm.trident.operation.TridentOperationContext'
java_import 'storm.trident.operation.Function'
java_import 'java.util.Map'
java_package 'redstorm.proxy'
class ProxyFunction
java_implements Function
java_signature 'Function (String base_class_path, String real_class_name)'
def initialize(base_class_path, real_class_name)
@real = Object.module_eval(real_class_name).new
rescue NameError
require base_class_path
@real = Object.module_eval(real_class_name).new
end
java_signature 'void execute(TridentTuple, TridentCollector)'
def execute(_trident_tuple, _trident_collector)
@real.execute(_trident_tuple, _trident_collector)
end
java_signature 'void cleanup()'
def cleanup()
@real.cleanup()
end
java_signature 'void prepare(Map, TridentOperationContext)'
def prepare(_map, _trident_operation_context)
@real.prepare(_map, _trident_operation_context)
end
end

View File

@ -1,84 +0,0 @@
require 'java'
java_import 'backtype.storm.spout.SpoutOutputCollector'
java_import 'backtype.storm.task.TopologyContext'
java_import 'backtype.storm.topology.IRichSpout'
java_import 'backtype.storm.topology.OutputFieldsDeclarer'
java_import 'backtype.storm.tuple.Tuple'
java_import 'backtype.storm.tuple.Fields'
java_import 'backtype.storm.tuple.Values'
java_import 'java.util.Map'
java_package 'redstorm.proxy'
# the Spout class is a proxy to the real spout to avoid having to deal with all the
# Java artifacts when creating a spout.
#
# The real spout class implementation must define these methods:
# - open(conf, context, collector)
# - next_tuple
# - declare_output_fields
#
# and optionnaly:
# - ack(msg_id)
# - fail(msg_id)
# - close
#
class Spout
java_implements IRichSpout
java_signature 'IRichSpout (String base_class_path, String real_spout_class_name)'
def initialize(base_class_path, real_spout_class_name)
@real_spout = Object.module_eval(real_spout_class_name).new
rescue NameError
require base_class_path
@real_spout = Object.module_eval(real_spout_class_name).new
end
java_signature 'void open(Map, TopologyContext, SpoutOutputCollector)'
def open(conf, context, collector)
@real_spout.open(conf, context, collector)
end
java_signature 'void close()'
def close
@real_spout.close if @real_spout.respond_to?(:close)
end
java_signature 'void activate()'
def activate
@real_spout.activate if @real_spout.respond_to?(:activate)
end
java_signature 'void deactivate()'
def deactivate
@real_spout.deactivate if @real_spout.respond_to?(:deactivate)
end
java_signature 'void nextTuple()'
def nextTuple
@real_spout.next_tuple
end
java_signature 'void ack(Object)'
def ack(msg_id)
@real_spout.ack(msg_id) if @real_spout.respond_to?(:ack)
end
java_signature 'void fail(Object)'
def fail(msg_id)
@real_spout.fail(msg_id) if @real_spout.respond_to?(:fail)
end
java_signature 'void declareOutputFields(OutputFieldsDeclarer)'
def declareOutputFields(declarer)
@real_spout.declare_output_fields(declarer)
end
java_signature 'Map<String, Object> getComponentConfiguration()'
def getComponentConfiguration
@real_spout.get_component_configuration
end
end

View File

@ -1,43 +0,0 @@
require 'java'
java_import 'backtype.storm.task.TopologyContext'
java_import 'backtype.storm.transactional.ITransactionalSpout'
java_import 'backtype.storm.transactional.ICommitterTransactionalSpout'
java_import 'backtype.storm.topology.OutputFieldsDeclarer'
java_import 'java.util.Map'
java_package 'redstorm.proxy'
class TransactionalCommitterSpout
java_implements 'ICommitterTransactionalSpout'
java_signature 'ICommitterTransactionalSpout (String base_class_path, String real_spout_class_name)'
def initialize(base_class_path, real_spout_class_name)
@real_spout = Object.module_eval(real_spout_class_name).new
rescue NameError
require base_class_path
@real_spout = Object.module_eval(real_spout_class_name).new
end
java_signature 'ICommitterTransactionalSpout.Emitter getEmitter(Map, TopologyContext)'
def getEmitter(conf, context)
@real_spout.get_emitter(conf, context)
end
java_signature 'ITransactionalSpout.Coordinator getCoordinator(Map, TopologyContext)'
def getCoordinator(conf, context)
@real_spout.get_coordinator(conf, context)
end
java_signature 'void declareOutputFields(OutputFieldsDeclarer)'
def declareOutputFields(declarer)
@real_spout.declare_output_fields(declarer)
end
java_signature 'Map<String, Object> getComponentConfiguration()'
def getComponentConfiguration
@real_spout.get_component_configuration
end
end

View File

@ -1,41 +0,0 @@
require 'java'
java_import 'backtype.storm.task.TopologyContext'
java_import 'backtype.storm.transactional.ITransactionalSpout'
java_import 'backtype.storm.topology.OutputFieldsDeclarer'
java_import 'java.util.Map'
java_package 'redstorm.proxy'
class TransactionalSpout
java_implements 'ITransactionalSpout'
java_signature 'ITransactionalSpout (String base_class_path, String real_spout_class_name)'
def initialize(base_class_path, real_spout_class_name)
@real_spout = Object.module_eval(real_spout_class_name).new
rescue NameError
require base_class_path
@real_spout = Object.module_eval(real_spout_class_name).new
end
java_signature 'ITransactionalSpout.Emitter getEmitter(Map, TopologyContext)'
def getEmitter(conf, context)
@real_spout.get_emitter(conf, context)
end
java_signature 'ITransactionalSpout.Coordinator getCoordinator(Map, TopologyContext)'
def getCoordinator(conf, context)
@real_spout.get_coordinator(conf, context)
end
java_signature 'void declareOutputFields(OutputFieldsDeclarer)'
def declareOutputFields(declarer)
@real_spout.declare_output_fields(declarer)
end
java_signature 'Map<String, Object> getComponentConfiguration()'
def getComponentConfiguration
@real_spout.get_component_configuration
end
end

View File

@ -20,14 +20,14 @@ java_import 'backtype.storm.tuple.Fields'
java_import 'backtype.storm.tuple.Tuple' java_import 'backtype.storm.tuple.Tuple'
java_import 'backtype.storm.tuple.Values' java_import 'backtype.storm.tuple.Values'
java_import 'redstorm.storm.jruby.JRubyBolt' # java_import 'redstorm.storm.jruby.JRubyBolt'
java_import 'redstorm.storm.jruby.JRubySpout' # java_import 'redstorm.storm.jruby.JRubySpout'
java_import 'redstorm.storm.jruby.JRubyBatchBolt' # java_import 'redstorm.storm.jruby.JRubyBatchBolt'
java_import 'redstorm.storm.jruby.JRubyBatchCommitterBolt' # java_import 'redstorm.storm.jruby.JRubyBatchCommitterBolt'
java_import 'redstorm.storm.jruby.JRubyBatchSpout' # java_import 'redstorm.storm.jruby.JRubyBatchSpout'
java_import 'redstorm.storm.jruby.JRubyTransactionalSpout' # java_import 'redstorm.storm.jruby.JRubyTransactionalSpout'
java_import 'redstorm.storm.jruby.JRubyTransactionalBolt' # java_import 'redstorm.storm.jruby.JRubyTransactionalBolt'
java_import 'redstorm.storm.jruby.JRubyTransactionalCommitterBolt' # java_import 'redstorm.storm.jruby.JRubyTransactionalCommitterBolt'
java_package 'redstorm' java_package 'redstorm'

View File

@ -10,81 +10,99 @@ import backtype.storm.tuple.Tuple;
import backtype.storm.tuple.Fields; import backtype.storm.tuple.Fields;
import java.util.Map; import java.util.Map;
import org.jruby.Ruby;
import org.jruby.RubyObject;
import org.jruby.runtime.Helpers;
import org.jruby.runtime.builtin.IRubyObject;
import org.jruby.javasupport.JavaUtil;
import org.jruby.RubyModule;
import org.jruby.exceptions.RaiseException;
/** /**
* the JRubyBolt class is a simple proxy class to the actual bolt implementation in JRuby. * the JRubyBolt class is a proxy class to the actual bolt implementation in JRuby.
* this proxy is required to bypass the serialization/deserialization process when dispatching * this proxy is required to bypass the serialization/deserialization issues of JRuby objects.
* the bolts to the workers. JRuby does not yet support serialization from Java * JRuby classes do not support Java serialization.
* (Java serialization call on a JRuby class).
* *
* Note that the JRuby bolt proxy class is instanciated in the prepare method which is called after * Note that the JRuby bolt class is instanciated in the prepare method which is called after
* deserialization at the worker and in the declareOutputFields method which is called once before * deserialization at the worker and in the declareOutputFields & getComponentConfiguration
* serialization at topology creation. * methods which are called once before serialization at topology creation.
*/ */
public class JRubyBatchBolt extends BaseBatchBolt { public class JRubyBatchBolt extends BaseBatchBolt {
IBatchBolt _proxyBolt; private final String _realBoltClassName;
String _realBoltClassName; private final String[] _fields;
String _baseClassPath; private final String _bootstrap;
String[] _fields;
// transient to avoid serialization
private transient IRubyObject _ruby_bolt;
private transient Ruby __ruby__;
/** /**
* create a new JRubyBolt * create a new JRubyBolt
* *
* @param baseClassPath the topology/project base JRuby class file path * @param baseClassPath the topology/project base JRuby class file path
* @param realBoltClassName the fully qualified JRuby bolt implementation class name * @param realBoltClassName the fully qualified JRuby bolt implementation class name
* @param fields the output fields names
*/ */
public JRubyBatchBolt(String baseClassPath, String realBoltClassName, String[] fields) { public JRubyBatchBolt(String baseClassPath, String realBoltClassName, String[] fields) {
_baseClassPath = baseClassPath;
_realBoltClassName = realBoltClassName; _realBoltClassName = realBoltClassName;
_fields = fields; _fields = fields;
_bootstrap = "require '" + baseClassPath + "'";
} }
@Override @Override
public void prepare(final Map stormConf, final TopologyContext context, final BatchOutputCollector collector, final Object id) { public void prepare(final Map conf, final TopologyContext context, final BatchOutputCollector collector, final Object id) {
// create instance of the jruby class here, after deserialization in the workers. _ruby_bolt = initialize_ruby_bolt();
_proxyBolt = newProxyBolt(_baseClassPath, _realBoltClassName); IRubyObject ruby_conf = JavaUtil.convertJavaToRuby(__ruby__, conf);
_proxyBolt.prepare(stormConf, context, collector, id); IRubyObject ruby_context = JavaUtil.convertJavaToRuby(__ruby__, context);
IRubyObject ruby_collector = JavaUtil.convertJavaToRuby(__ruby__, collector);
IRubyObject ruby_id = JavaUtil.convertJavaToRuby(__ruby__, id);
Helpers.invoke(__ruby__.getCurrentContext(), _ruby_bolt, "prepare", ruby_conf, ruby_context, ruby_collector, ruby_id);
} }
@Override @Override
public void execute(Tuple input) { public void execute(Tuple input) {
_proxyBolt.execute(input); IRubyObject ruby_input = JavaUtil.convertJavaToRuby(__ruby__, input);
Helpers.invoke(__ruby__.getCurrentContext(), _ruby_bolt, "execute", ruby_input);
} }
@Override @Override
public void finishBatch() { public void finishBatch() {
_proxyBolt.finishBatch(); Helpers.invoke(__ruby__.getCurrentContext(), _ruby_bolt, "finish_batch");
} }
@Override @Override
public void declareOutputFields(OutputFieldsDeclarer declarer) { public void declareOutputFields(OutputFieldsDeclarer declarer) {
// declareOutputFields is executed in the topology creation time, before serialisation.
// do not set the _proxyBolt instance variable here to avoid JRuby serialization
// issues. Just create tmp bolt instance to call declareOutputFields.
if (_fields.length > 0) { if (_fields.length > 0) {
declarer.declare(new Fields(_fields)); declarer.declare(new Fields(_fields));
} else { } else {
IBatchBolt bolt = newProxyBolt(_baseClassPath, _realBoltClassName); IRubyObject ruby_bolt = initialize_ruby_bolt();
bolt.declareOutputFields(declarer); IRubyObject ruby_declarer = JavaUtil.convertJavaToRuby(__ruby__, declarer);
Helpers.invoke(__ruby__.getCurrentContext(), ruby_bolt, "declare_output_fields", ruby_declarer);
} }
} }
@Override @Override
public Map<String, Object> getComponentConfiguration() { public Map<String, Object> getComponentConfiguration() {
// getComponentConfiguration is executed in the topology creation time, before serialisation. // getComponentConfiguration is executed in the topology creation time, before serialisation.
// do not set the _proxyBolt instance variable here to avoid JRuby serialization // just create tmp bolt instance to call getComponentConfiguration.
// issues. Just create tmp bolt instance to call declareOutputFields.
IBatchBolt bolt = newProxyBolt(_baseClassPath, _realBoltClassName);
return bolt.getComponentConfiguration();
}
private static IBatchBolt newProxyBolt(String baseClassPath, String realBoltClassName) { IRubyObject ruby_bolt = initialize_ruby_bolt();
IRubyObject ruby_result = Helpers.invoke(__ruby__.getCurrentContext(), ruby_bolt, "get_component_configuration");
return (Map)ruby_result.toJava(Map.class);
}
private IRubyObject initialize_ruby_bolt() {
__ruby__ = Ruby.getGlobalRuntime();
RubyModule ruby_class;
try { try {
redstorm.proxy.BatchBolt proxy = new redstorm.proxy.BatchBolt(baseClassPath, realBoltClassName); ruby_class = __ruby__.getClassFromPath(_realBoltClassName);
return proxy;
} }
catch (Exception e) { catch (RaiseException e) {
throw new RuntimeException(e); // after deserialization we need to recreate ruby environment
__ruby__.evalScriptlet(_bootstrap);
ruby_class = __ruby__.getClassFromPath(_realBoltClassName);
} }
return Helpers.invoke(__ruby__.getCurrentContext(), ruby_class, "new");
} }
} }

View File

@ -6,83 +6,118 @@ import storm.trident.operation.TridentCollector;
import storm.trident.spout.IBatchSpout; import storm.trident.spout.IBatchSpout;
import java.util.Map; import java.util.Map;
import org.jruby.Ruby;
import org.jruby.RubyObject;
import org.jruby.runtime.Helpers;
import org.jruby.runtime.builtin.IRubyObject;
import org.jruby.javasupport.JavaUtil;
import org.jruby.RubyModule;
import org.jruby.exceptions.RaiseException;
/** /**
* the JRubySpout class is a simple proxy class to the actual spout implementation in JRuby. * the JRubySpout class is a proxy class to the actual spout implementation in JRuby.
* this proxy is required to bypass the serialization/deserialization process when dispatching * this proxy is required to bypass the serialization/deserialization issues of JRuby objects.
* the spout to the workers. JRuby does not yet support serialization from Java * JRuby classes do not support Java serialization.
* (Java serialization call on a JRuby class).
* *
* Note that the JRuby spout proxy class is instanciated in the open method which is called after * Note that the JRuby spout class is instanciated in the open method which is called after
* deserialization at the worker and in both the declareOutputFields and isDistributed methods which * deserialization at the worker and in the declareOutputFields & getComponentConfiguration
* are called once before serialization at topology creation. * methods which are called once before serialization at topology creation.
*/ */
public class JRubyBatchSpout implements IBatchSpout { public class JRubyBatchSpout implements IBatchSpout {
IBatchSpout _proxySpout; private final String _realSpoutClassName;
String _realSpoutClassName; private final String[] _fields;
String _baseClassPath; private final String _bootstrap;
// transient to avoid serialization
private transient IRubyObject _ruby_spout;
private transient Ruby __ruby__;
/** /**
* create a new JRubySpout * create a new JRubyBatchSpout
* *
* @param baseClassPath the topology/project base JRuby class file path * @param baseClassPath the topology/project base JRuby class file path
* @param realSpoutClassName the fully qualified JRuby spout implementation class name * @param realSpoutClassName the fully qualified JRuby spout implementation class name
* @param fields the output fields names
*/ */
public JRubyBatchSpout(String baseClassPath, String realSpoutClassName) { public JRubyBatchSpout(String baseClassPath, String realSpoutClassName) {
_baseClassPath = baseClassPath;
_realSpoutClassName = realSpoutClassName; _realSpoutClassName = realSpoutClassName;
_fields = null;
_bootstrap = "require '" + baseClassPath + "'";
}
/* constructor for compatibility with JRubySpout signature */
public JRubyBatchSpout(String baseClassPath, String realSpoutClassName, String[] fields) {
_realSpoutClassName = realSpoutClassName;
_fields = fields;
_bootstrap = "require '" + baseClassPath + "'";
} }
@Override @Override
public void open(final Map conf, final TopologyContext context) { public void open(final Map conf, final TopologyContext context) {
// create instance of the jruby proxy class here, after deserialization in the workers. // // create instance of the jruby proxy class here, after deserialization in the workers.
_proxySpout = newProxySpout(_baseClassPath, _realSpoutClassName); // _proxySpout = newProxySpout(_baseClassPath, _realSpoutClassName);
_proxySpout.open(conf, context); // _proxySpout.open(conf, context);
_ruby_spout = initialize_ruby_spout();
IRubyObject ruby_conf = JavaUtil.convertJavaToRuby(__ruby__, conf);
IRubyObject ruby_context = JavaUtil.convertJavaToRuby(__ruby__, context);
Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "open", ruby_conf, ruby_context);
} }
@Override @Override
public void emitBatch(long batchId, TridentCollector collector) { public void emitBatch(long batchId, TridentCollector collector) {
_proxySpout.emitBatch(batchId, collector); // _proxySpout.emitBatch(batchId, collector);
}
IRubyObject ruby_batch_id = JavaUtil.convertJavaToRuby(__ruby__, batchId);
IRubyObject ruby_collector = JavaUtil.convertJavaToRuby(__ruby__, collector);
Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "emit_batch", ruby_batch_id, ruby_collector);
}
@Override @Override
public void close() { public void close() {
_proxySpout.close(); // _proxySpout.close();
Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "close");
} }
@Override @Override
public void ack(long batchId) { public void ack(long batchId) {
_proxySpout.ack(batchId); // _proxySpout.ack(batchId);
IRubyObject ruby_batch_id = JavaUtil.convertJavaToRuby(__ruby__, batchId);
Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "ack", ruby_batch_id);
} }
@Override @Override
public Fields getOutputFields() { public Fields getOutputFields() {
if (_proxySpout == null) { IRubyObject ruby_spout = initialize_ruby_spout();
// getOutputFields is executed in the topology creation time before serialisation. IRubyObject ruby_result = Helpers.invoke(__ruby__.getCurrentContext(), ruby_spout, "get_output_fields");
// do not set the _proxySpout instance variable here to avoid JRuby serialization return (Fields)ruby_result.toJava(Fields.class);
// issues. Just create tmp spout instance to call getOutputFields.
IBatchSpout spout = newProxySpout(_baseClassPath, _realSpoutClassName);
return spout.getOutputFields();
} else {
return _proxySpout.getOutputFields();
}
} }
@Override @Override
public Map<String, Object> getComponentConfiguration() { public Map<String, Object> getComponentConfiguration() {
// getComponentConfiguration is executed in the topology creation time before serialisation. // getComponentConfiguration is executed in the topology creation time, before serialisation.
// do not set the _proxySpout instance variable here to avoid JRuby serialization // just create tmp spout instance to call getComponentConfiguration.
// issues. Just create tmp spout instance to call declareOutputFields.
IBatchSpout spout = newProxySpout(_baseClassPath, _realSpoutClassName); IRubyObject ruby_spout = initialize_ruby_spout();
return spout.getComponentConfiguration(); IRubyObject ruby_result = Helpers.invoke(__ruby__.getCurrentContext(), ruby_spout, "get_component_configuration");
return (Map)ruby_result.toJava(Map.class);
} }
private static IBatchSpout newProxySpout(String baseClassPath, String realSpoutClassName) { private IRubyObject initialize_ruby_spout() {
__ruby__ = Ruby.getGlobalRuntime();
RubyModule ruby_class;
try { try {
redstorm.proxy.BatchSpout proxy = new redstorm.proxy.BatchSpout(baseClassPath, realSpoutClassName); ruby_class = __ruby__.getClassFromPath(_realSpoutClassName);
return proxy;
} }
catch (Exception e) { catch (RaiseException e) {
throw new RuntimeException(e); // after deserialization we need to recreate ruby environment
__ruby__.evalScriptlet(_bootstrap);
ruby_class = __ruby__.getClassFromPath(_realSpoutClassName);
} }
return Helpers.invoke(__ruby__.getCurrentContext(), ruby_class, "new");
} }
} }

View File

@ -19,13 +19,12 @@ import org.jruby.exceptions.RaiseException;
/** /**
* the JRubyBolt class is a proxy class to the actual bolt implementation in JRuby. * the JRubyBolt class is a proxy class to the actual bolt implementation in JRuby.
* this proxy is required to bypass the serialization/deserialization issues of JRuby objects. * this proxy is required to bypass the serialization/deserialization issues of JRuby objects.
* JRuby classes are not support Java serialization. * JRuby classes do not support Java serialization.
* *
* Note that the JRuby bolt class is instanciated in the prepare method which is called after * Note that the JRuby bolt class is instanciated in the prepare method which is called after
* deserialization at the worker and in the declareOutputFields & getComponentConfiguration * deserialization at the worker and in the declareOutputFields & getComponentConfiguration
* methods which are called once before serialization at topology creation. * methods which are called once before serialization at topology creation.
*/ */
public class JRubyBolt implements IRichBolt { public class JRubyBolt implements IRichBolt {
private final String _realBoltClassName; private final String _realBoltClassName;
private final String[] _fields; private final String[] _fields;

View File

@ -6,46 +6,64 @@ import java.util.Map;
import storm.trident.operation.TridentOperationContext; import storm.trident.operation.TridentOperationContext;
import storm.trident.operation.Function; import storm.trident.operation.Function;
import org.jruby.Ruby;
import org.jruby.RubyObject;
import org.jruby.runtime.Helpers;
import org.jruby.runtime.builtin.IRubyObject;
import org.jruby.javasupport.JavaUtil;
import org.jruby.RubyModule;
import org.jruby.exceptions.RaiseException;
public class JRubyProxyFunction implements Function { public class JRubyProxyFunction implements Function {
Function _proxy; private final String _realClassName;
String _realClassName; private final String _bootstrap;
String _baseClassPath;
// transient to avoid serialization
private transient IRubyObject _proxy;
private transient Ruby __ruby__;
public JRubyProxyFunction(final String baseClassPath, final String realClassName) { public JRubyProxyFunction(final String baseClassPath, final String realClassName) {
_baseClassPath = baseClassPath;
_realClassName = realClassName; _realClassName = realClassName;
_bootstrap = "require '" + baseClassPath + "'";
} }
@Override @Override
public void execute(final TridentTuple _tridentTuple, final TridentCollector _tridentCollector) { public void execute(final TridentTuple tuple, final TridentCollector collector) {
if(_proxy == null) { if(_proxy == null) {
_proxy = newProxy(_baseClassPath, _realClassName); _proxy = initialize_proxy();
} }
_proxy.execute(_tridentTuple, _tridentCollector); IRubyObject ruby_tuple = JavaUtil.convertJavaToRuby(__ruby__, tuple);
IRubyObject ruby_collector = JavaUtil.convertJavaToRuby(__ruby__, collector);
Helpers.invoke(__ruby__.getCurrentContext(), _proxy, "execute", ruby_tuple, ruby_collector);
} }
@Override @Override
public void cleanup() { public void cleanup() {
_proxy.cleanup(); Helpers.invoke(__ruby__.getCurrentContext(), _proxy, "cleanup");
} }
@Override @Override
public void prepare(final Map _map, final TridentOperationContext _tridentOperationContext) { public void prepare(final Map conf, final TridentOperationContext context) {
if(_proxy == null) { if(_proxy == null) {
_proxy = newProxy(_baseClassPath, _realClassName); _proxy = initialize_proxy();
} }
_proxy.prepare(_map, _tridentOperationContext); IRubyObject ruby_conf = JavaUtil.convertJavaToRuby(__ruby__, conf);
IRubyObject ruby_context = JavaUtil.convertJavaToRuby(__ruby__, context);
Helpers.invoke(__ruby__.getCurrentContext(), _proxy, "prepare", ruby_conf, ruby_context);
} }
private IRubyObject initialize_proxy() {
__ruby__ = Ruby.getGlobalRuntime();
private static Function newProxy(final String baseClassPath, final String realClassName) { RubyModule ruby_class;
try { try {
redstorm.proxy.ProxyFunction proxy = new redstorm.proxy.ProxyFunction(baseClassPath, realClassName); ruby_class = __ruby__.getClassFromPath(_realClassName);
return proxy;
} }
catch (Exception e) { catch (RaiseException e) {
throw new RuntimeException(e); // after deserialization we need to recreate ruby environment
__ruby__.evalScriptlet(_bootstrap);
ruby_class = __ruby__.getClassFromPath(_realClassName);
} }
return Helpers.invoke(__ruby__.getCurrentContext(), ruby_class, "new");
} }
} }

View File

@ -8,100 +8,122 @@ import backtype.storm.tuple.Tuple;
import backtype.storm.tuple.Fields; import backtype.storm.tuple.Fields;
import java.util.Map; import java.util.Map;
import org.jruby.Ruby;
import org.jruby.RubyObject;
import org.jruby.runtime.Helpers;
import org.jruby.runtime.builtin.IRubyObject;
import org.jruby.javasupport.JavaUtil;
import org.jruby.RubyModule;
import org.jruby.exceptions.RaiseException;
/** /**
* the JRubySpout class is a simple proxy class to the actual spout implementation in JRuby. * the JRubySpout class is a proxy class to the actual spout implementation in JRuby.
* this proxy is required to bypass the serialization/deserialization process when dispatching * this proxy is required to bypass the serialization/deserialization issues of JRuby objects.
* the spout to the workers. JRuby does not yet support serialization from Java * JRuby classes do not support Java serialization.
* (Java serialization call on a JRuby class).
* *
* Note that the JRuby spout proxy class is instanciated in the open method which is called after * Note that the JRuby spout class is instanciated in the open method which is called after
* deserialization at the worker and in both the declareOutputFields and isDistributed methods which * deserialization at the worker and in the declareOutputFields & getComponentConfiguration
* are called once before serialization at topology creation. * methods which are called once before serialization at topology creation.
*/ */
public class JRubySpout implements IRichSpout { public class JRubySpout implements IRichSpout {
IRichSpout _proxySpout; private final String _realSpoutClassName;
String _realSpoutClassName; private final String[] _fields;
String _baseClassPath; private final String _bootstrap;
String[] _fields;
// transient to avoid serialization
private transient IRubyObject _ruby_spout;
private transient Ruby __ruby__;
/** /**
* create a new JRubySpout * create a new JRubySpout
* *
* @param baseClassPath the topology/project base JRuby class file path * @param baseClassPath the topology/project base JRuby class file path
* @param realSpoutClassName the fully qualified JRuby spout implementation class name * @param realSpoutClassName the fully qualified JRuby spout implementation class name
* @param fields the output fields names
*/ */
public JRubySpout(String baseClassPath, String realSpoutClassName, String[] fields) { public JRubySpout(String baseClassPath, String realSpoutClassName, String[] fields) {
_baseClassPath = baseClassPath;
_realSpoutClassName = realSpoutClassName; _realSpoutClassName = realSpoutClassName;
_fields = fields; _fields = fields;
_bootstrap = "require '" + baseClassPath + "'";
} }
@Override @Override
public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) { public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
// create instance of the jruby proxy class here, after deserialization in the workers. _ruby_spout = initialize_ruby_spout();
_proxySpout = newProxySpout(_baseClassPath, _realSpoutClassName); IRubyObject ruby_conf = JavaUtil.convertJavaToRuby(__ruby__, conf);
_proxySpout.open(conf, context, collector); IRubyObject ruby_context = JavaUtil.convertJavaToRuby(__ruby__, context);
IRubyObject ruby_collector = JavaUtil.convertJavaToRuby(__ruby__, collector);
Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "open", ruby_conf, ruby_context, ruby_collector);
} }
@Override @Override
public void close() { public void close() {
_proxySpout.close(); Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "close");
} }
@Override @Override
public void activate() { public void activate() {
_proxySpout.activate(); Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "activate");
} }
@Override @Override
public void deactivate() { public void deactivate() {
_proxySpout.deactivate(); Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "deactivate");
} }
@Override @Override
public void nextTuple() { public void nextTuple() {
_proxySpout.nextTuple(); Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "next_tuple");
} }
@Override @Override
public void ack(Object msgId) { public void ack(Object msgId) {
_proxySpout.ack(msgId); IRubyObject ruby_msg_id = JavaUtil.convertJavaToRuby(__ruby__, msgId);
Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "ack", ruby_msg_id);
} }
@Override @Override
public void fail(Object msgId) { public void fail(Object msgId) {
_proxySpout.fail(msgId); IRubyObject ruby_msg_id = JavaUtil.convertJavaToRuby(__ruby__, msgId);
Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "fail", ruby_msg_id);
} }
@Override @Override
public void declareOutputFields(OutputFieldsDeclarer declarer) { public void declareOutputFields(OutputFieldsDeclarer declarer) {
// declareOutputFields is executed in the topology creation time before serialisation. // declareOutputFields is executed in the topology creation time, before serialisation.
// do not set the _proxySpout instance variable here to avoid JRuby serialization // just create tmp spout instance to call declareOutputFields.
// issues. Just create tmp spout instance to call declareOutputFields.
if (_fields.length > 0) { if (_fields.length > 0) {
declarer.declare(new Fields(_fields)); declarer.declare(new Fields(_fields));
} else { } else {
IRichSpout spout = newProxySpout(_baseClassPath, _realSpoutClassName); IRubyObject ruby_spout = initialize_ruby_spout();
spout.declareOutputFields(declarer); IRubyObject ruby_declarer = JavaUtil.convertJavaToRuby(__ruby__, declarer);
Helpers.invoke(__ruby__.getCurrentContext(), ruby_spout, "declare_output_fields", ruby_declarer);
} }
} }
@Override @Override
public Map<String, Object> getComponentConfiguration() { public Map<String, Object> getComponentConfiguration() {
// getComponentConfiguration is executed in the topology creation time before serialisation. // getComponentConfiguration is executed in the topology creation time, before serialisation.
// do not set the _proxySpout instance variable here to avoid JRuby serialization // just create tmp spout instance to call getComponentConfiguration.
// issues. Just create tmp spout instance to call declareOutputFields.
IRichSpout spout = newProxySpout(_baseClassPath, _realSpoutClassName); IRubyObject ruby_spout = initialize_ruby_spout();
return spout.getComponentConfiguration(); IRubyObject ruby_result = Helpers.invoke(__ruby__.getCurrentContext(), ruby_spout, "get_component_configuration");
return (Map)ruby_result.toJava(Map.class);
} }
private static IRichSpout newProxySpout(String baseClassPath, String realSpoutClassName) { private IRubyObject initialize_ruby_spout() {
__ruby__ = Ruby.getGlobalRuntime();
RubyModule ruby_class;
try { try {
redstorm.proxy.Spout proxy = new redstorm.proxy.Spout(baseClassPath, realSpoutClassName); ruby_class = __ruby__.getClassFromPath(_realSpoutClassName);
return proxy;
} }
catch (Exception e) { catch (RaiseException e) {
throw new RuntimeException(e); // after deserialization we need to recreate ruby environment
__ruby__.evalScriptlet(_bootstrap);
ruby_class = __ruby__.getClassFromPath(_realSpoutClassName);
} }
return Helpers.invoke(__ruby__.getCurrentContext(), ruby_class, "new");
} }
} }

View File

@ -11,80 +11,102 @@ import backtype.storm.tuple.Tuple;
import backtype.storm.tuple.Fields; import backtype.storm.tuple.Fields;
import java.util.Map; import java.util.Map;
import org.jruby.Ruby;
import org.jruby.RubyObject;
import org.jruby.runtime.Helpers;
import org.jruby.runtime.builtin.IRubyObject;
import org.jruby.javasupport.JavaUtil;
import org.jruby.RubyModule;
import org.jruby.exceptions.RaiseException;
/** /**
* the JRubyBolt class is a simple proxy class to the actual bolt implementation in JRuby. * the JRubyBolt class is a proxy class to the actual bolt implementation in JRuby.
* this proxy is required to bypass the serialization/deserialization process when dispatching * this proxy is required to bypass the serialization/deserialization issues of JRuby objects.
* the bolts to the workers. JRuby does not yet support serialization from Java * JRuby classes do not support Java serialization.
* (Java serialization call on a JRuby class).
* *
* Note that the JRuby bolt proxy class is instanciated in the prepare method which is called after * Note that the JRuby bolt class is instanciated in the prepare method which is called after
* deserialization at the worker and in the declareOutputFields method which is called once before * deserialization at the worker and in the declareOutputFields & getComponentConfiguration
* serialization at topology creation. * methods which are called once before serialization at topology creation.
*/ */
public class JRubyTransactionalBolt extends BaseTransactionalBolt { public class JRubyTransactionalBolt extends BaseTransactionalBolt {
IBatchBolt _proxyBolt; private final String _realBoltClassName;
String _realBoltClassName; private final String[] _fields;
String _baseClassPath; private final String _bootstrap;
String[] _fields;
/** // transient to avoid serialization
protected transient IRubyObject _ruby_bolt;
protected transient Ruby __ruby__;
/**
* create a new JRubyBolt * create a new JRubyBolt
* *
* @param baseClassPath the topology/project base JRuby class file path * @param baseClassPath the topology/project base JRuby class file path
* @param realBoltClassName the fully qualified JRuby bolt implementation class name * @param realBoltClassName the fully qualified JRuby bolt implementation class name
* @param fields the output fields names
*/ */
public JRubyTransactionalBolt(String baseClassPath, String realBoltClassName, String[] fields) { public JRubyTransactionalBolt(String baseClassPath, String realBoltClassName, String[] fields) {
_baseClassPath = baseClassPath;
_realBoltClassName = realBoltClassName; _realBoltClassName = realBoltClassName;
_fields = fields; _fields = fields;
_bootstrap = "require '" + baseClassPath + "'";
} }
@Override @Override
public void prepare(final Map stormConf, final TopologyContext context, final BatchOutputCollector collector, final TransactionAttempt id) { public void prepare(final Map conf, final TopologyContext context, final BatchOutputCollector collector, final TransactionAttempt id) {
// create instance of the jruby class here, after deserialization in the workers. _ruby_bolt = initialize_ruby_bolt();
_proxyBolt = newProxyBolt(_baseClassPath, _realBoltClassName); IRubyObject ruby_conf = JavaUtil.convertJavaToRuby(__ruby__, conf);
_proxyBolt.prepare(stormConf, context, collector, id); IRubyObject ruby_context = JavaUtil.convertJavaToRuby(__ruby__, context);
IRubyObject ruby_collector = JavaUtil.convertJavaToRuby(__ruby__, collector);
IRubyObject ruby_id = JavaUtil.convertJavaToRuby(__ruby__, id);
Helpers.invoke(__ruby__.getCurrentContext(), _ruby_bolt, "prepare", ruby_conf, ruby_context, ruby_collector, ruby_id);
} }
@Override @Override
public void execute(Tuple input) { public void execute(Tuple input) {
_proxyBolt.execute(input); IRubyObject ruby_input = JavaUtil.convertJavaToRuby(__ruby__, input);
Helpers.invoke(__ruby__.getCurrentContext(), _ruby_bolt, "execute", ruby_input);
} }
@Override @Override
public void finishBatch() { public void finishBatch() {
_proxyBolt.finishBatch(); Helpers.invoke(__ruby__.getCurrentContext(), _ruby_bolt, "finish_batch");
} }
@Override @Override
public void declareOutputFields(OutputFieldsDeclarer declarer) { public void declareOutputFields(OutputFieldsDeclarer declarer) {
// declareOutputFields is executed in the topology creation time, before serialisation. // declareOutputFields is executed in the topology creation time, before serialisation.
// do not set the _proxyBolt instance variable here to avoid JRuby serialization // just create tmp bolt instance to call declareOutputFields.
// issues. Just create tmp bolt instance to call declareOutputFields.
if (_fields.length > 0) { if (_fields.length > 0) {
declarer.declare(new Fields(_fields)); declarer.declare(new Fields(_fields));
} else { } else {
IBatchBolt bolt = newProxyBolt(_baseClassPath, _realBoltClassName); IRubyObject ruby_bolt = initialize_ruby_bolt();
bolt.declareOutputFields(declarer); IRubyObject ruby_declarer = JavaUtil.convertJavaToRuby(__ruby__, declarer);
Helpers.invoke(__ruby__.getCurrentContext(), ruby_bolt, "declare_output_fields", ruby_declarer);
} }
} }
@Override @Override
public Map<String, Object> getComponentConfiguration() { public Map<String, Object> getComponentConfiguration() {
// getComponentConfiguration is executed in the topology creation time, before serialisation. // getComponentConfiguration is executed in the topology creation time, before serialisation.
// do not set the _proxyBolt instance variable here to avoid JRuby serialization // just create tmp bolt instance to call getComponentConfiguration.
// issues. Just create tmp bolt instance to call declareOutputFields.
IBatchBolt bolt = newProxyBolt(_baseClassPath, _realBoltClassName); IRubyObject ruby_bolt = initialize_ruby_bolt();
return bolt.getComponentConfiguration(); IRubyObject ruby_result = Helpers.invoke(__ruby__.getCurrentContext(), ruby_bolt, "get_component_configuration");
return (Map)ruby_result.toJava(Map.class);
} }
private static IBatchBolt newProxyBolt(String baseClassPath, String realBoltClassName) { protected IRubyObject initialize_ruby_bolt() {
__ruby__ = Ruby.getGlobalRuntime();
RubyModule ruby_class;
try { try {
redstorm.proxy.BatchBolt proxy = new redstorm.proxy.BatchBolt(baseClassPath, realBoltClassName); ruby_class = __ruby__.getClassFromPath(_realBoltClassName);
return proxy;
} }
catch (Exception e) { catch (RaiseException e) {
throw new RuntimeException(e); // after deserialization we need to recreate ruby environment
__ruby__.evalScriptlet(_bootstrap);
ruby_class = __ruby__.getClassFromPath(_realBoltClassName);
} }
return Helpers.invoke(__ruby__.getCurrentContext(), ruby_class, "new");
} }
} }

View File

@ -5,27 +5,16 @@ import backtype.storm.transactional.TransactionAttempt;
import backtype.storm.transactional.ICommitter; import backtype.storm.transactional.ICommitter;
/** /**
* the JRubyBolt class is a simple proxy class to the actual bolt implementation in JRuby. * the JRubyBolt class is a proxy class to the actual bolt implementation in JRuby.
* this proxy is required to bypass the serialization/deserialization process when dispatching * this proxy is required to bypass the serialization/deserialization issues of JRuby objects.
* the bolts to the workers. JRuby does not yet support serialization from Java * JRuby classes do not support Java serialization.
* (Java serialization call on a JRuby class).
* *
* Note that the JRuby bolt proxy class is instanciated in the prepare method which is called after * Note that the JRuby bolt class is instanciated in the prepare method which is called after
* deserialization at the worker and in the declareOutputFields method which is called once before * deserialization at the worker and in the declareOutputFields & getComponentConfiguration
* serialization at topology creation. * methods which are called once before serialization at topology creation.
*/ */
public class JRubyTransactionalCommitterBolt extends JRubyTransactionalBolt implements ICommitter { public class JRubyTransactionalCommitterBolt extends JRubyTransactionalBolt implements ICommitter {
public JRubyTransactionalCommitterBolt(String baseClassPath, String realBoltClassName, String[] fields) { public JRubyTransactionalCommitterBolt(String baseClassPath, String realBoltClassName, String[] fields) {
super(baseClassPath, realBoltClassName, fields); super(baseClassPath, realBoltClassName, fields);
} }
private static IBatchBolt newProxyBolt(String baseClassPath, String realBoltClassName) {
try {
redstorm.proxy.BatchCommitterBolt proxy = new redstorm.proxy.BatchCommitterBolt(baseClassPath, realBoltClassName);
return proxy;
}
catch (Exception e) {
throw new RuntimeException(e);
}
}
} }

View File

@ -5,40 +5,28 @@ import backtype.storm.transactional.ITransactionalSpout;
import backtype.storm.task.TopologyContext; import backtype.storm.task.TopologyContext;
import java.util.Map; import java.util.Map;
/** import org.jruby.Ruby;
* the JRubyBolt class is a simple proxy class to the actual bolt implementation in JRuby. import org.jruby.RubyObject;
* this proxy is required to bypass the serialization/deserialization process when dispatching import org.jruby.runtime.Helpers;
* the bolts to the workers. JRuby does not yet support serialization from Java import org.jruby.runtime.builtin.IRubyObject;
* (Java serialization call on a JRuby class). import org.jruby.javasupport.JavaUtil;
* import org.jruby.RubyModule;
* Note that the JRuby bolt proxy class is instanciated in the prepare method which is called after import org.jruby.exceptions.RaiseException;
* deserialization at the worker and in the declareOutputFields method which is called once before
* serialization at topology creation.
*/
public class JRubyTransactionalCommitterSpout extends JRubyTransactionalSpout implements ICommitterTransactionalSpout { public class JRubyTransactionalCommitterSpout extends JRubyTransactionalSpout implements ICommitterTransactionalSpout {
ICommitterTransactionalSpout _proxySpout;
public JRubyTransactionalCommitterSpout(String baseClassPath, String realSpoutClassName, String[] fields) { public JRubyTransactionalCommitterSpout(String baseClassPath, String realSpoutClassName, String[] fields) {
super(baseClassPath, realSpoutClassName, fields); super(baseClassPath, realSpoutClassName, fields);
} }
@Override @Override
public ICommitterTransactionalSpout.Emitter getEmitter(Map conf, TopologyContext context) { public ICommitterTransactionalSpout.Emitter getEmitter(Map conf, TopologyContext context) {
// create instance of the jruby class here, after deserialization in the workers. if (_ruby_spout == null) {
if (_proxySpout == null) { IRubyObject _ruby_spout = initialize_ruby_spout();
_proxySpout = newProxySpout(_baseClassPath, _realSpoutClassName);
}
return _proxySpout.getEmitter(conf, context);
}
private static ICommitterTransactionalSpout newProxySpout(String baseClassPath, String realSpoutClassName) {
try {
redstorm.proxy.TransactionalCommitterSpout proxy = new redstorm.proxy.TransactionalCommitterSpout(baseClassPath, realSpoutClassName);
return proxy;
}
catch (Exception e) {
throw new RuntimeException(e);
} }
IRubyObject ruby_conf = JavaUtil.convertJavaToRuby(__ruby__, conf);
IRubyObject ruby_context = JavaUtil.convertJavaToRuby(__ruby__, context);
IRubyObject ruby_result = Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "get_emitter", ruby_conf, ruby_context);
return (ICommitterTransactionalSpout.Emitter)ruby_result.toJava(ICommitterTransactionalSpout.Emitter.class);
} }
} }

View File

@ -9,81 +9,104 @@ import backtype.storm.tuple.Tuple;
import backtype.storm.tuple.Fields; import backtype.storm.tuple.Fields;
import java.util.Map; import java.util.Map;
import org.jruby.Ruby;
import org.jruby.RubyObject;
import org.jruby.runtime.Helpers;
import org.jruby.runtime.builtin.IRubyObject;
import org.jruby.javasupport.JavaUtil;
import org.jruby.RubyModule;
import org.jruby.exceptions.RaiseException;
/** /**
* the JRubySpout class is a simple proxy class to the actual spout implementation in JRuby. * the JRubySpout class is a simple proxy class to the actual spout implementation in JRuby.
* this proxy is required to bypass the serialization/deserialization process when dispatching * this proxy is required to bypass the serialization/deserialization process when dispatching
* the spout to the workers. JRuby does not yet support serialization from Java * the spout to the workers. JRuby does not yet support serialization from Java
* (Java serialization call on a JRuby class). * (Java serialization call on a JRuby class).
* *
* Note that the JRuby spout proxy class is instanciated in the open method which is called after * Note that the JRuby spout proxy class is instanciated in the open method which is called after
* deserialization at the worker and in both the declareOutputFields and isDistributed methods which * deserialization at the worker and in both the declareOutputFields and isDistributed methods which
* are called once before serialization at topology creation. * are called once before serialization at topology creation.
*/ */
public class JRubyTransactionalSpout extends BaseTransactionalSpout { public class JRubyTransactionalSpout extends BaseTransactionalSpout {
ITransactionalSpout _proxySpout; private final String _realSpoutClassName;
String _realSpoutClassName; private final String[] _fields;
String _baseClassPath; private final String _bootstrap;
String[] _fields;
// transient to avoid serialization
protected transient IRubyObject _ruby_spout;
protected transient Ruby __ruby__;
/** /**
* create a new JRubySpout * create a new JRubySpout
* *
* @param baseClassPath the topology/project base JRuby class file path * @param baseClassPath the topology/project base JRuby class file path
* @param realSpoutClassName the fully qualified JRuby spout implementation class name * @param realSpoutClassName the fully qualified JRuby spout implementation class name
* @param fields the output fields names
*/ */
public JRubyTransactionalSpout(String baseClassPath, String realSpoutClassName, String[] fields) { public JRubyTransactionalSpout(String baseClassPath, String realSpoutClassName, String[] fields) {
_baseClassPath = baseClassPath;
_realSpoutClassName = realSpoutClassName; _realSpoutClassName = realSpoutClassName;
_fields = fields; _fields = fields;
_bootstrap = "require '" + baseClassPath + "'";
} }
@Override @Override
public ITransactionalSpout.Coordinator getCoordinator(Map conf, TopologyContext context) { public ITransactionalSpout.Coordinator getCoordinator(Map conf, TopologyContext context) {
// create instance of the jruby class here, after deserialization in the workers. if (_ruby_spout == null) {
if (_proxySpout == null) { IRubyObject _ruby_spout = initialize_ruby_spout();
_proxySpout = newProxySpout(_baseClassPath, _realSpoutClassName);
} }
return _proxySpout.getCoordinator(conf, context); IRubyObject ruby_conf = JavaUtil.convertJavaToRuby(__ruby__, conf);
IRubyObject ruby_context = JavaUtil.convertJavaToRuby(__ruby__, context);
IRubyObject ruby_result = Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "get_coordinator", ruby_conf, ruby_context);
return (ITransactionalSpout.Coordinator)ruby_result.toJava(ITransactionalSpout.Coordinator.class);
} }
@Override @Override
public ITransactionalSpout.Emitter getEmitter(Map conf, TopologyContext context) { public ITransactionalSpout.Emitter getEmitter(Map conf, TopologyContext context) {
// create instance of the jruby class here, after deserialization in the workers. if (_ruby_spout == null) {
if (_proxySpout == null) { IRubyObject _ruby_spout = initialize_ruby_spout();
_proxySpout = newProxySpout(_baseClassPath, _realSpoutClassName);
} }
return _proxySpout.getEmitter(conf, context); IRubyObject ruby_conf = JavaUtil.convertJavaToRuby(__ruby__, conf);
IRubyObject ruby_context = JavaUtil.convertJavaToRuby(__ruby__, context);
IRubyObject ruby_result = Helpers.invoke(__ruby__.getCurrentContext(), _ruby_spout, "get_emitter", ruby_conf, ruby_context);
return (ITransactionalSpout.Emitter)ruby_result.toJava(ITransactionalSpout.Emitter.class);
} }
@Override @Override
public void declareOutputFields(OutputFieldsDeclarer declarer) { public void declareOutputFields(OutputFieldsDeclarer declarer) {
// declareOutputFields is executed in the topology creation time before serialisation. // declareOutputFields is executed in the topology creation time, before serialisation.
// do not set the _proxySpout instance variable here to avoid JRuby serialization // just create tmp spout instance to call declareOutputFields.
// issues. Just create tmp spout instance to call declareOutputFields.
if (_fields.length > 0) { if (_fields.length > 0) {
declarer.declare(new Fields(_fields)); declarer.declare(new Fields(_fields));
} else { } else {
ITransactionalSpout spout = newProxySpout(_baseClassPath, _realSpoutClassName); IRubyObject ruby_spout = initialize_ruby_spout();
spout.declareOutputFields(declarer); IRubyObject ruby_declarer = JavaUtil.convertJavaToRuby(__ruby__, declarer);
Helpers.invoke(__ruby__.getCurrentContext(), ruby_spout, "declare_output_fields", ruby_declarer);
} }
} }
@Override @Override
public Map<String, Object> getComponentConfiguration() { public Map<String, Object> getComponentConfiguration() {
// getComponentConfiguration is executed in the topology creation time before serialisation. // getComponentConfiguration is executed in the topology creation time, before serialisation.
// do not set the _proxySpout instance variable here to avoid JRuby serialization // just create tmp spout instance to call getComponentConfiguration.
// issues. Just create tmp spout instance to call declareOutputFields.
ITransactionalSpout spout = newProxySpout(_baseClassPath, _realSpoutClassName); IRubyObject ruby_spout = initialize_ruby_spout();
return spout.getComponentConfiguration(); IRubyObject ruby_result = Helpers.invoke(__ruby__.getCurrentContext(), ruby_spout, "get_component_configuration");
return (Map)ruby_result.toJava(Map.class);
} }
private static ITransactionalSpout newProxySpout(String baseClassPath, String realSpoutClassName) { protected IRubyObject initialize_ruby_spout() {
__ruby__ = Ruby.getGlobalRuntime();
RubyModule ruby_class;
try { try {
redstorm.proxy.TransactionalSpout proxy = new redstorm.proxy.TransactionalSpout(baseClassPath, realSpoutClassName); ruby_class = __ruby__.getClassFromPath(_realSpoutClassName);
return proxy;
} }
catch (Exception e) { catch (RaiseException e) {
throw new RuntimeException(e); // after deserialization we need to recreate ruby environment
__ruby__.evalScriptlet(_bootstrap);
ruby_class = __ruby__.getClassFromPath(_realSpoutClassName);
} }
return Helpers.invoke(__ruby__.getCurrentContext(), ruby_class, "new");
} }
} }