From f6219945c9dc0de3705dab9a9273e2be924224ca Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Wed, 17 Jan 2018 11:33:40 -0800 Subject: [PATCH 01/28] support segments --- lib/ldclient-rb.rb | 4 +- lib/ldclient-rb/config.rb | 11 +- lib/ldclient-rb/evaluation.rb | 67 +++++++++-- .../{feature_store.rb => in_memory_store.rb} | 26 +++-- lib/ldclient-rb/ldclient.rb | 5 +- lib/ldclient-rb/polling.rb | 7 +- ...{redis_feature_store.rb => redis_store.rb} | 110 ++++++++++++------ lib/ldclient-rb/requestor.rb | 12 ++ lib/ldclient-rb/stream.rb | 54 +++++++-- spec/stream_spec.rb | 34 ++++-- 10 files changed, 247 insertions(+), 83 deletions(-) rename lib/ldclient-rb/{feature_store.rb => in_memory_store.rb} (65%) rename lib/ldclient-rb/{redis_feature_store.rb => redis_store.rb} (66%) diff --git a/lib/ldclient-rb.rb b/lib/ldclient-rb.rb index ed28621f..3f24012d 100644 --- a/lib/ldclient-rb.rb +++ b/lib/ldclient-rb.rb @@ -9,6 +9,6 @@ require "ldclient-rb/polling" require "ldclient-rb/event_serializer" require "ldclient-rb/events" -require "ldclient-rb/feature_store" -require "ldclient-rb/redis_feature_store" +require "ldclient-rb/in_memory_store" +require "ldclient-rb/redis_store" require "ldclient-rb/requestor" diff --git a/lib/ldclient-rb/config.rb b/lib/ldclient-rb/config.rb index adb50ff8..c449363f 100644 --- a/lib/ldclient-rb/config.rb +++ b/lib/ldclient-rb/config.rb @@ -66,6 +66,7 @@ def initialize(opts = {}) @connect_timeout = opts[:connect_timeout] || Config.default_connect_timeout @read_timeout = opts[:read_timeout] || Config.default_read_timeout @feature_store = opts[:feature_store] || Config.default_feature_store + @segment_store = opts[:segment_store] || Config.default_segment_store @stream = opts.has_key?(:stream) ? opts[:stream] : Config.default_stream @use_ldd = opts.has_key?(:use_ldd) ? opts[:use_ldd] : Config.default_use_ldd @offline = opts.has_key?(:offline) ? opts[:offline] : Config.default_offline @@ -171,7 +172,11 @@ def offline? # attr_reader :feature_store - + # + # A store for segment configuration rules. + # + attr_reader :segment_store + # The proxy configuration string # attr_reader :proxy @@ -252,6 +257,10 @@ def self.default_feature_store InMemoryFeatureStore.new end + def self.default_segment_store + InMemorySegmentStore.new + end + def self.default_offline false end diff --git a/lib/ldclient-rb/evaluation.rb b/lib/ldclient-rb/evaluation.rb index 4a6c942f..4ea184c3 100644 --- a/lib/ldclient-rb/evaluation.rb +++ b/lib/ldclient-rb/evaluation.rb @@ -68,6 +68,10 @@ module Evaluation rescue => e false end + end, + segmentMatch: + lambda do |a, b| + false # we should never reach this - instead we special-case this operator in clause_match_user end } @@ -78,7 +82,7 @@ class EvaluationError < StandardError # generated during prerequisite evaluation. Raises EvaluationError if the flag is not well-formed # Will return nil, but not raise an exception, indicating that the rules (including fallthrough) did not match # In that case, the caller should return the default value. - def evaluate(flag, user, store) + def evaluate(flag, user, feature_store, segment_store) if flag.nil? raise EvaluationError, "Flag does not exist" end @@ -90,7 +94,7 @@ def evaluate(flag, user, store) events = [] if flag[:on] - res = eval_internal(flag, user, store, events) + res = eval_internal(flag, user, feature_store, segment_store, events) return { value: res, events: events } if !res.nil? end @@ -103,18 +107,18 @@ def evaluate(flag, user, store) { value: nil, events: events } end - def eval_internal(flag, user, store, events) + def eval_internal(flag, user, feature_store, segment_store, events) failed_prereq = false # Evaluate prerequisites, if any if !flag[:prerequisites].nil? flag[:prerequisites].each do |prerequisite| - prereq_flag = store.get(prerequisite[:key]) + prereq_flag = feature_store.get(prerequisite[:key]) if prereq_flag.nil? || !prereq_flag[:on] failed_prereq = true else begin - prereq_res = eval_internal(prereq_flag, user, store, events) + prereq_res = eval_internal(prereq_flag, user, feature_store, segment_store, events) variation = get_variation(prereq_flag, prerequisite[:variation]) events.push(kind: "feature", key: prereq_flag[:key], value: prereq_res, version: prereq_flag[:version], prereqOf: flag[:key]) if prereq_res.nil? || prereq_res != variation @@ -134,10 +138,10 @@ def eval_internal(flag, user, store, events) # The prerequisites were satisfied. # Now walk through the evaluation steps and get the correct # variation index - eval_rules(flag, user) + eval_rules(flag, user, segment_store) end - def eval_rules(flag, user) + def eval_rules(flag, user, segment_store) # Check user target matches if !flag[:targets].nil? flag[:targets].each do |target| @@ -152,7 +156,7 @@ def eval_rules(flag, user) # Check custom rules if !flag[:rules].nil? flag[:rules].each do |rule| - return variation_for_user(rule, user, flag) if rule_match_user(rule, user) + return variation_for_user(rule, user, flag) if rule_match_user(rule, user, segment_store) end end @@ -172,17 +176,31 @@ def get_variation(flag, index) flag[:variations][index] end - def rule_match_user(rule, user) + def rule_match_user(rule, user, segment_store) return false if !rule[:clauses] rule[:clauses].each do |clause| - return false if !clause_match_user(clause, user) + return false if !clause_match_user(clause, user, segment_store) end return true end - def clause_match_user(clause, user) + def clause_match_user(clause, user, segment_store) + # In the case of a segment match operator, we check if the user is in any of the segments, + # and possibly negate + if (clause[:op] == :segmentMatch) + clause[:values].each do |v| + segment = segment_store.get(v) + if !segment.nil? + return maybe_negate(clause, true) if segment_match_user(segment, user) + end + end + end + clause_match_user_no_segments(clause, user) + end + + def clause_match_user_no_segments(clause, user) val = user_value(user, clause[:attribute]) return false if val.nil? @@ -220,6 +238,33 @@ def variation_for_user(rule, user, flag) end end + def segment_match_user(segment, user) + return false unless user[:key] + + return true if segment[:included].include?(user[:key]) + return false if segment[:excluded].include?(user[:key]) + + segment[:rules].each do |r| + return true if segment_rule_match_user(r, user, segment[:key], segment[:salt]) + end + + return false + end + + def segment_rule_match_user(rule, user, segment_key, salt) + rule[:clauses].each do |c| + return false unless clause_match_user_no_segments(c, user) + end + + # If the weight is absent, this rule matches + return true if !rule.weight + + # All of the clauses are met. See if the user buckets in + bucket = bucket_user(user, segment_key, rule[:bucketBy].nil ? "key" : rule[:bucketBy], salt) + weight = rule[:weight].to_f / 100000.0 + return bucket < weight + end + def bucket_user(user, key, bucket_by, salt) return nil unless user[:key] diff --git a/lib/ldclient-rb/feature_store.rb b/lib/ldclient-rb/in_memory_store.rb similarity index 65% rename from lib/ldclient-rb/feature_store.rb rename to lib/ldclient-rb/in_memory_store.rb index f1a17446..95851bdb 100644 --- a/lib/ldclient-rb/feature_store.rb +++ b/lib/ldclient-rb/in_memory_store.rb @@ -1,53 +1,53 @@ require "concurrent/atomics" module LaunchDarkly - class InMemoryFeatureStore + class InMemoryVersionedStore def initialize - @features = Hash.new + @items = Hash.new @lock = Concurrent::ReadWriteLock.new @initialized = Concurrent::AtomicBoolean.new(false) end def get(key) @lock.with_read_lock do - f = @features[key.to_sym] + f = @items[key.to_sym] (f.nil? || f[:deleted]) ? nil : f end end def all @lock.with_read_lock do - @features.select { |_k, f| not f[:deleted] } + @items.select { |_k, f| not f[:deleted] } end end def delete(key, version) @lock.with_write_lock do - old = @features[key.to_sym] + old = @items[key.to_sym] if !old.nil? && old[:version] < version old[:deleted] = true old[:version] = version - @features[key.to_sym] = old + @items[key.to_sym] = old elsif old.nil? - @features[key.to_sym] = { deleted: true, version: version } + @items[key.to_sym] = { deleted: true, version: version } end end end def init(fs) @lock.with_write_lock do - @features.replace(fs) + @items.replace(fs) @initialized.make_true end end def upsert(key, feature) @lock.with_write_lock do - old = @features[key.to_sym] + old = @items[key.to_sym] if old.nil? || old[:version] < feature[:version] - @features[key.to_sym] = feature + @items[key.to_sym] = feature end end end @@ -60,4 +60,10 @@ def stop # nothing to do end end + + class InMemoryFeatureStore < InMemoryVersionedStore + end + + class InMemorySegmentStore < InMemoryVersionedStore + end end diff --git a/lib/ldclient-rb/ldclient.rb b/lib/ldclient-rb/ldclient.rb index 0d6c7f73..34b6c760 100644 --- a/lib/ldclient-rb/ldclient.rb +++ b/lib/ldclient-rb/ldclient.rb @@ -27,6 +27,7 @@ def initialize(sdk_key, config = Config.default, wait_for_sec = 5) @sdk_key = sdk_key @config = config @store = config.feature_store + @segment_store = config.segment_store @event_processor = EventProcessor.new(sdk_key, config) @@ -139,7 +140,7 @@ def variation(key, user, default) end begin - res = evaluate(feature, user, @store) + res = evaluate(feature, user, @store, @segment_store) if !res[:events].nil? res[:events].each do |event| @event_processor.add_event(event) @@ -200,7 +201,7 @@ def all_flags(user) features = @store.all # TODO rescue if necessary - Hash[features.map{ |k, f| [k, evaluate(f, user, @store)[:value]] }] + Hash[features.map{ |k, f| [k, evaluate(f, user, @store, @segment_store)[:value]] }] rescue => exn @config.logger.warn("[LDClient] Error evaluating all flags: #{exn.inspect}. \nTrace: #{exn.backtrace}") return Hash.new diff --git a/lib/ldclient-rb/polling.rb b/lib/ldclient-rb/polling.rb index 060e72bf..8a2eecce 100644 --- a/lib/ldclient-rb/polling.rb +++ b/lib/ldclient-rb/polling.rb @@ -31,9 +31,10 @@ def stop end def poll - flags = @requestor.request_all_flags - if flags - @config.feature_store.init(flags) + allData = @requestor.request_all_data + if allData + @config.feature_store.init(allData.flags) + @config.segment_store.init(allData.segments) if @initialized.make_true @config.logger.info("[LDClient] Polling connection initialized") end diff --git a/lib/ldclient-rb/redis_feature_store.rb b/lib/ldclient-rb/redis_store.rb similarity index 66% rename from lib/ldclient-rb/redis_feature_store.rb rename to lib/ldclient-rb/redis_store.rb index 1fbedf10..8bd17ef8 100644 --- a/lib/ldclient-rb/redis_feature_store.rb +++ b/lib/ldclient-rb/redis_store.rb @@ -4,15 +4,9 @@ module LaunchDarkly # - # An implementation of the LaunchDarkly client's feature store that uses a Redis - # instance. Feature data can also be further cached in memory to reduce overhead - # of calls to Redis. + # Base class for all Redis versioned object stores. # - # To use this class, you must first have the `redis`, `connection-pool`, and `moneta` - # gems installed. Then, create an instance and store it in the `feature_store` - # property of your client configuration. - # - class RedisFeatureStore + class RedisVersionedStore begin require "redis" require "connection_pool" @@ -37,22 +31,22 @@ class RedisFeatureStore # def initialize(opts = {}) if !REDIS_ENABLED - raise RuntimeError.new("can't use RedisFeatureStore because one of these gems is missing: redis, connection_pool, moneta") + raise RuntimeError.new("can't use #{storeName} because one of these gems is missing: redis, connection_pool, moneta") end @redis_opts = opts[:redis_opts] || Hash.new if opts[:redis_url] @redis_opts[:url] = opts[:redis_url] end if !@redis_opts.include?(:url) - @redis_opts[:url] = RedisFeatureStore.default_redis_url + @redis_opts[:url] = RedisVersionedStore.default_redis_url end max_connections = opts[:max_connections] || 16 @pool = opts[:pool] || ConnectionPool.new(size: max_connections) do Redis.new(@redis_opts) end - @prefix = opts[:prefix] || RedisFeatureStore.default_prefix + @prefix = opts[:prefix] || RedisVersionedStore.default_prefix @logger = opts[:logger] || Config.default_logger - @features_key = @prefix + ':features' + @items_key = @prefix + baseKeySuffix @expiration_seconds = opts[:expiration] || 15 @capacity = opts[:capacity] || 1000 @@ -71,11 +65,17 @@ def initialize(opts = {}) } with_connection do |redis| - @logger.info("RedisFeatureStore: using Redis instance at #{redis.connection[:host]}:#{redis.connection[:port]} \ + @logger.info("#{storeName}: using Redis instance at #{redis.connection[:host]}:#{redis.connection[:port]} \ and prefix: #{@prefix}") end end + def baseKeySuffix + end + + def storeName + end + # # Default value for the `redis_url` constructor parameter; points to an instance of Redis # running at `localhost` with its default port. @@ -94,12 +94,12 @@ def self.default_prefix def get(key) f = @cache[key.to_sym] if f.nil? - @logger.debug("RedisFeatureStore: no cache hit for #{key}, requesting from Redis") + @logger.debug("#{storeName}: no cache hit for #{key}, requesting from Redis") f = with_connection do |redis| begin get_redis(redis,key.to_sym) rescue => e - @logger.error("RedisFeatureStore: could not retrieve feature #{key} from Redis, with error: #{e}") + @logger.error("#{storeName}: could not retrieve #{key} from Redis, with error: #{e}") nil end end @@ -108,10 +108,10 @@ def get(key) end end if f.nil? - @logger.warn("RedisFeatureStore: feature #{key} not found") + @logger.warn("#{storeName}: #{key} not found") nil elsif f[:deleted] - @logger.warn("RedisFeatureStore: feature #{key} was deleted, returning nil") + @logger.warn("#{storeName}: #{key} was deleted, returning nil") nil else f @@ -122,13 +122,13 @@ def all fs = {} with_connection do |redis| begin - hashfs = redis.hgetall(@features_key) + hashfs = redis.hgetall(@items_key) rescue => e - @logger.error("RedisFeatureStore: could not retrieve all flags from Redis with error: #{e}; returning none") + @logger.error("#{storeName}: could not retrieve all items from Redis with error: #{e}; returning none") hashfs = {} end - hashfs.each do |k, jsonFeature| - f = JSON.parse(jsonFeature, symbolize_names: true) + hashfs.each do |k, jsonItem| + f = JSON.parse(jsonItem, symbolize_names: true) if !f[:deleted] fs[k.to_sym] = f end @@ -149,7 +149,7 @@ def delete(key, version) f1[:version] = version put_redis_and_cache(redis, key, f1) else - @logger.warn("RedisFeatureStore: attempted to delete flag: #{key} version: #{f[:version]} \ + @logger.warn("#{storeName}: attempted to delete #{key} version: #{f[:version]} \ with a version that is the same or older: #{version}") end end @@ -160,20 +160,20 @@ def init(fs) @cache.clear with_connection do |redis| redis.multi do |multi| - redis.del(@features_key) + redis.del(@items_key) fs.each { |k, f| put_redis_and_cache(multi, k, f) } end end @inited.set(true) - @logger.info("RedisFeatureStore: initialized with #{fs.count} feature flags") + @logger.info("#{storeName}: initialized with #{fs.count} items") end - def upsert(key, feature) + def upsert(key, item) with_connection do |redis| - redis.watch(@features_key) do + redis.watch(@items_key) do old = get_redis(redis, key) - if old.nil? || (old[:version] < feature[:version]) - put_redis_and_cache(redis, key, feature) + if old.nil? || (old[:version] < item[:version]) + put_redis_and_cache(redis, key, item) end redis.unwatch end @@ -204,10 +204,10 @@ def with_connection def get_redis(redis, key) begin - json_feature = redis.hget(@features_key, key) - JSON.parse(json_feature, symbolize_names: true) if json_feature + json_item = redis.hget(@items_key, key) + JSON.parse(json_item, symbolize_names: true) if json_item rescue => e - @logger.error("RedisFeatureStore: could not retrieve feature #{key} from Redis, error: #{e}") + @logger.error("#{storeName}: could not retrieve #{key} from Redis, error: #{e}") nil end end @@ -216,17 +216,55 @@ def put_cache(key, value) @cache.store(key, value, expires: @expiration_seconds) end - def put_redis_and_cache(redis, key, feature) + def put_redis_and_cache(redis, key, item) begin - redis.hset(@features_key, key, feature.to_json) + redis.hset(@items_key, key, item.to_json) rescue => e - @logger.error("RedisFeatureStore: could not store #{key} in Redis, error: #{e}") + @logger.error("#{storeName}: could not store #{key} in Redis, error: #{e}") end - put_cache(key.to_sym, feature) + put_cache(key.to_sym, item) end def query_inited - with_connection { |redis| redis.exists(@features_key) } + with_connection { |redis| redis.exists(@items_key) } + end + end + + # + # An implementation of the LaunchDarkly client's feature store that uses a Redis + # instance. Feature data can also be further cached in memory to reduce overhead + # of calls to Redis. + # + # To use this class, you must first have the `redis`, `connection-pool`, and `moneta` + # gems installed. Then, create an instance and store it in the `feature_store` + # property of your client configuration. + # + class RedisFeatureStore < RedisVersionedStore + def baseKeySuffix + ':features' + end + + def storeName + 'RedisFeatureStore' + end + end + + # + # An implementation of the LaunchDarkly client's segment store that uses a Redis + # instance. Segment data can also be further cached in memory to reduce overhead + # of calls to Redis. + # + # To use this class, you must first have the `redis`, `connection-pool`, and `moneta` + # gems installed. Then, create an instance and store it in the `segment_store` + # property of your client configuration. + # + class RedisSegmentStore < RedisVersionedStore + def baseKeySuffix + ':segments' + end + + def storeName + 'RedisSegmentStore' end end end diff --git a/lib/ldclient-rb/requestor.rb b/lib/ldclient-rb/requestor.rb index ecd23a54..af29d6ab 100644 --- a/lib/ldclient-rb/requestor.rb +++ b/lib/ldclient-rb/requestor.rb @@ -26,6 +26,18 @@ def request_flag(key) make_request("/sdk/latest-flags/" + key) end + def request_all_segments() + make_request("/sdk/latest-segments") + end + + def request_segment(key) + make_request("/sdk/latest-segments/" + key) + end + + def request_all_data() + make_requesT("/sdk/latest-all") + end + def make_request(path) uri = @config.base_uri + path res = @client.get (uri) do |req| diff --git a/lib/ldclient-rb/stream.rb b/lib/ldclient-rb/stream.rb index 0d5766bf..9ebe2cd4 100644 --- a/lib/ldclient-rb/stream.rb +++ b/lib/ldclient-rb/stream.rb @@ -14,7 +14,8 @@ class StreamProcessor def initialize(sdk_key, config, requestor) @sdk_key = sdk_key @config = config - @store = config.feature_store + @feature_store = config.feature_store + @segment_store = config.segment_store @requestor = requestor @initialized = Concurrent::AtomicBoolean.new(false) @started = Concurrent::AtomicBoolean.new(false) @@ -66,30 +67,69 @@ def stop end end + private + def process_message(message, method) @config.logger.debug("[LDClient] Stream received #{method} message: #{message.data}") if method == PUT message = JSON.parse(message.data, symbolize_names: true) - @store.init(message) + @feature_store.init(message[:flags]) + @segment_store.init(message[:segments]) @initialized.make_true @config.logger.info("[LDClient] Stream initialized") elsif method == PATCH message = JSON.parse(message.data, symbolize_names: true) - @store.upsert(message[:path][1..-1], message[:data]) + key = feature_key_for_path(message[:path]) + if key + @feature_store.upsert(key, message[:data]) + else + key = segment_key_for_path(message[:path]) + if key + @segment_store.upsert(key, message[:data]) + end + end elsif method == DELETE message = JSON.parse(message.data, symbolize_names: true) - @store.delete(message[:path][1..-1], message[:version]) + key = feature_key_for_path(message[:path]) + if key + @feature_store.delete(key, message[:version]) + else + key = segment_key_for_path(message[:path]) + if key + @segment_store.delete(key, message[:version]) + end + end elsif method == INDIRECT_PUT - @store.init(@requestor.request_all_flags) + allData = @requestor.request_all_data + @feature_store.init(allData[:flags]) + @segment_store.init(allData[:segments]) @initialized.make_true @config.logger.info("[LDClient] Stream initialized (via indirect message)") elsif method == INDIRECT_PATCH - @store.upsert(message.data, @requestor.request_flag(message.data)) + key = feature_key_for_path(message[:path]) + if key + @feature_store.upsert(key, @requestor.request_flag(key)) + else + key = segment_key_for_path(message[:path]) + if key + @segment_store.upsert(key, @requestor.request_segment(key)) + end + end else @config.logger.warn("[LDClient] Unknown message received: #{method}") end end - private :process_message + def feature_key_for_path(path) + key_for_path(path, '/flags/') + end + + def segment_key_for_path(path) + key_for_path(path, '/segments/') + end + + def key_for_path(path, prefix) + path.start_with?(prefix) ? path[prefix.length..-1] : nil + end end end diff --git a/spec/stream_spec.rb b/spec/stream_spec.rb index ce6c4185..f18915f2 100644 --- a/spec/stream_spec.rb +++ b/spec/stream_spec.rb @@ -37,21 +37,33 @@ let(:processor) { subject.new("sdk_key", config, requestor) } describe '#process_message' do - let(:put_message) { OpenStruct.new({data: '{"key": {"value": "asdf"}}'}) } - let(:patch_message) { OpenStruct.new({data: '{"path": "akey", "data": {"value": "asdf", "version": 1}}'}) } - let(:delete_message) { OpenStruct.new({data: '{"path": "akey", "version": 2}'}) } + let(:put_message) { OpenStruct.new({data: '{"flags":{"key": {"value": "asdf"}},"segments":{"segkey": {"value": "asdf"}}}'}) } + let(:patch_flag_message) { OpenStruct.new({data: '{"path": "/flags/key", "data": {"value": "asdf", "version": 1}}'}) } + let(:patch_seg_message) { OpenStruct.new({data: '{"path": "/segments/key", "data": {"value": "asdf", "version": 1}}'}) } + let(:delete_flag_message) { OpenStruct.new({data: '{"path": "/flags/key", "version": 2}'}) } + let(:delete_seg_message) { OpenStruct.new({data: '{"path": "/segments/key", "version": 2}'}) } it "will accept PUT methods" do processor.send(:process_message, put_message, LaunchDarkly::PUT) - expect(processor.instance_variable_get(:@store).get("key")).to eq(value: "asdf") + expect(config.feature_store.get("key")).to eq(value: "asdf") + expect(config.segment_store.get("segkey")).to eq(value: "asdf") end - it "will accept PATCH methods" do - processor.send(:process_message, patch_message, LaunchDarkly::PATCH) - expect(processor.instance_variable_get(:@store).get("key")).to eq(value: "asdf", version: 1) + it "will accept PATCH methods for flags" do + processor.send(:process_message, patch_flag_message, LaunchDarkly::PATCH) + expect(config.feature_store.get("key")).to eq(value: "asdf", version: 1) end - it "will accept DELETE methods" do - processor.send(:process_message, patch_message, LaunchDarkly::PATCH) - processor.send(:process_message, delete_message, LaunchDarkly::DELETE) - expect(processor.instance_variable_get(:@store).get("key")).to eq(nil) + it "will accept PATCH methods for segments" do + processor.send(:process_message, patch_seg_message, LaunchDarkly::PATCH) + expect(config.segment_store.get("key")).to eq(value: "asdf", version: 1) + end + it "will accept DELETE methods for flags" do + processor.send(:process_message, patch_flag_message, LaunchDarkly::PATCH) + processor.send(:process_message, delete_flag_message, LaunchDarkly::DELETE) + expect(config.feature_store.get("key")).to eq(nil) + end + it "will accept DELETE methods for segments" do + processor.send(:process_message, patch_seg_message, LaunchDarkly::PATCH) + processor.send(:process_message, delete_seg_message, LaunchDarkly::DELETE) + expect(config.segment_store.get("key")).to eq(nil) end it "will log a warning if the method is not recognized" do expect(processor.instance_variable_get(:@config).logger).to receive :warn From 5d6c8a8eabe42dade09184abbdd9821eef4f131b Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Wed, 17 Jan 2018 11:41:11 -0800 Subject: [PATCH 02/28] segment store tests --- spec/in_memory_segment_store_spec.rb | 12 ++++ spec/redis_segment_store_spec.rb | 43 +++++++++++++ spec/segment_store_spec_base.rb | 95 ++++++++++++++++++++++++++++ 3 files changed, 150 insertions(+) create mode 100644 spec/in_memory_segment_store_spec.rb create mode 100644 spec/redis_segment_store_spec.rb create mode 100644 spec/segment_store_spec_base.rb diff --git a/spec/in_memory_segment_store_spec.rb b/spec/in_memory_segment_store_spec.rb new file mode 100644 index 00000000..c4725e45 --- /dev/null +++ b/spec/in_memory_segment_store_spec.rb @@ -0,0 +1,12 @@ +require "segment_store_spec_base" +require "spec_helper" + +def create_in_memory_store() + LaunchDarkly::InMemorySegmentStore.new +end + +describe LaunchDarkly::InMemorySegmentStore do + subject { LaunchDarkly::InMemorySegmentStore } + + include_examples "segment_store", method(:create_in_memory_store) +end diff --git a/spec/redis_segment_store_spec.rb b/spec/redis_segment_store_spec.rb new file mode 100644 index 00000000..0fbaaa9b --- /dev/null +++ b/spec/redis_segment_store_spec.rb @@ -0,0 +1,43 @@ +require "segment_store_spec_base" +require "json" +require "spec_helper" + + + +$my_prefix = 'testprefix' +$null_log = ::Logger.new($stdout) +$null_log.level = ::Logger::FATAL + + +def create_redis_store() + LaunchDarkly::RedisSegmentStore.new(prefix: $my_prefix, logger: $null_log, expiration: 60) +end + +def create_redis_store_uncached() + LaunchDarkly::RedisSegmentStore.new(prefix: $my_prefix, logger: $null_log, expiration: 0) +end + + +describe LaunchDarkly::RedisSegmentStore do + subject { LaunchDarkly::RedisSegmentStore } + + let(:segment0_with_higher_version) do + f = segment0.clone + f[:version] = segment0[:version] + 10 + f + end + + # These tests will all fail if there isn't a Redis instance running on the default port. + + context "real Redis with local cache" do + + include_examples "segment_store", method(:create_redis_store) + + end + + context "real Redis without local cache" do + + include_examples "segment_store", method(:create_redis_store_uncached) + + end +end diff --git a/spec/segment_store_spec_base.rb b/spec/segment_store_spec_base.rb new file mode 100644 index 00000000..02ecd448 --- /dev/null +++ b/spec/segment_store_spec_base.rb @@ -0,0 +1,95 @@ +require "spec_helper" + +RSpec.shared_examples "segment_store" do |create_store_method| + + let(:segment0) { + { + key: "test-segment", + version: 11, + salt: "718ea30a918a4eba8734b57ab1a93227", + rules: [] + } + } + let(:key0) { segment0[:key].to_sym } + + let!(:store) do + s = create_store_method.call() + s.init({ key0 => segment0 }) + s + end + + def new_version_plus(f, deltaVersion, attrs = {}) + f1 = f.clone + f1[:version] = f[:version] + deltaVersion + f1.update(attrs) + f1 + end + + + it "is initialized" do + expect(store.initialized?).to eq true + end + + it "can get existing feature with symbol key" do + expect(store.get(key0)).to eq segment0 + end + + it "can get existing feature with string key" do + expect(store.get(key0.to_s)).to eq segment0 + end + + it "gets nil for nonexisting feature" do + expect(store.get('nope')).to be_nil + end + + it "can get all features" do + feature1 = segment0.clone + feature1[:key] = "test-feature-flag1" + feature1[:version] = 5 + feature1[:on] = false + store.upsert(:"test-feature-flag1", feature1) + expect(store.all).to eq ({ key0 => segment0, :"test-feature-flag1" => feature1 }) + end + + it "can add new feature" do + feature1 = segment0.clone + feature1[:key] = "test-feature-flag1" + feature1[:version] = 5 + feature1[:on] = false + store.upsert(:"test-feature-flag1", feature1) + expect(store.get(:"test-feature-flag1")).to eq feature1 + end + + it "can update feature with newer version" do + f1 = new_version_plus(segment0, 1, { on: !segment0[:on] }) + store.upsert(key0, f1) + expect(store.get(key0)).to eq f1 + end + + it "cannot update feature with same version" do + f1 = new_version_plus(segment0, 0, { on: !segment0[:on] }) + store.upsert(key0, f1) + expect(store.get(key0)).to eq segment0 + end + + it "cannot update feature with older version" do + f1 = new_version_plus(segment0, -1, { on: !segment0[:on] }) + store.upsert(key0, f1) + expect(store.get(key0)).to eq segment0 + end + + it "can delete feature with newer version" do + store.delete(key0, segment0[:version] + 1) + expect(store.get(key0)).to be_nil + end + + it "cannot delete feature with same version" do + store.delete(key0, segment0[:version]) + expect(store.get(key0)).to eq segment0 + end + + it "cannot delete feature with older version" do + store.delete(key0, segment0[:version] - 1) + expect(store.get(key0)).to eq segment0 + end +end From 4e4f6a3ead6bb75a331e518cabcef1d7703631f4 Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Wed, 17 Jan 2018 11:43:42 -0800 Subject: [PATCH 03/28] doc comment --- lib/ldclient-rb/config.rb | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/lib/ldclient-rb/config.rb b/lib/ldclient-rb/config.rb index c449363f..c1a6d5be 100644 --- a/lib/ldclient-rb/config.rb +++ b/lib/ldclient-rb/config.rb @@ -34,6 +34,10 @@ class Config # @option opts [Object] :cache_store A cache store for the Faraday HTTP caching # library. Defaults to the Rails cache in a Rails environment, or a # thread-safe in-memory store otherwise. + # @option opts [Object] :feature_store A store for feature flag data. Defaults to an in-memory + # cache, or you can use RedisFeatureStore. + # @option opts [Object] :segment_store A store for segment data. Defaults to an in-memory + # cache, or you can use RedisSegmentStore. # @option opts [Boolean] :use_ldd (false) Whether you are using the LaunchDarkly relay proxy in # daemon mode. In this configuration, the client will not use a streaming connection to listen # for updates, but instead will get feature state from a Redis instance. The `stream` and From 064844075e751017a1564890a9d30d9c4a015496 Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Wed, 17 Jan 2018 15:27:09 -0800 Subject: [PATCH 04/28] misc cleanup, tests --- lib/ldclient-rb/evaluation.rb | 7 +- spec/evaluation_spec.rb | 149 ++++++++++++++++++++++++++++++++++ 2 files changed, 153 insertions(+), 3 deletions(-) create mode 100644 spec/evaluation_spec.rb diff --git a/lib/ldclient-rb/evaluation.rb b/lib/ldclient-rb/evaluation.rb index 4ea184c3..c5e3499d 100644 --- a/lib/ldclient-rb/evaluation.rb +++ b/lib/ldclient-rb/evaluation.rb @@ -196,6 +196,7 @@ def clause_match_user(clause, user, segment_store) return maybe_negate(clause, true) if segment_match_user(segment, user) end end + return maybe_negate(clause, false) end clause_match_user_no_segments(clause, user) end @@ -243,7 +244,7 @@ def segment_match_user(segment, user) return true if segment[:included].include?(user[:key]) return false if segment[:excluded].include?(user[:key]) - + segment[:rules].each do |r| return true if segment_rule_match_user(r, user, segment[:key], segment[:salt]) end @@ -257,10 +258,10 @@ def segment_rule_match_user(rule, user, segment_key, salt) end # If the weight is absent, this rule matches - return true if !rule.weight + return true if !rule[:weight] # All of the clauses are met. See if the user buckets in - bucket = bucket_user(user, segment_key, rule[:bucketBy].nil ? "key" : rule[:bucketBy], salt) + bucket = bucket_user(user, segment_key, rule[:bucketBy].nil? ? "key" : rule[:bucketBy], salt) weight = rule[:weight].to_f / 100000.0 return bucket < weight end diff --git a/spec/evaluation_spec.rb b/spec/evaluation_spec.rb new file mode 100644 index 00000000..f197d3eb --- /dev/null +++ b/spec/evaluation_spec.rb @@ -0,0 +1,149 @@ +require "spec_helper" + + +describe LaunchDarkly::Evaluation do + subject { LaunchDarkly::Evaluation } + let(:features) { LaunchDarkly::InMemoryFeatureStore.new } + let(:segments) { LaunchDarkly::InMemorySegmentStore.new } + let(:user) { + { + key: "userkey", + email: "test@example.com", + name: "Bob" + } + } + + def make_flag(key) + { + key: key, + rules: [], + variations: [ false, true ], + on: true, + fallthrough: { variation: 0 }, + version: 1 + } + end + + def make_segment(key) + { + key: key, + included: [], + excluded: [], + salt: 'abcdef', + version: 1 + } + end + + def make_segment_match_clause(segment) + { + op: :segmentMatch, + values: [ segment[:key] ], + negate: false + } + end + + def make_user_matching_clause(user, attr) + { + attribute: attr.to_s, + op: :in, + values: [ user[attr.to_sym] ], + negate: false + } + end + + include LaunchDarkly::Evaluation + + describe 'segment matching' do + it 'explicitly includes user' do + segment = make_segment('segkey') + segment[:included] = [ user[:key] ] + segments.upsert('segkey', segment) + clause = make_segment_match_clause(segment) + + result = clause_match_user(clause, user, segments) + expect(result).to be true + end + + it 'explicitly excludes user' do + segment = make_segment('segkey') + segment[:excluded] = [ user[:key] ] + segments.upsert('segkey', segment) + clause = make_segment_match_clause(segment) + + result = clause_match_user(clause, user, segments) + expect(result).to be false + end + + it 'both includes and excludes user; include takes priority' do + segment = make_segment('segkey') + segment[:included] = [ user[:key] ] + segment[:excluded] = [ user[:key] ] + segments.upsert('segkey', segment) + clause = make_segment_match_clause(segment) + + result = clause_match_user(clause, user, segments) + expect(result).to be true + end + + it 'matches user with full rollout' do + segClause = make_user_matching_clause(user, :email) + segRule = { + clauses: [ segClause ], + weight: 100000 + } + segment = make_segment('segkey') + segment[:rules] = [ segRule ] + segments.upsert('segkey', segment) + clause = make_segment_match_clause(segment) + + result = clause_match_user(clause, user, segments) + expect(result).to be true + end + + it 'doesn''t match user with zero rollout' do + segClause = make_user_matching_clause(user, :email) + segRule = { + clauses: [ segClause ], + weight: 0 + } + segment = make_segment('segkey') + segment[:rules] = [ segRule ] + segments.upsert('segkey', segment) + clause = make_segment_match_clause(segment) + + result = clause_match_user(clause, user, segments) + expect(result).to be false + end + + it 'matches user with multiple clauses' do + segClause1 = make_user_matching_clause(user, :email) + segClause2 = make_user_matching_clause(user, :name) + segRule = { + clauses: [ segClause1, segClause2 ] + } + segment = make_segment('segkey') + segment[:rules] = [ segRule ] + segments.upsert('segkey', segment) + clause = make_segment_match_clause(segment) + + result = clause_match_user(clause, user, segments) + expect(result).to be true + end + + it 'doesn''t match user with multiple clauses' do + segClause1 = make_user_matching_clause(user, :email) + segClause2 = make_user_matching_clause(user, :name) + segClause2[:values] = [ 'wrong' ] + segRule = { + clauses: [ segClause1, segClause2 ] + } + segment = make_segment('segkey') + segment[:rules] = [ segRule ] + segments.upsert('segkey', segment) + clause = make_segment_match_clause(segment) + + result = clause_match_user(clause, user, segments) + expect(result).to be false + end + end +end From 27855dcbe27a699e354d8e03e42f0ea8653f2009 Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Fri, 19 Jan 2018 17:33:35 -0800 Subject: [PATCH 05/28] fix omitted parse --- lib/ldclient-rb/stream.rb | 1 + 1 file changed, 1 insertion(+) diff --git a/lib/ldclient-rb/stream.rb b/lib/ldclient-rb/stream.rb index 9ebe2cd4..de578c64 100644 --- a/lib/ldclient-rb/stream.rb +++ b/lib/ldclient-rb/stream.rb @@ -106,6 +106,7 @@ def process_message(message, method) @initialized.make_true @config.logger.info("[LDClient] Stream initialized (via indirect message)") elsif method == INDIRECT_PATCH + message = JSON.parse(message.data, symbolize_names: true) key = feature_key_for_path(message[:path]) if key @feature_store.upsert(key, @requestor.request_flag(key)) From 049f68e907d1eb1862828777a0ceb03c7761ed3d Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Fri, 19 Jan 2018 17:38:09 -0800 Subject: [PATCH 06/28] fix indirect patch logic - message data is the path --- lib/ldclient-rb/stream.rb | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/lib/ldclient-rb/stream.rb b/lib/ldclient-rb/stream.rb index de578c64..a98451a5 100644 --- a/lib/ldclient-rb/stream.rb +++ b/lib/ldclient-rb/stream.rb @@ -106,12 +106,11 @@ def process_message(message, method) @initialized.make_true @config.logger.info("[LDClient] Stream initialized (via indirect message)") elsif method == INDIRECT_PATCH - message = JSON.parse(message.data, symbolize_names: true) - key = feature_key_for_path(message[:path]) + key = feature_key_for_path(message.data) if key @feature_store.upsert(key, @requestor.request_flag(key)) else - key = segment_key_for_path(message[:path]) + key = segment_key_for_path(message.data) if key @segment_store.upsert(key, @requestor.request_segment(key)) end From 04fe620453c94ffebaa6d1e6fff6e47bba378c0b Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Thu, 25 Jan 2018 10:45:11 -0800 Subject: [PATCH 07/28] update to use unified generic feature store --- lib/ldclient-rb.rb | 2 +- lib/ldclient-rb/config.rb | 14 +-- lib/ldclient-rb/evaluation.rb | 22 ++-- lib/ldclient-rb/in_memory_store.rb | 60 ++++++----- lib/ldclient-rb/ldclient.rb | 9 +- lib/ldclient-rb/polling.rb | 6 +- lib/ldclient-rb/redis_store.rb | 149 +++++++++++---------------- lib/ldclient-rb/stream.rb | 54 +++++----- spec/evaluation_spec.rb | 37 ++++--- spec/feature_store_spec_base.rb | 40 +++---- spec/in_memory_segment_store_spec.rb | 12 --- spec/redis_segment_store_spec.rb | 43 -------- spec/stream_spec.rb | 33 +++--- 13 files changed, 198 insertions(+), 283 deletions(-) delete mode 100644 spec/in_memory_segment_store_spec.rb delete mode 100644 spec/redis_segment_store_spec.rb diff --git a/lib/ldclient-rb.rb b/lib/ldclient-rb.rb index 3f24012d..ce943d13 100644 --- a/lib/ldclient-rb.rb +++ b/lib/ldclient-rb.rb @@ -3,12 +3,12 @@ require "ldclient-rb/ldclient" require "ldclient-rb/cache_store" require "ldclient-rb/memoized_value" +require "ldclient-rb/in_memory_store" require "ldclient-rb/config" require "ldclient-rb/newrelic" require "ldclient-rb/stream" require "ldclient-rb/polling" require "ldclient-rb/event_serializer" require "ldclient-rb/events" -require "ldclient-rb/in_memory_store" require "ldclient-rb/redis_store" require "ldclient-rb/requestor" diff --git a/lib/ldclient-rb/config.rb b/lib/ldclient-rb/config.rb index c1a6d5be..5a6e7c26 100644 --- a/lib/ldclient-rb/config.rb +++ b/lib/ldclient-rb/config.rb @@ -34,10 +34,8 @@ class Config # @option opts [Object] :cache_store A cache store for the Faraday HTTP caching # library. Defaults to the Rails cache in a Rails environment, or a # thread-safe in-memory store otherwise. - # @option opts [Object] :feature_store A store for feature flag data. Defaults to an in-memory + # @option opts [Object] :feature_store A store for feature flags and related data. Defaults to an in-memory # cache, or you can use RedisFeatureStore. - # @option opts [Object] :segment_store A store for segment data. Defaults to an in-memory - # cache, or you can use RedisSegmentStore. # @option opts [Boolean] :use_ldd (false) Whether you are using the LaunchDarkly relay proxy in # daemon mode. In this configuration, the client will not use a streaming connection to listen # for updates, but instead will get feature state from a Redis instance. The `stream` and @@ -70,7 +68,6 @@ def initialize(opts = {}) @connect_timeout = opts[:connect_timeout] || Config.default_connect_timeout @read_timeout = opts[:read_timeout] || Config.default_read_timeout @feature_store = opts[:feature_store] || Config.default_feature_store - @segment_store = opts[:segment_store] || Config.default_segment_store @stream = opts.has_key?(:stream) ? opts[:stream] : Config.default_stream @use_ldd = opts.has_key?(:use_ldd) ? opts[:use_ldd] : Config.default_use_ldd @offline = opts.has_key?(:offline) ? opts[:offline] : Config.default_offline @@ -176,11 +173,6 @@ def offline? # attr_reader :feature_store - # - # A store for segment configuration rules. - # - attr_reader :segment_store - # The proxy configuration string # attr_reader :proxy @@ -261,10 +253,6 @@ def self.default_feature_store InMemoryFeatureStore.new end - def self.default_segment_store - InMemorySegmentStore.new - end - def self.default_offline false end diff --git a/lib/ldclient-rb/evaluation.rb b/lib/ldclient-rb/evaluation.rb index ef80b944..13fdaac1 100644 --- a/lib/ldclient-rb/evaluation.rb +++ b/lib/ldclient-rb/evaluation.rb @@ -112,7 +112,7 @@ class EvaluationError < StandardError # generated during prerequisite evaluation. Raises EvaluationError if the flag is not well-formed # Will return nil, but not raise an exception, indicating that the rules (including fallthrough) did not match # In that case, the caller should return the default value. - def evaluate(flag, user, feature_store, segment_store) + def evaluate(flag, user, feature_store) if flag.nil? raise EvaluationError, "Flag does not exist" end @@ -124,7 +124,7 @@ def evaluate(flag, user, feature_store, segment_store) events = [] if flag[:on] - res = eval_internal(flag, user, feature_store, segment_store, events) + res = eval_internal(flag, user, feature_store, events) return { value: res, events: events } if !res.nil? end @@ -137,7 +137,7 @@ def evaluate(flag, user, feature_store, segment_store) { value: nil, events: events } end - def eval_internal(flag, user, feature_store, segment_store, events) + def eval_internal(flag, user, feature_store, events) failed_prereq = false # Evaluate prerequisites, if any if !flag[:prerequisites].nil? @@ -148,7 +148,7 @@ def eval_internal(flag, user, feature_store, segment_store, events) failed_prereq = true else begin - prereq_res = eval_internal(prereq_flag, user, feature_store, segment_store, events) + prereq_res = eval_internal(prereq_flag, user, feature_store, events) variation = get_variation(prereq_flag, prerequisite[:variation]) events.push(kind: "feature", key: prereq_flag[:key], value: prereq_res, version: prereq_flag[:version], prereqOf: flag[:key]) if prereq_res.nil? || prereq_res != variation @@ -168,10 +168,10 @@ def eval_internal(flag, user, feature_store, segment_store, events) # The prerequisites were satisfied. # Now walk through the evaluation steps and get the correct # variation index - eval_rules(flag, user, segment_store) + eval_rules(flag, user, feature_store) end - def eval_rules(flag, user, segment_store) + def eval_rules(flag, user, feature_store) # Check user target matches if !flag[:targets].nil? flag[:targets].each do |target| @@ -186,7 +186,7 @@ def eval_rules(flag, user, segment_store) # Check custom rules if !flag[:rules].nil? flag[:rules].each do |rule| - return variation_for_user(rule, user, flag) if rule_match_user(rule, user, segment_store) + return variation_for_user(rule, user, flag) if rule_match_user(rule, user, feature_store) end end @@ -206,22 +206,22 @@ def get_variation(flag, index) flag[:variations][index] end - def rule_match_user(rule, user, segment_store) + def rule_match_user(rule, user, feature_store) return false if !rule[:clauses] rule[:clauses].each do |clause| - return false if !clause_match_user(clause, user, segment_store) + return false if !clause_match_user(clause, user, feature_store) end return true end - def clause_match_user(clause, user, segment_store) + def clause_match_user(clause, user, feature_store) # In the case of a segment match operator, we check if the user is in any of the segments, # and possibly negate if (clause[:op] == :segmentMatch) clause[:values].each do |v| - segment = segment_store.get(v) + segment = feature_store.get(SEGMENTS, v) if !segment.nil? return maybe_negate(clause, true) if segment_match_user(segment, user) end diff --git a/lib/ldclient-rb/in_memory_store.rb b/lib/ldclient-rb/in_memory_store.rb index 95851bdb..42d8fda0 100644 --- a/lib/ldclient-rb/in_memory_store.rb +++ b/lib/ldclient-rb/in_memory_store.rb @@ -1,53 +1,69 @@ require "concurrent/atomics" module LaunchDarkly - class InMemoryVersionedStore + FEATURES = { + namespace: "features" + } + + SEGMENTS = { + namespace: "segments" + } + + class InMemoryFeatureStore def initialize @items = Hash.new @lock = Concurrent::ReadWriteLock.new @initialized = Concurrent::AtomicBoolean.new(false) end - def get(key) + def get(kind, key) @lock.with_read_lock do - f = @items[key.to_sym] + coll = @items[kind] + f = coll.nil? ? nil : coll[key.to_sym] (f.nil? || f[:deleted]) ? nil : f end end - def all + def all(kind) @lock.with_read_lock do - @items.select { |_k, f| not f[:deleted] } + coll = @items[kind] + (coll.nil? ? Hash.new : coll).select { |_k, f| not f[:deleted] } end end - def delete(key, version) + def delete(kind, key, version) @lock.with_write_lock do - old = @items[key.to_sym] + coll = @items[kind] + if coll.nil? + coll = Hash.new + @items[kind] = coll + end + old = coll[key.to_sym] - if !old.nil? && old[:version] < version - old[:deleted] = true - old[:version] = version - @items[key.to_sym] = old - elsif old.nil? - @items[key.to_sym] = { deleted: true, version: version } + if old.nil? || old[:version] < version + coll[key.to_sym] = { deleted: true, version: version } end end end - def init(fs) + def init(allData) @lock.with_write_lock do - @items.replace(fs) + @items.replace(allData) @initialized.make_true end end - def upsert(key, feature) + def upsert(kind, item) @lock.with_write_lock do - old = @items[key.to_sym] + coll = @items[kind] + if coll.nil? + coll = Hash.new + @items[kind] = coll + end + old = coll[item[:key].to_sym] - if old.nil? || old[:version] < feature[:version] - @items[key.to_sym] = feature + if old.nil? || old[:version] < item[:version] + coll[item[:key].to_sym] = item end end end @@ -60,10 +76,4 @@ def stop # nothing to do end end - - class InMemoryFeatureStore < InMemoryVersionedStore - end - - class InMemorySegmentStore < InMemoryVersionedStore - end end diff --git a/lib/ldclient-rb/ldclient.rb b/lib/ldclient-rb/ldclient.rb index ab506e46..973b043c 100644 --- a/lib/ldclient-rb/ldclient.rb +++ b/lib/ldclient-rb/ldclient.rb @@ -27,7 +27,6 @@ def initialize(sdk_key, config = Config.default, wait_for_sec = 5) @sdk_key = sdk_key @config = config @store = config.feature_store - @segment_store = config.segment_store @event_processor = EventProcessor.new(sdk_key, config) @@ -131,7 +130,7 @@ def variation(key, user, default) end sanitize_user(user) - feature = @store.get(key) + feature = @store.get(FEATURES, key) if feature.nil? @config.logger.info("[LDClient] Unknown feature flag #{key}. Returning default value") @@ -140,7 +139,7 @@ def variation(key, user, default) end begin - res = evaluate(feature, user, @store, @segment_store) + res = evaluate(feature, user, @store) if !res[:events].nil? res[:events].each do |event| @event_processor.add_event(event) @@ -198,10 +197,10 @@ def all_flags(user) end begin - features = @store.all + features = @store.all(FEATURES) # TODO rescue if necessary - Hash[features.map{ |k, f| [k, evaluate(f, user, @store, @segment_store)[:value]] }] + Hash[features.map{ |k, f| [k, evaluate(f, user, @store)[:value]] }] rescue => exn @config.logger.warn("[LDClient] Error evaluating all flags: #{exn.inspect}. \nTrace: #{exn.backtrace}") return Hash.new diff --git a/lib/ldclient-rb/polling.rb b/lib/ldclient-rb/polling.rb index 8a2eecce..4fd8b70e 100644 --- a/lib/ldclient-rb/polling.rb +++ b/lib/ldclient-rb/polling.rb @@ -33,8 +33,10 @@ def stop def poll allData = @requestor.request_all_data if allData - @config.feature_store.init(allData.flags) - @config.segment_store.init(allData.segments) + @config.feature_store.init({ + FEATURES => message[:flags], + SEGMENTS => message[:segments] + }) if @initialized.make_true @config.logger.info("[LDClient] Polling connection initialized") end diff --git a/lib/ldclient-rb/redis_store.rb b/lib/ldclient-rb/redis_store.rb index d33d488f..f6ff34d3 100644 --- a/lib/ldclient-rb/redis_store.rb +++ b/lib/ldclient-rb/redis_store.rb @@ -4,9 +4,15 @@ module LaunchDarkly # - # Base class for all Redis versioned object stores. + # An implementation of the LaunchDarkly client's feature store that uses a Redis + # instance. Feature data can also be further cached in memory to reduce overhead + # of calls to Redis. # - class RedisVersionedStore + # To use this class, you must first have the `redis`, `connection-pool`, and `moneta` + # gems installed. Then, create an instance and store it in the `feature_store` + # property of your client configuration. + # + class RedisFeatureStore begin require "redis" require "connection_pool" @@ -31,22 +37,21 @@ class RedisVersionedStore # def initialize(opts = {}) if !REDIS_ENABLED - raise RuntimeError.new("can't use #{storeName} because one of these gems is missing: redis, connection_pool, moneta") + raise RuntimeError.new("can't use RedisFeatureStore because one of these gems is missing: redis, connection_pool, moneta") end @redis_opts = opts[:redis_opts] || Hash.new if opts[:redis_url] @redis_opts[:url] = opts[:redis_url] end if !@redis_opts.include?(:url) - @redis_opts[:url] = RedisVersionedStore.default_redis_url + @redis_opts[:url] = RedisFeatureStore.default_redis_url end max_connections = opts[:max_connections] || 16 @pool = opts[:pool] || ConnectionPool.new(size: max_connections) do Redis.new(@redis_opts) end - @prefix = opts[:prefix] || RedisVersionedStore.default_prefix + @prefix = opts[:prefix] || RedisFeatureStore.default_prefix @logger = opts[:logger] || Config.default_logger - @items_key = @prefix + baseKeySuffix @expiration_seconds = opts[:expiration] || 15 @capacity = opts[:capacity] || 1000 @@ -65,17 +70,11 @@ def initialize(opts = {}) } with_connection do |redis| - @logger.info("#{storeName}: using Redis instance at #{redis.connection[:host]}:#{redis.connection[:port]} \ + @logger.info("RedisFeatureStore: using Redis instance at #{redis.connection[:host]}:#{redis.connection[:port]} \ and prefix: #{@prefix}") end end - def baseKeySuffix - end - - def storeName - end - # # Default value for the `redis_url` constructor parameter; points to an instance of Redis # running at `localhost` with its default port. @@ -91,40 +90,40 @@ def self.default_prefix 'launchdarkly' end - def get(key) - f = @cache[key.to_sym] + def get(kind, key) + f = @cache[cache_key(kind, key)] if f.nil? - @logger.debug("#{storeName}: no cache hit for #{key}, requesting from Redis") + @logger.debug("RedisFeatureStore: no cache hit for #{key} in \"#{kind[:namespace]}\", requesting from Redis") f = with_connection do |redis| begin - get_redis(redis,key.to_sym) + get_redis(kind, redis, key.to_sym) rescue => e - @logger.error("#{storeName}: could not retrieve #{key} from Redis, with error: #{e}") + @logger.error("RedisFeatureStore: could not retrieve #{key} from Redis in \"#{kind[:namespace]}\", with error: #{e}") nil end end if !f.nil? - put_cache(key.to_sym, f) + put_cache(kind, key, f) end end if f.nil? - @logger.debug("#{storeName}: #{key} not found") + @logger.debug("RedisFeatureStore: #{key} not found in \"#{kind[:namespace]}\"") nil elsif f[:deleted] - @logger.debug("#{storeName}: #{key} was deleted, returning nil") + @logger.debug("RedisFeatureStore: #{key} was deleted in \"#{kind[:namespace]}\", returning nil") nil else f end end - def all + def all(kind) fs = {} with_connection do |redis| begin - hashfs = redis.hgetall(@items_key) + hashfs = redis.hgetall(items_key(kind)) rescue => e - @logger.error("#{storeName}: could not retrieve all items from Redis with error: #{e}; returning none") + @logger.error("RedisFeatureStore: could not retrieve all \"#{kind[:namespace]}\" items from Redis with error: #{e}; returning none") hashfs = {} end hashfs.each do |k, jsonItem| @@ -137,43 +136,47 @@ def all fs end - def delete(key, version) + def delete(kind, key, version) with_connection do |redis| - f = get_redis(redis, key) + f = get_redis(kind, redis, key) if f.nil? - put_redis_and_cache(redis, key, { deleted: true, version: version }) + put_redis_and_cache(kind, redis, key, { deleted: true, version: version }) else if f[:version] < version f1 = f.clone f1[:deleted] = true f1[:version] = version - put_redis_and_cache(redis, key, f1) + put_redis_and_cache(kind, redis, key, f1) else - @logger.warn("#{storeName}: attempted to delete #{key} version: #{f[:version]} \ - with a version that is the same or older: #{version}") + @logger.warn("RedisFeatureStore: attempted to delete #{key} version: #{f[:version]} \ + in \"#{kind[:namespace]}\" with a version that is the same or older: #{version}") end end end end - def init(fs) + def init(allData) @cache.clear + count = 0 with_connection do |redis| - redis.multi do |multi| - multi.del(@items_key) - fs.each { |k, f| put_redis_and_cache(multi, k, f) } - end + allData.each { |kind, items| + redis.multi do |multi| + multi.del(items_key(kind)) + count = count + items.count + items.each { |k, v| put_redis_and_cache(kind, multi, k, v) } + end + } end @inited.set(true) - @logger.info("#{storeName}: initialized with #{fs.count} items") + @logger.info("RedisFeatureStore: initialized with #{count} items") end - def upsert(key, item) + def upsert(kind, item) with_connection do |redis| - redis.watch(@items_key) do - old = get_redis(redis, key) + redis.watch(items_key(kind)) do + old = get_redis(kind, redis, item[:key]) if old.nil? || (old[:version] < item[:version]) - put_redis_and_cache(redis, key, item) + put_redis_and_cache(kind, redis, item[:key], item) end redis.unwatch end @@ -198,73 +201,43 @@ def clear_local_cache() private + def items_key(kind) + @prefix + ":" + kind[:namespace] + end + + def cache_key(kind, key) + kind[:namespace] + ":" + key.to_s + end + def with_connection @pool.with { |redis| yield(redis) } end - def get_redis(redis, key) + def get_redis(kind, redis, key) begin - json_item = redis.hget(@items_key, key) + json_item = redis.hget(items_key(kind), key) JSON.parse(json_item, symbolize_names: true) if json_item rescue => e - @logger.error("#{storeName}: could not retrieve #{key} from Redis, error: #{e}") + @logger.error("RedisFeatureStore: could not retrieve #{key} from Redis, error: #{e}") nil end end - def put_cache(key, value) - @cache.store(key, value, expires: @expiration_seconds) + def put_cache(kind, key, value) + @cache.store(cache_key(kind, key), value, expires: @expiration_seconds) end - def put_redis_and_cache(redis, key, item) + def put_redis_and_cache(kind, redis, key, item) begin - redis.hset(@items_key, key, item.to_json) + redis.hset(items_key(kind), key, item.to_json) rescue => e - @logger.error("#{storeName}: could not store #{key} in Redis, error: #{e}") + @logger.error("RedisFeatureStore: could not store #{key} in Redis, error: #{e}") end - put_cache(key.to_sym, item) + put_cache(kind, key.to_sym, item) end def query_inited - with_connection { |redis| redis.exists(@items_key) } - end - end - - # - # An implementation of the LaunchDarkly client's feature store that uses a Redis - # instance. Feature data can also be further cached in memory to reduce overhead - # of calls to Redis. - # - # To use this class, you must first have the `redis`, `connection-pool`, and `moneta` - # gems installed. Then, create an instance and store it in the `feature_store` - # property of your client configuration. - # - class RedisFeatureStore < RedisVersionedStore - def baseKeySuffix - ':features' - end - - def storeName - 'RedisFeatureStore' - end - end - - # - # An implementation of the LaunchDarkly client's segment store that uses a Redis - # instance. Segment data can also be further cached in memory to reduce overhead - # of calls to Redis. - # - # To use this class, you must first have the `redis`, `connection-pool`, and `moneta` - # gems installed. Then, create an instance and store it in the `segment_store` - # property of your client configuration. - # - class RedisSegmentStore < RedisVersionedStore - def baseKeySuffix - ':segments' - end - - def storeName - 'RedisSegmentStore' + with_connection { |redis| redis.exists(items_key(FEATURES)) } end end end diff --git a/lib/ldclient-rb/stream.rb b/lib/ldclient-rb/stream.rb index a98451a5..532e5534 100644 --- a/lib/ldclient-rb/stream.rb +++ b/lib/ldclient-rb/stream.rb @@ -10,12 +10,16 @@ module LaunchDarkly INDIRECT_PATCH = :'indirect/patch' READ_TIMEOUT_SECONDS = 300 # 5 minutes; the stream should send a ping every 3 minutes + KEY_PATHS = { + FEATURES => "/flags/", + SEGMENTS => "/segments/" + } + class StreamProcessor def initialize(sdk_key, config, requestor) @sdk_key = sdk_key @config = config @feature_store = config.feature_store - @segment_store = config.segment_store @requestor = requestor @initialized = Concurrent::AtomicBoolean.new(false) @started = Concurrent::AtomicBoolean.new(false) @@ -73,46 +77,46 @@ def process_message(message, method) @config.logger.debug("[LDClient] Stream received #{method} message: #{message.data}") if method == PUT message = JSON.parse(message.data, symbolize_names: true) - @feature_store.init(message[:flags]) - @segment_store.init(message[:segments]) + @feature_store.init({ + FEATURES => message[:flags], + SEGMENTS => message[:segments] + }) @initialized.make_true @config.logger.info("[LDClient] Stream initialized") elsif method == PATCH message = JSON.parse(message.data, symbolize_names: true) - key = feature_key_for_path(message[:path]) - if key - @feature_store.upsert(key, message[:data]) - else - key = segment_key_for_path(message[:path]) + for kind in [FEATURES, SEGMENTS] + key = key_for_path(kind, message[:path]) if key - @segment_store.upsert(key, message[:data]) + @feature_store.upsert(kind, message[:data]) + break end end elsif method == DELETE message = JSON.parse(message.data, symbolize_names: true) - key = feature_key_for_path(message[:path]) - if key - @feature_store.delete(key, message[:version]) - else - key = segment_key_for_path(message[:path]) + for kind in [FEATURES, SEGMENTS] + key = key_for_path(kind, message[:path]) if key - @segment_store.delete(key, message[:version]) + @feature_store.delete(kind, key, message[:version]) + break end end elsif method == INDIRECT_PUT allData = @requestor.request_all_data - @feature_store.init(allData[:flags]) - @segment_store.init(allData[:segments]) + ({ + FEATURES => allData[:flags], + SEGMENTS => allData[:segments] + }) @initialized.make_true @config.logger.info("[LDClient] Stream initialized (via indirect message)") elsif method == INDIRECT_PATCH key = feature_key_for_path(message.data) if key - @feature_store.upsert(key, @requestor.request_flag(key)) + @feature_store.upsert(FEATURES, @requestor.request_flag(key)) else key = segment_key_for_path(message.data) if key - @segment_store.upsert(key, @requestor.request_segment(key)) + @feature_store.upsert(SEGMENTS, key, @requestor.request_segment(key)) end end else @@ -120,16 +124,8 @@ def process_message(message, method) end end - def feature_key_for_path(path) - key_for_path(path, '/flags/') - end - - def segment_key_for_path(path) - key_for_path(path, '/segments/') - end - - def key_for_path(path, prefix) - path.start_with?(prefix) ? path[prefix.length..-1] : nil + def key_for_path(kind, path) + path.start_with?(KEY_PATHS[kind]) ? path[KEY_PATHS[kind].length..-1] : nil end end end diff --git a/spec/evaluation_spec.rb b/spec/evaluation_spec.rb index 6fab62d6..989e96cb 100644 --- a/spec/evaluation_spec.rb +++ b/spec/evaluation_spec.rb @@ -3,7 +3,6 @@ describe LaunchDarkly::Evaluation do subject { LaunchDarkly::Evaluation } let(:features) { LaunchDarkly::InMemoryFeatureStore.new } - let(:segments) { LaunchDarkly::InMemorySegmentStore.new } let(:user) { { key: "userkey", @@ -18,19 +17,19 @@ it "can match built-in attribute" do user = { key: 'x', name: 'Bob' } clause = { attribute: 'name', op: 'in', values: ['Bob'] } - expect(clause_match_user(clause, user, segments)).to be true + expect(clause_match_user(clause, user, features)).to be true end it "can match custom attribute" do user = { key: 'x', name: 'Bob', custom: { legs: 4 } } clause = { attribute: 'legs', op: 'in', values: [4] } - expect(clause_match_user(clause, user, segments)).to be true + expect(clause_match_user(clause, user, features)).to be true end it "returns false for missing attribute" do user = { key: 'x', name: 'Bob' } clause = { attribute: 'legs', op: 'in', values: [4] } - expect(clause_match_user(clause, user, segments)).to be false + expect(clause_match_user(clause, user, features)).to be false end end @@ -132,7 +131,7 @@ it "should return #{shouldBe} for #{value1} #{op} #{value2}" do user = { key: 'x', custom: { foo: value1 } } clause = { attribute: 'foo', op: op, values: [value2] } - expect(clause_match_user(clause, user, segments)).to be shouldBe + expect(clause_match_user(clause, user, features)).to be shouldBe end end end @@ -181,20 +180,20 @@ def make_user_matching_clause(user, attr) it 'explicitly includes user' do segment = make_segment('segkey') segment[:included] = [ user[:key] ] - segments.upsert('segkey', segment) + features.upsert(LaunchDarkly::SEGMENTS, segment) clause = make_segment_match_clause(segment) - result = clause_match_user(clause, user, segments) + result = clause_match_user(clause, user, features) expect(result).to be true end it 'explicitly excludes user' do segment = make_segment('segkey') segment[:excluded] = [ user[:key] ] - segments.upsert('segkey', segment) + features.upsert(LaunchDarkly::SEGMENTS, segment) clause = make_segment_match_clause(segment) - result = clause_match_user(clause, user, segments) + result = clause_match_user(clause, user, features) expect(result).to be false end @@ -202,10 +201,10 @@ def make_user_matching_clause(user, attr) segment = make_segment('segkey') segment[:included] = [ user[:key] ] segment[:excluded] = [ user[:key] ] - segments.upsert('segkey', segment) + features.upsert(LaunchDarkly::SEGMENTS, segment) clause = make_segment_match_clause(segment) - result = clause_match_user(clause, user, segments) + result = clause_match_user(clause, user, features) expect(result).to be true end @@ -217,10 +216,10 @@ def make_user_matching_clause(user, attr) } segment = make_segment('segkey') segment[:rules] = [ segRule ] - segments.upsert('segkey', segment) + features.upsert(LaunchDarkly::SEGMENTS, segment) clause = make_segment_match_clause(segment) - result = clause_match_user(clause, user, segments) + result = clause_match_user(clause, user, features) expect(result).to be true end @@ -232,10 +231,10 @@ def make_user_matching_clause(user, attr) } segment = make_segment('segkey') segment[:rules] = [ segRule ] - segments.upsert('segkey', segment) + features.upsert(LaunchDarkly::SEGMENTS, segment) clause = make_segment_match_clause(segment) - result = clause_match_user(clause, user, segments) + result = clause_match_user(clause, user, features) expect(result).to be false end @@ -247,10 +246,10 @@ def make_user_matching_clause(user, attr) } segment = make_segment('segkey') segment[:rules] = [ segRule ] - segments.upsert('segkey', segment) + features.upsert(LaunchDarkly::SEGMENTS, segment) clause = make_segment_match_clause(segment) - result = clause_match_user(clause, user, segments) + result = clause_match_user(clause, user, features) expect(result).to be true end @@ -263,10 +262,10 @@ def make_user_matching_clause(user, attr) } segment = make_segment('segkey') segment[:rules] = [ segRule ] - segments.upsert('segkey', segment) + features.upsert(LaunchDarkly::SEGMENTS, segment) clause = make_segment_match_clause(segment) - result = clause_match_user(clause, user, segments) + result = clause_match_user(clause, user, features) expect(result).to be false end end diff --git a/spec/feature_store_spec_base.rb b/spec/feature_store_spec_base.rb index d589acab..d6c1cedc 100644 --- a/spec/feature_store_spec_base.rb +++ b/spec/feature_store_spec_base.rb @@ -31,7 +31,7 @@ let!(:store) do s = create_store_method.call() - s.init({ key0 => feature0 }) + s.init(LaunchDarkly::FEATURES => { key0 => feature0 }) s end @@ -48,15 +48,15 @@ def new_version_plus(f, deltaVersion, attrs = {}) end it "can get existing feature with symbol key" do - expect(store.get(key0)).to eq feature0 + expect(store.get(LaunchDarkly::FEATURES, key0)).to eq feature0 end it "can get existing feature with string key" do - expect(store.get(key0.to_s)).to eq feature0 + expect(store.get(LaunchDarkly::FEATURES, key0.to_s)).to eq feature0 end it "gets nil for nonexisting feature" do - expect(store.get('nope')).to be_nil + expect(store.get(LaunchDarkly::FEATURES, 'nope')).to be_nil end it "can get all features" do @@ -64,8 +64,8 @@ def new_version_plus(f, deltaVersion, attrs = {}) feature1[:key] = "test-feature-flag1" feature1[:version] = 5 feature1[:on] = false - store.upsert(:"test-feature-flag1", feature1) - expect(store.all).to eq ({ key0 => feature0, :"test-feature-flag1" => feature1 }) + store.upsert(LaunchDarkly::FEATURES, feature1) + expect(store.all(LaunchDarkly::FEATURES)).to eq ({ key0 => feature0, :"test-feature-flag1" => feature1 }) end it "can add new feature" do @@ -73,40 +73,40 @@ def new_version_plus(f, deltaVersion, attrs = {}) feature1[:key] = "test-feature-flag1" feature1[:version] = 5 feature1[:on] = false - store.upsert(:"test-feature-flag1", feature1) - expect(store.get(:"test-feature-flag1")).to eq feature1 + store.upsert(LaunchDarkly::FEATURES, feature1) + expect(store.get(LaunchDarkly::FEATURES, :"test-feature-flag1")).to eq feature1 end it "can update feature with newer version" do f1 = new_version_plus(feature0, 1, { on: !feature0[:on] }) - store.upsert(key0, f1) - expect(store.get(key0)).to eq f1 + store.upsert(LaunchDarkly::FEATURES, f1) + expect(store.get(LaunchDarkly::FEATURES, key0)).to eq f1 end it "cannot update feature with same version" do f1 = new_version_plus(feature0, 0, { on: !feature0[:on] }) - store.upsert(key0, f1) - expect(store.get(key0)).to eq feature0 + store.upsert(LaunchDarkly::FEATURES, f1) + expect(store.get(LaunchDarkly::FEATURES, key0)).to eq feature0 end it "cannot update feature with older version" do f1 = new_version_plus(feature0, -1, { on: !feature0[:on] }) - store.upsert(key0, f1) - expect(store.get(key0)).to eq feature0 + store.upsert(LaunchDarkly::FEATURES, f1) + expect(store.get(LaunchDarkly::FEATURES, key0)).to eq feature0 end it "can delete feature with newer version" do - store.delete(key0, feature0[:version] + 1) - expect(store.get(key0)).to be_nil + store.delete(LaunchDarkly::FEATURES, key0, feature0[:version] + 1) + expect(store.get(LaunchDarkly::FEATURES, key0)).to be_nil end it "cannot delete feature with same version" do - store.delete(key0, feature0[:version]) - expect(store.get(key0)).to eq feature0 + store.delete(LaunchDarkly::FEATURES, key0, feature0[:version]) + expect(store.get(LaunchDarkly::FEATURES, key0)).to eq feature0 end it "cannot delete feature with older version" do - store.delete(key0, feature0[:version] - 1) - expect(store.get(key0)).to eq feature0 + store.delete(LaunchDarkly::FEATURES, key0, feature0[:version] - 1) + expect(store.get(LaunchDarkly::FEATURES, key0)).to eq feature0 end end diff --git a/spec/in_memory_segment_store_spec.rb b/spec/in_memory_segment_store_spec.rb deleted file mode 100644 index c4725e45..00000000 --- a/spec/in_memory_segment_store_spec.rb +++ /dev/null @@ -1,12 +0,0 @@ -require "segment_store_spec_base" -require "spec_helper" - -def create_in_memory_store() - LaunchDarkly::InMemorySegmentStore.new -end - -describe LaunchDarkly::InMemorySegmentStore do - subject { LaunchDarkly::InMemorySegmentStore } - - include_examples "segment_store", method(:create_in_memory_store) -end diff --git a/spec/redis_segment_store_spec.rb b/spec/redis_segment_store_spec.rb deleted file mode 100644 index 0fbaaa9b..00000000 --- a/spec/redis_segment_store_spec.rb +++ /dev/null @@ -1,43 +0,0 @@ -require "segment_store_spec_base" -require "json" -require "spec_helper" - - - -$my_prefix = 'testprefix' -$null_log = ::Logger.new($stdout) -$null_log.level = ::Logger::FATAL - - -def create_redis_store() - LaunchDarkly::RedisSegmentStore.new(prefix: $my_prefix, logger: $null_log, expiration: 60) -end - -def create_redis_store_uncached() - LaunchDarkly::RedisSegmentStore.new(prefix: $my_prefix, logger: $null_log, expiration: 0) -end - - -describe LaunchDarkly::RedisSegmentStore do - subject { LaunchDarkly::RedisSegmentStore } - - let(:segment0_with_higher_version) do - f = segment0.clone - f[:version] = segment0[:version] + 10 - f - end - - # These tests will all fail if there isn't a Redis instance running on the default port. - - context "real Redis with local cache" do - - include_examples "segment_store", method(:create_redis_store) - - end - - context "real Redis without local cache" do - - include_examples "segment_store", method(:create_redis_store_uncached) - - end -end diff --git a/spec/stream_spec.rb b/spec/stream_spec.rb index f18915f2..4b569888 100644 --- a/spec/stream_spec.rb +++ b/spec/stream_spec.rb @@ -3,20 +3,23 @@ describe LaunchDarkly::InMemoryFeatureStore do subject { LaunchDarkly::InMemoryFeatureStore } + + include LaunchDarkly + let(:store) { subject.new } let(:key) { :asdf } - let(:feature) { { value: "qwer", version: 0 } } + let(:feature) { { key: "asdf", value: "qwer", version: 0 } } describe '#all' do it "will get all keys" do - store.upsert(key, feature) - data = store.all + store.upsert(LaunchDarkly::FEATURES, feature) + data = store.all(LaunchDarkly::FEATURES) expect(data).to eq(key => feature) end it "will not get deleted keys" do - store.upsert(key, feature) - store.delete(key, 1) - data = store.all + store.upsert(LaunchDarkly::FEATURES, feature) + store.delete(LaunchDarkly::FEATURES, key, 1) + data = store.all(LaunchDarkly::FEATURES) expect(data).to eq({}) end end @@ -37,33 +40,33 @@ let(:processor) { subject.new("sdk_key", config, requestor) } describe '#process_message' do - let(:put_message) { OpenStruct.new({data: '{"flags":{"key": {"value": "asdf"}},"segments":{"segkey": {"value": "asdf"}}}'}) } - let(:patch_flag_message) { OpenStruct.new({data: '{"path": "/flags/key", "data": {"value": "asdf", "version": 1}}'}) } - let(:patch_seg_message) { OpenStruct.new({data: '{"path": "/segments/key", "data": {"value": "asdf", "version": 1}}'}) } + let(:put_message) { OpenStruct.new({data: '{"flags":{"asdf": {"key": "asdf"}},"segments":{"segkey": {"key": "segkey"}}}'}) } + let(:patch_flag_message) { OpenStruct.new({data: '{"path": "/flags/key", "data": {"key": "asdf", "version": 1}}'}) } + let(:patch_seg_message) { OpenStruct.new({data: '{"path": "/segments/key", "data": {"key": "asdf", "version": 1}}'}) } let(:delete_flag_message) { OpenStruct.new({data: '{"path": "/flags/key", "version": 2}'}) } let(:delete_seg_message) { OpenStruct.new({data: '{"path": "/segments/key", "version": 2}'}) } it "will accept PUT methods" do processor.send(:process_message, put_message, LaunchDarkly::PUT) - expect(config.feature_store.get("key")).to eq(value: "asdf") - expect(config.segment_store.get("segkey")).to eq(value: "asdf") + expect(config.feature_store.get(LaunchDarkly::FEATURES, "asdf")).to eq(key: "asdf") + expect(config.feature_store.get(LaunchDarkly::SEGMENTS, "segkey")).to eq(key: "segkey") end it "will accept PATCH methods for flags" do processor.send(:process_message, patch_flag_message, LaunchDarkly::PATCH) - expect(config.feature_store.get("key")).to eq(value: "asdf", version: 1) + expect(config.feature_store.get(LaunchDarkly::FEATURES, "asdf")).to eq(key: "asdf", version: 1) end it "will accept PATCH methods for segments" do processor.send(:process_message, patch_seg_message, LaunchDarkly::PATCH) - expect(config.segment_store.get("key")).to eq(value: "asdf", version: 1) + expect(config.feature_store.get(LaunchDarkly::SEGMENTS, "asdf")).to eq(key: "asdf", version: 1) end it "will accept DELETE methods for flags" do processor.send(:process_message, patch_flag_message, LaunchDarkly::PATCH) processor.send(:process_message, delete_flag_message, LaunchDarkly::DELETE) - expect(config.feature_store.get("key")).to eq(nil) + expect(config.feature_store.get(LaunchDarkly::FEATURES, "key")).to eq(nil) end it "will accept DELETE methods for segments" do processor.send(:process_message, patch_seg_message, LaunchDarkly::PATCH) processor.send(:process_message, delete_seg_message, LaunchDarkly::DELETE) - expect(config.segment_store.get("key")).to eq(nil) + expect(config.feature_store.get(LaunchDarkly::SEGMENTS, "key")).to eq(nil) end it "will log a warning if the method is not recognized" do expect(processor.instance_variable_get(:@config).logger).to receive :warn From 696e42cb09921b409efcf330c500eaf0a919bdfe Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Thu, 25 Jan 2018 10:49:13 -0800 Subject: [PATCH 08/28] comments --- lib/ldclient-rb/in_memory_store.rb | 10 ++++++++++ lib/ldclient-rb/redis_store.rb | 5 +++-- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/lib/ldclient-rb/in_memory_store.rb b/lib/ldclient-rb/in_memory_store.rb index 42d8fda0..f545e427 100644 --- a/lib/ldclient-rb/in_memory_store.rb +++ b/lib/ldclient-rb/in_memory_store.rb @@ -1,6 +1,11 @@ require "concurrent/atomics" module LaunchDarkly + + # These constants denote the types of data that can be stored in the feature store. If + # we add another storable data type in the future, as long as it follows the same pattern + # (having "key", "version", and "deleted" properties), we only need to add a corresponding + # constant here and the existing store should be able to handle it. FEATURES = { namespace: "features" } @@ -9,6 +14,11 @@ module LaunchDarkly namespace: "segments" } + # + # Default implementation of the LaunchDarkly client's feature store, using an in-memory + # cache. This object holds feature flags and related data received from the + # streaming API. + # class InMemoryFeatureStore def initialize @items = Hash.new diff --git a/lib/ldclient-rb/redis_store.rb b/lib/ldclient-rb/redis_store.rb index f6ff34d3..64b10f83 100644 --- a/lib/ldclient-rb/redis_store.rb +++ b/lib/ldclient-rb/redis_store.rb @@ -5,7 +5,8 @@ module LaunchDarkly # # An implementation of the LaunchDarkly client's feature store that uses a Redis - # instance. Feature data can also be further cached in memory to reduce overhead + # instance. This object holds feature flags and related data received from the + # streaming API. Feature data can also be further cached in memory to reduce overhead # of calls to Redis. # # To use this class, you must first have the `redis`, `connection-pool`, and `moneta` @@ -32,7 +33,7 @@ class RedisFeatureStore # @option opts [Logger] :logger a `Logger` instance; defaults to `Config.default_logger` # @option opts [Integer] :max_connections size of the Redis connection pool # @option opts [Integer] :expiration expiration time for the in-memory cache, in seconds; 0 for no local caching - # @option opts [Integer] :capacity maximum number of feature flags to cache locally + # @option opts [Integer] :capacity maximum number of feature flags (or related objects) to cache locally # @option opts [Object] :pool custom connection pool, used for testing only # def initialize(opts = {}) From c79a762ad33598e554da53561e154bc65aba6589 Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Thu, 25 Jan 2018 11:04:28 -0800 Subject: [PATCH 09/28] misc fixes --- lib/ldclient-rb/evaluation.rb | 2 +- lib/ldclient-rb/stream.rb | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/lib/ldclient-rb/evaluation.rb b/lib/ldclient-rb/evaluation.rb index 13fdaac1..275b8e95 100644 --- a/lib/ldclient-rb/evaluation.rb +++ b/lib/ldclient-rb/evaluation.rb @@ -219,7 +219,7 @@ def rule_match_user(rule, user, feature_store) def clause_match_user(clause, user, feature_store) # In the case of a segment match operator, we check if the user is in any of the segments, # and possibly negate - if (clause[:op] == :segmentMatch) + if (clause[:op].to_sym == :segmentMatch) clause[:values].each do |v| segment = feature_store.get(SEGMENTS, v) if !segment.nil? diff --git a/lib/ldclient-rb/stream.rb b/lib/ldclient-rb/stream.rb index 532e5534..7459aedc 100644 --- a/lib/ldclient-rb/stream.rb +++ b/lib/ldclient-rb/stream.rb @@ -41,7 +41,7 @@ def start 'User-Agent' => 'RubyClient/' + LaunchDarkly::VERSION } opts = {:headers => headers, :with_credentials => true, :proxy => @config.proxy, :read_timeout => READ_TIMEOUT_SECONDS} - @es = Celluloid::EventSource.new(@config.stream_uri + "/flags", opts) do |conn| + @es = Celluloid::EventSource.new(@config.stream_uri + "/all", opts) do |conn| conn.on(PUT) { |message| process_message(message, PUT) } conn.on(PATCH) { |message| process_message(message, PATCH) } conn.on(DELETE) { |message| process_message(message, DELETE) } @@ -78,8 +78,8 @@ def process_message(message, method) if method == PUT message = JSON.parse(message.data, symbolize_names: true) @feature_store.init({ - FEATURES => message[:flags], - SEGMENTS => message[:segments] + FEATURES => message[:data][:flags], + SEGMENTS => message[:data][:segments] }) @initialized.make_true @config.logger.info("[LDClient] Stream initialized") From da6c0d67ccfbdf5101447fedc0d3ebd592798434 Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Thu, 25 Jan 2018 11:04:36 -0800 Subject: [PATCH 10/28] bump version --- lib/ldclient-rb/version.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/ldclient-rb/version.rb b/lib/ldclient-rb/version.rb index e74409d6..efdc72b6 100644 --- a/lib/ldclient-rb/version.rb +++ b/lib/ldclient-rb/version.rb @@ -1,3 +1,3 @@ module LaunchDarkly - VERSION = "2.4.1" + VERSION = "3.0.0" end From b442243a63e9a2e728e1f49cb19eef553b13cd83 Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Thu, 25 Jan 2018 11:07:33 -0800 Subject: [PATCH 11/28] changelog --- CHANGELOG.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 778b0284..99ca998f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,13 @@ All notable changes to the LaunchDarkly Ruby SDK will be documented in this file. This project adheres to [Semantic Versioning](http://semver.org). +## [3.0.0] +## Added +- Add support for user segments + +## Changed +- Changed interface of feature store classes to support segments + ## [2.4.1] - 2018-01-23 ## Changed - Reduce logging level for missing flags From 54c45e81e6bbbe362cf983e5a8c148669a0f810f Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Thu, 25 Jan 2018 11:28:17 -0800 Subject: [PATCH 12/28] fix test --- spec/stream_spec.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spec/stream_spec.rb b/spec/stream_spec.rb index 4b569888..e4495b52 100644 --- a/spec/stream_spec.rb +++ b/spec/stream_spec.rb @@ -40,7 +40,7 @@ let(:processor) { subject.new("sdk_key", config, requestor) } describe '#process_message' do - let(:put_message) { OpenStruct.new({data: '{"flags":{"asdf": {"key": "asdf"}},"segments":{"segkey": {"key": "segkey"}}}'}) } + let(:put_message) { OpenStruct.new({data: '{"data":{"flags":{"asdf": {"key": "asdf"}},"segments":{"segkey": {"key": "segkey"}}}}'}) } let(:patch_flag_message) { OpenStruct.new({data: '{"path": "/flags/key", "data": {"key": "asdf", "version": 1}}'}) } let(:patch_seg_message) { OpenStruct.new({data: '{"path": "/segments/key", "data": {"key": "asdf", "version": 1}}'}) } let(:delete_flag_message) { OpenStruct.new({data: '{"path": "/flags/key", "version": 2}'}) } From 23512f488c88269cd6c7577deb94831afc60f5cd Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Thu, 25 Jan 2018 17:05:50 -0800 Subject: [PATCH 13/28] revert changelog & version --- CHANGELOG.md | 7 ------- lib/ldclient-rb/version.rb | 2 +- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 99ca998f..778b0284 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,13 +2,6 @@ All notable changes to the LaunchDarkly Ruby SDK will be documented in this file. This project adheres to [Semantic Versioning](http://semver.org). -## [3.0.0] -## Added -- Add support for user segments - -## Changed -- Changed interface of feature store classes to support segments - ## [2.4.1] - 2018-01-23 ## Changed - Reduce logging level for missing flags diff --git a/lib/ldclient-rb/version.rb b/lib/ldclient-rb/version.rb index efdc72b6..e74409d6 100644 --- a/lib/ldclient-rb/version.rb +++ b/lib/ldclient-rb/version.rb @@ -1,3 +1,3 @@ module LaunchDarkly - VERSION = "3.0.0" + VERSION = "2.4.1" end From a5067b07ed3c03e0d4d0412fc284526bc9bd1e80 Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Wed, 31 Jan 2018 15:09:20 -0800 Subject: [PATCH 14/28] typo --- lib/ldclient-rb/requestor.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/ldclient-rb/requestor.rb b/lib/ldclient-rb/requestor.rb index af29d6ab..40928806 100644 --- a/lib/ldclient-rb/requestor.rb +++ b/lib/ldclient-rb/requestor.rb @@ -35,7 +35,7 @@ def request_segment(key) end def request_all_data() - make_requesT("/sdk/latest-all") + make_request("/sdk/latest-all") end def make_request(path) From 7107b16f53d6e780b9787dbace91ff83d2d6603a Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Wed, 31 Jan 2018 15:13:12 -0800 Subject: [PATCH 15/28] typo --- lib/ldclient-rb/polling.rb | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/ldclient-rb/polling.rb b/lib/ldclient-rb/polling.rb index 4fd8b70e..d6cd2f28 100644 --- a/lib/ldclient-rb/polling.rb +++ b/lib/ldclient-rb/polling.rb @@ -34,8 +34,8 @@ def poll allData = @requestor.request_all_data if allData @config.feature_store.init({ - FEATURES => message[:flags], - SEGMENTS => message[:segments] + FEATURES => allData[:flags], + SEGMENTS => allData[:segments] }) if @initialized.make_true @config.logger.info("[LDClient] Polling connection initialized") From 9f12e68125bc664d97476727cdd8c30696767e28 Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Wed, 31 Jan 2018 15:48:39 -0800 Subject: [PATCH 16/28] typo --- lib/ldclient-rb/evaluation.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/ldclient-rb/evaluation.rb b/lib/ldclient-rb/evaluation.rb index 275b8e95..648b0f70 100644 --- a/lib/ldclient-rb/evaluation.rb +++ b/lib/ldclient-rb/evaluation.rb @@ -142,7 +142,7 @@ def eval_internal(flag, user, feature_store, events) # Evaluate prerequisites, if any if !flag[:prerequisites].nil? flag[:prerequisites].each do |prerequisite| - prereq_flag = feature_store.get(prerequisite[:key]) + prereq_flag = feature_store.get(FEATURES, prerequisite[:key]) if prereq_flag.nil? || !prereq_flag[:on] failed_prereq = true From c6094ce4a9d2dde1e4f0de8dd0c3c93284ea4b4b Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Mon, 5 Feb 2018 16:57:58 -0800 Subject: [PATCH 17/28] test improvements --- spec/evaluation_spec.rb | 30 +++++++++++++++++++++++++----- 1 file changed, 25 insertions(+), 5 deletions(-) diff --git a/spec/evaluation_spec.rb b/spec/evaluation_spec.rb index 989e96cb..3388e252 100644 --- a/spec/evaluation_spec.rb +++ b/spec/evaluation_spec.rb @@ -31,6 +31,28 @@ clause = { attribute: 'legs', op: 'in', values: [4] } expect(clause_match_user(clause, user, features)).to be false end + + it "retrieves segment from segment store for segmentMatch operator" do + segment = { + key: 'segkey', + included: [ 'userkey' ], + version: 1, + deleted: false + } + features.upsert(LaunchDarkly::SEGMENTS, segment) + + user = { key: 'userkey' } + clause = { attribute: '', op: 'segmentMatch', values: ['segkey'] } + + expect(clause_match_user(clause, user, features)).to be true + end + + it "falls through with no errors if referenced segment is not found" do + user = { key: 'userkey' } + clause = { attribute: '', op: 'segmentMatch', values: ['segkey'] } + + expect(clause_match_user(clause, user, features)).to be false + end end describe "operators" do @@ -174,8 +196,6 @@ def make_user_matching_clause(user, attr) } end - include LaunchDarkly::Evaluation - describe 'segment matching' do it 'explicitly includes user' do segment = make_segment('segkey') @@ -223,7 +243,7 @@ def make_user_matching_clause(user, attr) expect(result).to be true end - it 'doesn''t match user with zero rollout' do + it "doesn't match user with zero rollout" do segClause = make_user_matching_clause(user, :email) segRule = { clauses: [ segClause ], @@ -238,7 +258,7 @@ def make_user_matching_clause(user, attr) expect(result).to be false end - it 'matches user with multiple clauses' do + it "matches user with multiple clauses" do segClause1 = make_user_matching_clause(user, :email) segClause2 = make_user_matching_clause(user, :name) segRule = { @@ -253,7 +273,7 @@ def make_user_matching_clause(user, attr) expect(result).to be true end - it 'doesn''t match user with multiple clauses' do + it "doesn't match user with multiple clauses if a clause doesn't match" do segClause1 = make_user_matching_clause(user, :email) segClause2 = make_user_matching_clause(user, :name) segClause2[:values] = [ 'wrong' ] From 70038f53b6d3de62f6b99c5f7e7ab8d3b172be4c Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Tue, 6 Feb 2018 12:44:14 -0800 Subject: [PATCH 18/28] misc cleanup --- lib/ldclient-rb/evaluation.rb | 30 ++++++++++++++---------------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/lib/ldclient-rb/evaluation.rb b/lib/ldclient-rb/evaluation.rb index 855be6ab..3ec5600c 100644 --- a/lib/ldclient-rb/evaluation.rb +++ b/lib/ldclient-rb/evaluation.rb @@ -112,7 +112,7 @@ class EvaluationError < StandardError # generated during prerequisite evaluation. Raises EvaluationError if the flag is not well-formed # Will return nil, but not raise an exception, indicating that the rules (including fallthrough) did not match # In that case, the caller should return the default value. - def evaluate(flag, user, feature_store) + def evaluate(flag, user, store) if flag.nil? raise EvaluationError, "Flag does not exist" end @@ -124,7 +124,7 @@ def evaluate(flag, user, feature_store) events = [] if flag[:on] - res = eval_internal(flag, user, feature_store, events) + res = eval_internal(flag, user, store, events) return { value: res, events: events } if !res.nil? end @@ -137,18 +137,18 @@ def evaluate(flag, user, feature_store) { value: nil, events: events } end - def eval_internal(flag, user, feature_store, events) + def eval_internal(flag, user, store, events) failed_prereq = false # Evaluate prerequisites, if any if !flag[:prerequisites].nil? flag[:prerequisites].each do |prerequisite| - prereq_flag = feature_store.get(FEATURES, prerequisite[:key]) + prereq_flag = store.get(FEATURES, prerequisite[:key]) if prereq_flag.nil? || !prereq_flag[:on] failed_prereq = true else begin - prereq_res = eval_internal(prereq_flag, user, feature_store, events) + prereq_res = eval_internal(prereq_flag, user, store, events) variation = get_variation(prereq_flag, prerequisite[:variation]) events.push(kind: "feature", key: prereq_flag[:key], value: prereq_res, version: prereq_flag[:version], prereqOf: flag[:key]) if prereq_res.nil? || prereq_res != variation @@ -168,10 +168,10 @@ def eval_internal(flag, user, feature_store, events) # The prerequisites were satisfied. # Now walk through the evaluation steps and get the correct # variation index - eval_rules(flag, user, feature_store) + eval_rules(flag, user, store) end - def eval_rules(flag, user, feature_store) + def eval_rules(flag, user, store) # Check user target matches if !flag[:targets].nil? flag[:targets].each do |target| @@ -186,7 +186,7 @@ def eval_rules(flag, user, feature_store) # Check custom rules if !flag[:rules].nil? flag[:rules].each do |rule| - return variation_for_user(rule, user, flag) if rule_match_user(rule, user, feature_store) + return variation_for_user(rule, user, flag) if rule_match_user(rule, user, store) end end @@ -206,25 +206,23 @@ def get_variation(flag, index) flag[:variations][index] end - def rule_match_user(rule, user, feature_store) + def rule_match_user(rule, user, store) return false if !rule[:clauses] rule[:clauses].each do |clause| - return false if !clause_match_user(clause, user, feature_store) + return false if !clause_match_user(clause, user, store) end return true end - def clause_match_user(clause, user, feature_store) + def clause_match_user(clause, user, store) # In the case of a segment match operator, we check if the user is in any of the segments, # and possibly negate - if (clause[:op].to_sym == :segmentMatch) + if clause[:op].to_sym == :segmentMatch clause[:values].each do |v| - segment = feature_store.get(SEGMENTS, v) - if !segment.nil? - return maybe_negate(clause, true) if segment_match_user(segment, user) - end + segment = store.get(SEGMENTS, v) + return maybe_negate(clause, true) if !segment.nil? && segment_match_user(segment, user) end return maybe_negate(clause, false) end From 4270c29c2f9d174d74ffffb89f823f2dba4a05fd Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Tue, 6 Feb 2018 13:15:36 -0800 Subject: [PATCH 19/28] add a bunch of unit tests for flag evaluation --- spec/evaluation_spec.rb | 129 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 129 insertions(+) diff --git a/spec/evaluation_spec.rb b/spec/evaluation_spec.rb index 1d634496..0f8d1439 100644 --- a/spec/evaluation_spec.rb +++ b/spec/evaluation_spec.rb @@ -13,6 +13,129 @@ include LaunchDarkly::Evaluation + describe "evaluate" do + it "returns off variation if flag is off" do + flag = { + key: 'feature', + on: false, + offVariation: 1, + fallthrough: { variation: 0 }, + variations: ['a', 'b', 'c'] + } + user = { key: 'x' } + expect(evaluate(flag, user, features)).to eq({value: 'b', events: []}) + end + + it "returns nil if flag is off and off variation is unspecified" do + flag = { + key: 'feature', + on: false, + fallthrough: { variation: 0 }, + variations: ['a', 'b', 'c'] + } + user = { key: 'x' } + expect(evaluate(flag, user, features)).to eq({value: nil, events: []}) + end + + it "returns off variation if prerequisite is not found" do + flag = { + key: 'feature0', + on: true, + prerequisites: [{key: 'badfeature', variation: 1}], + fallthrough: { variation: 0 }, + offVariation: 1, + variations: ['a', 'b', 'c'] + } + user = { key: 'x' } + expect(evaluate(flag, user, features)).to eq({value: 'b', events: []}) + end + + it "returns off variation and event if prerequisite is not met" do + flag = { + key: 'feature0', + on: true, + prerequisites: [{key: 'feature1', variation: 1}], + fallthrough: { variation: 0 }, + offVariation: 1, + variations: ['a', 'b', 'c'], + version: 1 + } + flag1 = { + key: 'feature1', + on: true, + fallthrough: { variation: 0 }, + variations: ['d', 'e'], + version: 2 + } + features.upsert(LaunchDarkly::FEATURES, flag1) + user = { key: 'x' } + events_should_be = [{kind: 'feature', key: 'feature1', value: 'd', version: 2, prereqOf: 'feature0'}] + expect(evaluate(flag, user, features)).to eq({value: 'b', events: events_should_be}) + end + + it "returns fallthrough variation and event if prerequisite is met and there are no rules" do + flag = { + key: 'feature0', + on: true, + prerequisites: [{key: 'feature1', variation: 1}], + fallthrough: { variation: 0 }, + offVariation: 1, + variations: ['a', 'b', 'c'], + version: 1 + } + flag1 = { + key: 'feature1', + on: true, + fallthrough: { variation: 1 }, + variations: ['d', 'e'], + version: 2 + } + features.upsert(LaunchDarkly::FEATURES, flag1) + user = { key: 'x' } + events_should_be = [{kind: 'feature', key: 'feature1', value: 'e', version: 2, prereqOf: 'feature0'}] + expect(evaluate(flag, user, features)).to eq({value: 'a', events: events_should_be}) + end + + it "matches user from targets" do + flag = { + key: 'feature0', + on: true, + targets: [ + { values: [ 'whoever', 'userkey' ], variation: 2 } + ], + fallthrough: { variation: 0 }, + offVariation: 1, + variations: ['a', 'b', 'c'] + } + user = { key: 'userkey' } + expect(evaluate(flag, user, features)).to eq({value: 'c', events: []}) + end + + it "matches user from rules" do + flag = { + key: 'feature0', + on: true, + rules: [ + { + clauses: [ + { + attribute: 'key', + op: 'in', + values: [ 'userkey' ] + } + ], + variation: 2 + } + ], + fallthrough: { variation: 0 }, + offVariation: 1, + variations: ['a', 'b', 'c'] + } + user = { key: 'userkey' } + expect(evaluate(flag, user, features)).to eq({value: 'c', events: []}) + end + end + describe "clause_match_user" do it "can match built-in attribute" do user = { key: 'x', name: 'Bob' } @@ -32,6 +155,12 @@ expect(clause_match_user(clause, user, features)).to be false end + it "can be negated" do + user = { key: 'x', name: 'Bob' } + clause = { attribute: 'name', op: 'in', values: ['Bob'], negate: true } + expect(clause_match_user(clause, user, features)).to be false + end + it "retrieves segment from segment store for segmentMatch operator" do segment = { key: 'segkey', From 3e4e1b22b2bef92d54e54240046be68707b1a5a4 Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Tue, 6 Feb 2018 13:28:34 -0800 Subject: [PATCH 20/28] misc style cleanup --- lib/ldclient-rb/evaluation.rb | 60 +++++++++++++++-------------------- 1 file changed, 26 insertions(+), 34 deletions(-) diff --git a/lib/ldclient-rb/evaluation.rb b/lib/ldclient-rb/evaluation.rb index 3ec5600c..f9b855fb 100644 --- a/lib/ldclient-rb/evaluation.rb +++ b/lib/ldclient-rb/evaluation.rb @@ -140,30 +140,28 @@ def evaluate(flag, user, store) def eval_internal(flag, user, store, events) failed_prereq = false # Evaluate prerequisites, if any - if !flag[:prerequisites].nil? - flag[:prerequisites].each do |prerequisite| - prereq_flag = store.get(FEATURES, prerequisite[:key]) + (flag[:prerequisites] || []).each do |prerequisite| + prereq_flag = store.get(FEATURES, prerequisite[:key]) - if prereq_flag.nil? || !prereq_flag[:on] - failed_prereq = true - else - begin - prereq_res = eval_internal(prereq_flag, user, store, events) - variation = get_variation(prereq_flag, prerequisite[:variation]) - events.push(kind: "feature", key: prereq_flag[:key], value: prereq_res, version: prereq_flag[:version], prereqOf: flag[:key]) - if prereq_res.nil? || prereq_res != variation - failed_prereq = true - end - rescue => exn - @config.logger.error("[LDClient] Error evaluating prerequisite: #{exn.inspect}") + if prereq_flag.nil? || !prereq_flag[:on] + failed_prereq = true + else + begin + prereq_res = eval_internal(prereq_flag, user, store, events) + variation = get_variation(prereq_flag, prerequisite[:variation]) + events.push(kind: "feature", key: prereq_flag[:key], value: prereq_res, version: prereq_flag[:version], prereqOf: flag[:key]) + if prereq_res.nil? || prereq_res != variation failed_prereq = true end + rescue => exn + @config.logger.error("[LDClient] Error evaluating prerequisite: #{exn.inspect}") + failed_prereq = true end end + end - if failed_prereq - return nil - end + if failed_prereq + return nil end # The prerequisites were satisfied. # Now walk through the evaluation steps and get the correct @@ -173,21 +171,15 @@ def eval_internal(flag, user, store, events) def eval_rules(flag, user, store) # Check user target matches - if !flag[:targets].nil? - flag[:targets].each do |target| - if !target[:values].nil? - target[:values].each do |value| - return get_variation(flag, target[:variation]) if value == user[:key] - end - end + (flag[:targets] || []).each do |target| + (target[:values] || []).each do |value| + return get_variation(flag, target[:variation]) if value == user[:key] end end - + # Check custom rules - if !flag[:rules].nil? - flag[:rules].each do |rule| - return variation_for_user(rule, user, flag) if rule_match_user(rule, user, store) - end + (flag[:rules] || []).each do |rule| + return variation_for_user(rule, user, flag) if rule_match_user(rule, user, store) end # Check the fallthrough rule @@ -209,7 +201,7 @@ def get_variation(flag, index) def rule_match_user(rule, user, store) return false if !rule[:clauses] - rule[:clauses].each do |clause| + (rule[:clauses] || []).each do |clause| return false if !clause_match_user(clause, user, store) end @@ -220,7 +212,7 @@ def clause_match_user(clause, user, store) # In the case of a segment match operator, we check if the user is in any of the segments, # and possibly negate if clause[:op].to_sym == :segmentMatch - clause[:values].each do |v| + (clause[:values] || []).each do |v| segment = store.get(SEGMENTS, v) return maybe_negate(clause, true) if !segment.nil? && segment_match_user(segment, user) end @@ -273,7 +265,7 @@ def segment_match_user(segment, user) return true if segment[:included].include?(user[:key]) return false if segment[:excluded].include?(user[:key]) - segment[:rules].each do |r| + (segment[:rules] || []).each do |r| return true if segment_rule_match_user(r, user, segment[:key], segment[:salt]) end @@ -281,7 +273,7 @@ def segment_match_user(segment, user) end def segment_rule_match_user(rule, user, segment_key, salt) - rule[:clauses].each do |c| + (rule[:clauses] || []).each do |c| return false unless clause_match_user_no_segments(c, user) end From 0a1c0bdd803533bd3441970160d1aac4b5d0a853 Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Tue, 6 Feb 2018 13:49:19 -0800 Subject: [PATCH 21/28] misc cleanup --- lib/ldclient-rb/polling.rb | 8 ++++---- lib/ldclient-rb/redis_store.rb | 4 ++-- lib/ldclient-rb/stream.rb | 8 ++++---- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/lib/ldclient-rb/polling.rb b/lib/ldclient-rb/polling.rb index d6cd2f28..00cf5e8e 100644 --- a/lib/ldclient-rb/polling.rb +++ b/lib/ldclient-rb/polling.rb @@ -31,11 +31,11 @@ def stop end def poll - allData = @requestor.request_all_data - if allData + all_data = @requestor.request_all_data + if all_data @config.feature_store.init({ - FEATURES => allData[:flags], - SEGMENTS => allData[:segments] + FEATURES => all_data[:flags], + SEGMENTS => all_data[:segments] }) if @initialized.make_true @config.logger.info("[LDClient] Polling connection initialized") diff --git a/lib/ldclient-rb/redis_store.rb b/lib/ldclient-rb/redis_store.rb index 64b10f83..4ddff248 100644 --- a/lib/ldclient-rb/redis_store.rb +++ b/lib/ldclient-rb/redis_store.rb @@ -156,11 +156,11 @@ def delete(kind, key, version) end end - def init(allData) + def init(all_data) @cache.clear count = 0 with_connection do |redis| - allData.each { |kind, items| + all_data.each { |kind, items| redis.multi do |multi| multi.del(items_key(kind)) count = count + items.count diff --git a/lib/ldclient-rb/stream.rb b/lib/ldclient-rb/stream.rb index 7459aedc..671a80e5 100644 --- a/lib/ldclient-rb/stream.rb +++ b/lib/ldclient-rb/stream.rb @@ -102,10 +102,10 @@ def process_message(message, method) end end elsif method == INDIRECT_PUT - allData = @requestor.request_all_data - ({ - FEATURES => allData[:flags], - SEGMENTS => allData[:segments] + all_data = @requestor.request_all_data + @feature_store.init({ + FEATURES => all_data[:flags], + SEGMENTS => all_data[:segments] }) @initialized.make_true @config.logger.info("[LDClient] Stream initialized (via indirect message)") From 2a6c63a2ea60825478588987d10565d421f8a3cd Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Tue, 6 Feb 2018 13:51:41 -0800 Subject: [PATCH 22/28] misc cleanup --- lib/ldclient-rb/redis_store.rb | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/ldclient-rb/redis_store.rb b/lib/ldclient-rb/redis_store.rb index 4ddff248..b64f5c41 100644 --- a/lib/ldclient-rb/redis_store.rb +++ b/lib/ldclient-rb/redis_store.rb @@ -160,13 +160,13 @@ def init(all_data) @cache.clear count = 0 with_connection do |redis| - all_data.each { |kind, items| + all_data.each do |kind, items| redis.multi do |multi| multi.del(items_key(kind)) count = count + items.count items.each { |k, v| put_redis_and_cache(kind, multi, k, v) } end - } + end end @inited.set(true) @logger.info("RedisFeatureStore: initialized with #{count} items") From a2e275bb5535626dabc0a612faa00710b7450eba Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Tue, 6 Feb 2018 19:25:23 -0800 Subject: [PATCH 23/28] misc cleanup --- lib/ldclient-rb/in_memory_store.rb | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/ldclient-rb/in_memory_store.rb b/lib/ldclient-rb/in_memory_store.rb index f545e427..aea95d3b 100644 --- a/lib/ldclient-rb/in_memory_store.rb +++ b/lib/ldclient-rb/in_memory_store.rb @@ -56,9 +56,9 @@ def delete(kind, key, version) end end - def init(allData) + def init(all_data) @lock.with_write_lock do - @items.replace(allData) + @items.replace(all_data) @initialized.make_true end end From ea169710286cb40214e1b3cde9057346278a0cef Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Tue, 13 Feb 2018 10:52:30 -0800 Subject: [PATCH 24/28] single quotes, why not --- lib/ldclient-rb/redis_store.rb | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/lib/ldclient-rb/redis_store.rb b/lib/ldclient-rb/redis_store.rb index b64f5c41..2374eb23 100644 --- a/lib/ldclient-rb/redis_store.rb +++ b/lib/ldclient-rb/redis_store.rb @@ -94,12 +94,12 @@ def self.default_prefix def get(kind, key) f = @cache[cache_key(kind, key)] if f.nil? - @logger.debug("RedisFeatureStore: no cache hit for #{key} in \"#{kind[:namespace]}\", requesting from Redis") + @logger.debug("RedisFeatureStore: no cache hit for #{key} in '#{kind[:namespace]}', requesting from Redis") f = with_connection do |redis| begin get_redis(kind, redis, key.to_sym) rescue => e - @logger.error("RedisFeatureStore: could not retrieve #{key} from Redis in \"#{kind[:namespace]}\", with error: #{e}") + @logger.error("RedisFeatureStore: could not retrieve #{key} from Redis in '#{kind[:namespace]}', with error: #{e}") nil end end @@ -108,10 +108,10 @@ def get(kind, key) end end if f.nil? - @logger.debug("RedisFeatureStore: #{key} not found in \"#{kind[:namespace]}\"") + @logger.debug("RedisFeatureStore: #{key} not found in '#{kind[:namespace]}'") nil elsif f[:deleted] - @logger.debug("RedisFeatureStore: #{key} was deleted in \"#{kind[:namespace]}\", returning nil") + @logger.debug("RedisFeatureStore: #{key} was deleted in '#{kind[:namespace]}', returning nil") nil else f @@ -124,7 +124,7 @@ def all(kind) begin hashfs = redis.hgetall(items_key(kind)) rescue => e - @logger.error("RedisFeatureStore: could not retrieve all \"#{kind[:namespace]}\" items from Redis with error: #{e}; returning none") + @logger.error("RedisFeatureStore: could not retrieve all '#{kind[:namespace]}' items from Redis with error: #{e}; returning none") hashfs = {} end hashfs.each do |k, jsonItem| @@ -150,7 +150,7 @@ def delete(kind, key, version) put_redis_and_cache(kind, redis, key, f1) else @logger.warn("RedisFeatureStore: attempted to delete #{key} version: #{f[:version]} \ - in \"#{kind[:namespace]}\" with a version that is the same or older: #{version}") + in '#{kind[:namespace]}' with a version that is the same or older: #{version}") end end end From d0f5492a4723a653b89c9aea4279d3db728ab571 Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Tue, 13 Feb 2018 10:58:01 -0800 Subject: [PATCH 25/28] freeze constant properties --- lib/ldclient-rb/in_memory_store.rb | 3 +++ 1 file changed, 3 insertions(+) diff --git a/lib/ldclient-rb/in_memory_store.rb b/lib/ldclient-rb/in_memory_store.rb index aea95d3b..2373fd90 100644 --- a/lib/ldclient-rb/in_memory_store.rb +++ b/lib/ldclient-rb/in_memory_store.rb @@ -14,6 +14,9 @@ module LaunchDarkly namespace: "segments" } + FEATURES.freeze + SEGMENTS.freeze + # # Default implementation of the LaunchDarkly client's feature store, using an in-memory # cache. This object holds feature flags and related data received from the From 9dcceebd5348917a448cae2764e5c9407b64200b Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Tue, 13 Feb 2018 11:10:00 -0800 Subject: [PATCH 26/28] minor cleanup --- lib/ldclient-rb/in_memory_store.rb | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/lib/ldclient-rb/in_memory_store.rb b/lib/ldclient-rb/in_memory_store.rb index 2373fd90..e3e85879 100644 --- a/lib/ldclient-rb/in_memory_store.rb +++ b/lib/ldclient-rb/in_memory_store.rb @@ -8,14 +8,11 @@ module LaunchDarkly # constant here and the existing store should be able to handle it. FEATURES = { namespace: "features" - } + }.freeze SEGMENTS = { namespace: "segments" - } - - FEATURES.freeze - SEGMENTS.freeze + }.freeze # # Default implementation of the LaunchDarkly client's feature store, using an in-memory From 80caef2d59561d6d43acf0bc001b6767f17024ba Mon Sep 17 00:00:00 2001 From: Eli Bishop Date: Tue, 13 Feb 2018 11:16:36 -0800 Subject: [PATCH 27/28] add tests for weight=nil or absent --- spec/evaluation_spec.rb | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/spec/evaluation_spec.rb b/spec/evaluation_spec.rb index 0f8d1439..d166c348 100644 --- a/spec/evaluation_spec.rb +++ b/spec/evaluation_spec.rb @@ -411,6 +411,35 @@ def make_user_matching_clause(user, attr) expect(result).to be true end + it 'matches user by rule when weight is absent' do + segClause = make_user_matching_clause(user, :email) + segRule = { + clauses: [ segClause ] + } + segment = make_segment('segkey') + segment[:rules] = [ segRule ] + features.upsert(LaunchDarkly::SEGMENTS, segment) + clause = make_segment_match_clause(segment) + + result = clause_match_user(clause, user, features) + expect(result).to be true + end + + it 'matches user by rule when weight is nil' do + segClause = make_user_matching_clause(user, :email) + segRule = { + clauses: [ segClause ], + weight: nil + } + segment = make_segment('segkey') + segment[:rules] = [ segRule ] + features.upsert(LaunchDarkly::SEGMENTS, segment) + clause = make_segment_match_clause(segment) + + result = clause_match_user(clause, user, features) + expect(result).to be true + end + it 'matches user with full rollout' do segClause = make_user_matching_clause(user, :email) segRule = { From f2c085d33cfd15b298594f19257ea25a8a2c6cb6 Mon Sep 17 00:00:00 2001 From: Andrew Shannon Brown Date: Thu, 22 Feb 2018 12:02:01 -0800 Subject: [PATCH 28/28] [ch12303] Make pre ruby 2.1.0 faraday dependencies explicit and add jruby 9xx build (#40) Also work around jruby-9.0.5.0 issue with return statements inside lambdas used as blocks. --- circle.yml | 20 +++++++++++--------- ldclient-rb.gemspec | 9 +++++++-- lib/ldclient-rb/evaluation.rb | 6 ++++-- 3 files changed, 22 insertions(+), 13 deletions(-) diff --git a/circle.yml b/circle.yml index 7d82cb78..87a582d4 100644 --- a/circle.yml +++ b/circle.yml @@ -1,21 +1,23 @@ machine: environment: - RUBIES: "ruby-2.4.1;ruby-2.2.3;ruby-2.1.7;ruby-2.0.0;ruby-1.9.3;jruby-1.7.22" + RUBIES: "ruby-2.4.2 ruby-2.2.7 ruby-2.1.9 ruby-2.0.0 ruby-1.9.3 jruby-1.7.22 jruby-9.0.5.0 jruby-9.1.13.0" services: - redis dependencies: cache_directories: - - '../.rvm/rubies' + - '/opt/circleci/.rvm/rubies' override: - - > - rubiesArray=(${RUBIES//;/ }); - for i in "${rubiesArray[@]}"; + - | + set -e + for i in $RUBIES; do rvm install $i; rvm use $i; - gem install jruby-openssl; # required by bundler, no effect on Ruby MRI + if [[ $i == jruby* ]]; then + gem install jruby-openssl; # required by bundler, no effect on Ruby MRI + fi gem install bundler; bundle install; mv Gemfile.lock "Gemfile.lock.$i" @@ -23,9 +25,9 @@ dependencies: test: override: - - > - rubiesArray=(${RUBIES//;/ }); - for i in "${rubiesArray[@]}"; + - | + set -e + for i in $RUBIES; do rvm use $i; cp "Gemfile.lock.$i" Gemfile.lock; diff --git a/ldclient-rb.gemspec b/ldclient-rb.gemspec index 1a89c256..940d4fe3 100644 --- a/ldclient-rb.gemspec +++ b/ldclient-rb.gemspec @@ -30,8 +30,13 @@ Gem::Specification.new do |spec| spec.add_development_dependency "moneta", "~> 1.0.0" spec.add_runtime_dependency "json", [">= 1.8", "< 3"] - spec.add_runtime_dependency "faraday", [">= 0.9", "< 2"] - spec.add_runtime_dependency "faraday-http-cache", [">= 1.3.0", "< 3"] + if RUBY_VERSION >= "2.1.0" + spec.add_runtime_dependency "faraday", [">= 0.9", "< 2"] + spec.add_runtime_dependency "faraday-http-cache", [">= 1.3.0", "< 3"] + else + spec.add_runtime_dependency "faraday", [">= 0.9", "< 0.14.0"] + spec.add_runtime_dependency "faraday-http-cache", [">= 1.3.0", "< 2"] + end spec.add_runtime_dependency "semantic", "~> 1.6.0" spec.add_runtime_dependency "thread_safe", "~> 0.3" spec.add_runtime_dependency "net-http-persistent", "~> 2.9" diff --git a/lib/ldclient-rb/evaluation.rb b/lib/ldclient-rb/evaluation.rb index f9b855fb..5ff32dc4 100644 --- a/lib/ldclient-rb/evaluation.rb +++ b/lib/ldclient-rb/evaluation.rb @@ -22,16 +22,18 @@ module Evaluation end SEMVER_OPERAND = lambda do |v| + semver = nil if v.is_a? String for _ in 0..2 do begin - return Semantic::Version.new(v) + semver = Semantic::Version.new(v) + break # Some versions of jruby cannot properly handle a return here and return from the method that calls this lambda rescue ArgumentError v = addZeroVersionComponent(v) end end end - nil + semver end def self.addZeroVersionComponent(v)