You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by th...@apache.org on 2017/05/19 17:12:33 UTC

[19/36] avro git commit: [merge] marge with latest changes

http://git-wip-us.apache.org/repos/asf/avro/blob/884fbabd/lang/ruby/lib/avro/schema.rb
----------------------------------------------------------------------
diff --git a/lang/ruby/lib/avro/schema.rb b/lang/ruby/lib/avro/schema.rb
index 8e345e3..477528e 100644
--- a/lang/ruby/lib/avro/schema.rb
+++ b/lang/ruby/lib/avro/schema.rb
@@ -92,39 +92,10 @@ module Avro
 
     # Determine if a ruby datum is an instance of a schema
     def self.validate(expected_schema, datum)
-      case expected_schema.type_sym
-      when :null
-        datum.nil?
-      when :boolean
-        datum == true || datum == false
-      when :string, :bytes
-        datum.is_a? String
-      when :int
-        (datum.is_a?(Fixnum) || datum.is_a?(Bignum)) &&
-            (INT_MIN_VALUE <= datum) && (datum <= INT_MAX_VALUE)
-      when :long
-        (datum.is_a?(Fixnum) || datum.is_a?(Bignum)) &&
-            (LONG_MIN_VALUE <= datum) && (datum <= LONG_MAX_VALUE)
-      when :float, :double
-        datum.is_a?(Float) || datum.is_a?(Fixnum) || datum.is_a?(Bignum)
-      when :fixed
-        datum.is_a?(String) && datum.bytesize == expected_schema.size
-      when :enum
-        expected_schema.symbols.include? datum
-      when :array
-        datum.is_a?(Array) &&
-          datum.all?{|d| validate(expected_schema.items, d) }
-      when :map
-          datum.keys.all?{|k| k.is_a? String } &&
-          datum.values.all?{|v| validate(expected_schema.values, v) }
-      when :union
-        expected_schema.schemas.any?{|s| validate(s, datum) }
-      when :record, :error, :request
-        datum.is_a?(Hash) &&
-          expected_schema.fields.all?{|f| validate(f.type, datum[f.name]) }
-      else
-        raise "you suck #{expected_schema.inspect} is not allowed."
-      end
+      SchemaValidator.validate!(expected_schema, datum)
+      true
+    rescue SchemaValidator::ValidationError
+      false
     end
 
     def initialize(type)
@@ -347,7 +318,7 @@ module Avro
       attr_reader :size
       def initialize(name, space, size, names=nil)
         # Ensure valid cto args
-        unless size.is_a?(Fixnum) || size.is_a?(Bignum)
+        unless size.is_a?(Integer)
           raise AvroError, 'Fixed Schema requires a valid integer for size property.'
         end
         super(:fixed, name, space, names)

http://git-wip-us.apache.org/repos/asf/avro/blob/884fbabd/lang/ruby/lib/avro/schema_validator.rb
----------------------------------------------------------------------
diff --git a/lang/ruby/lib/avro/schema_validator.rb b/lang/ruby/lib/avro/schema_validator.rb
new file mode 100644
index 0000000..ca4f7f6
--- /dev/null
+++ b/lang/ruby/lib/avro/schema_validator.rb
@@ -0,0 +1,196 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+module Avro
+  class SchemaValidator
+    ROOT_IDENTIFIER = '.'.freeze
+    PATH_SEPARATOR = '.'.freeze
+    INT_RANGE = Schema::INT_MIN_VALUE..Schema::INT_MAX_VALUE
+    LONG_RANGE = Schema::LONG_MIN_VALUE..Schema::LONG_MAX_VALUE
+    COMPLEX_TYPES = [:array, :error, :map, :record, :request]
+
+    class Result
+      attr_reader :errors
+
+      def initialize
+        @errors = []
+      end
+
+      def <<(error)
+        @errors << error
+      end
+
+      def add_error(path, message)
+        self << "at #{path} #{message}"
+      end
+
+      def failure?
+        @errors.any?
+      end
+
+      def to_s
+        errors.join("\n")
+      end
+    end
+
+    class ValidationError < StandardError
+      attr_reader :result
+
+      def initialize(result = Result.new)
+        @result = result
+        super
+      end
+
+      def to_s
+        result.to_s
+      end
+    end
+
+    TypeMismatchError = Class.new(ValidationError)
+
+    class << self
+      def validate!(expected_schema, datum)
+        result = Result.new
+        validate_recursive(expected_schema, datum, ROOT_IDENTIFIER, result)
+        fail ValidationError, result if result.failure?
+        result
+      end
+
+      private
+
+      def validate_recursive(expected_schema, datum, path, result)
+        case expected_schema.type_sym
+        when :null
+          fail TypeMismatchError unless datum.nil?
+        when :boolean
+          fail TypeMismatchError unless [true, false].include?(datum)
+        when :string, :bytes
+          fail TypeMismatchError unless datum.is_a?(String)
+        when :int
+          fail TypeMismatchError unless datum.is_a?(Integer)
+          result.add_error(path, "out of bound value #{datum}") unless INT_RANGE.cover?(datum)
+        when :long
+          fail TypeMismatchError unless datum.is_a?(Integer)
+          result.add_error(path, "out of bound value #{datum}") unless LONG_RANGE.cover?(datum)
+        when :float, :double
+          fail TypeMismatchError unless [Float, Integer].any?(&datum.method(:is_a?))
+        when :fixed
+          if datum.is_a? String
+            message = "expected fixed with size #{expected_schema.size}, got \"#{datum}\" with size #{datum.size}"
+            result.add_error(path, message) unless datum.bytesize == expected_schema.size
+          else
+            result.add_error(path, "expected fixed with size #{expected_schema.size}, got #{actual_value_message(datum)}")
+          end
+        when :enum
+          message = "expected enum with values #{expected_schema.symbols}, got #{actual_value_message(datum)}"
+          result.add_error(path, message) unless expected_schema.symbols.include?(datum)
+        when :array
+          validate_array(expected_schema, datum, path, result)
+        when :map
+          validate_map(expected_schema, datum, path, result)
+        when :union
+          validate_union(expected_schema, datum, path, result)
+        when :record, :error, :request
+          fail TypeMismatchError unless datum.is_a?(Hash)
+          expected_schema.fields.each do |field|
+            deeper_path = deeper_path_for_hash(field.name, path)
+            validate_recursive(field.type, datum[field.name], deeper_path, result)
+          end
+        else
+          fail "Unexpected schema type #{expected_schema.type_sym} #{expected_schema.inspect}"
+        end
+      rescue TypeMismatchError
+        result.add_error(path, "expected type #{expected_schema.type_sym}, got #{actual_value_message(datum)}")
+      end
+
+      def validate_array(expected_schema, datum, path, result)
+        fail TypeMismatchError unless datum.is_a?(Array)
+        datum.each_with_index do |d, i|
+          validate_recursive(expected_schema.items, d, path + "[#{i}]", result)
+        end
+      end
+
+      def validate_map(expected_schema, datum, path, result)
+        datum.keys.each do |k|
+          result.add_error(path, "unexpected key type '#{ruby_to_avro_type(k.class)}' in map") unless k.is_a?(String)
+        end
+        datum.each do |k, v|
+          deeper_path = deeper_path_for_hash(k, path)
+          validate_recursive(expected_schema.values, v, deeper_path, result)
+        end
+      end
+
+      def validate_union(expected_schema, datum, path, result)
+        if expected_schema.schemas.size == 1
+          validate_recursive(expected_schema.schemas.first, datum, path, result)
+          return
+        end
+        types_and_results = validate_possible_types(datum, expected_schema, path)
+        failures, successes = types_and_results.partition { |r| r[:result].failure? }
+        return if successes.any?
+        complex_type_failed = failures.detect { |r| COMPLEX_TYPES.include?(r[:type]) }
+        if complex_type_failed
+          complex_type_failed[:result].errors.each { |error| result << error }
+        else
+          types = expected_schema.schemas.map { |s| "'#{s.type_sym}'" }.join(', ')
+          result.add_error(path, "expected union of [#{types}], got #{actual_value_message(datum)}")
+        end
+      end
+
+      def validate_possible_types(datum, expected_schema, path)
+        expected_schema.schemas.map do |schema|
+          result = Result.new
+          validate_recursive(schema, datum, path, result)
+          { type: schema.type_sym, result: result }
+        end
+      end
+
+      def deeper_path_for_hash(sub_key, path)
+        "#{path}#{PATH_SEPARATOR}#{sub_key}".squeeze(PATH_SEPARATOR)
+      end
+
+      private
+
+      def actual_value_message(value)
+        avro_type = if value.class == Integer
+                      ruby_integer_to_avro_type(value)
+                    else
+                      ruby_to_avro_type(value.class)
+                    end
+        if value.nil?
+          avro_type
+        else
+          "#{avro_type} with value #{value.inspect}"
+        end
+      end
+
+      def ruby_to_avro_type(ruby_class)
+        {
+          NilClass => 'null',
+          String => 'string',
+          Fixnum => 'int',
+          Bignum => 'long',
+          Float => 'float',
+          Hash => 'record'
+        }.fetch(ruby_class, ruby_class)
+      end
+
+      def ruby_integer_to_avro_type(value)
+        INT_RANGE.cover?(value) ? 'int' : 'long'
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/avro/blob/884fbabd/lang/ruby/test/test_schema.rb
----------------------------------------------------------------------
diff --git a/lang/ruby/test/test_schema.rb b/lang/ruby/test/test_schema.rb
index 1643307..a011823 100644
--- a/lang/ruby/test/test_schema.rb
+++ b/lang/ruby/test/test_schema.rb
@@ -27,13 +27,13 @@ class TestSchema < Test::Unit::TestCase
       ]}
     SCHEMA
 
-    assert_equal schema.name, 'OuterRecord'
-    assert_equal schema.fullname, 'OuterRecord'
+    assert_equal 'OuterRecord', schema.name
+    assert_equal 'OuterRecord', schema.fullname
     assert_nil schema.namespace
 
     schema.fields.each do |field|
-      assert_equal field.type.name, 'InnerRecord'
-      assert_equal field.type.fullname, 'InnerRecord'
+      assert_equal 'InnerRecord', field.type.name
+      assert_equal 'InnerRecord', field.type.fullname
       assert_nil field.type.namespace
     end
   end
@@ -50,13 +50,13 @@ class TestSchema < Test::Unit::TestCase
       ]}
     SCHEMA
 
-    assert_equal schema.name, 'OuterRecord'
-    assert_equal schema.fullname, 'my.name.space.OuterRecord'
-    assert_equal schema.namespace, 'my.name.space'
+    assert_equal 'OuterRecord', schema.name
+    assert_equal 'my.name.space.OuterRecord', schema.fullname
+    assert_equal 'my.name.space', schema.namespace
     schema.fields.each do |field|
-      assert_equal field.type.name, 'InnerRecord'
-      assert_equal field.type.fullname, 'my.name.space.InnerRecord'
-      assert_equal field.type.namespace, 'my.name.space'
+      assert_equal 'InnerRecord', field.type.name
+      assert_equal 'my.name.space.InnerRecord', field.type.fullname
+      assert_equal 'my.name.space', field.type.namespace
     end
   end
 
@@ -71,13 +71,13 @@ class TestSchema < Test::Unit::TestCase
       ]}
     SCHEMA
 
-    assert_equal schema.name, 'OuterRecord'
-    assert_equal schema.fullname, 'my.name.space.OuterRecord'
-    assert_equal schema.namespace, 'my.name.space'
+    assert_equal 'OuterRecord', schema.name
+    assert_equal 'my.name.space.OuterRecord', schema.fullname
+    assert_equal 'my.name.space', schema.namespace
     schema.fields.each do |field|
-      assert_equal field.type.name, 'InnerEnum'
-      assert_equal field.type.fullname, 'my.name.space.InnerEnum'
-      assert_equal field.type.namespace, 'my.name.space'
+      assert_equal 'InnerEnum', field.type.name
+      assert_equal 'my.name.space.InnerEnum', field.type.fullname
+      assert_equal 'my.name.space', field.type.namespace
     end
   end
 
@@ -96,18 +96,18 @@ class TestSchema < Test::Unit::TestCase
       ]}
     SCHEMA
 
-    assert_equal schema.name, 'OuterRecord'
-    assert_equal schema.fullname, 'outer.OuterRecord'
-    assert_equal schema.namespace, 'outer'
+    assert_equal 'OuterRecord', schema.name
+    assert_equal 'outer.OuterRecord', schema.fullname
+    assert_equal 'outer', schema.namespace
     middle = schema.fields.first.type
-    assert_equal middle.name, 'MiddleRecord'
-    assert_equal middle.fullname, 'middle.MiddleRecord'
-    assert_equal middle.namespace, 'middle'
+    assert_equal 'MiddleRecord', middle.name
+    assert_equal 'middle.MiddleRecord', middle.fullname
+    assert_equal 'middle', middle.namespace
     inner = middle.fields.first.type
-    assert_equal inner.name, 'InnerRecord'
-    assert_equal inner.fullname, 'middle.InnerRecord'
-    assert_equal inner.namespace, 'middle'
-    assert_equal inner.fields.first.type, middle
+    assert_equal 'InnerRecord', inner.name
+    assert_equal 'middle.InnerRecord', inner.fullname
+    assert_equal 'middle', inner.namespace
+    assert_equal middle, inner.fields.first.type
   end
 
   def test_to_avro_includes_namespaces
@@ -120,7 +120,7 @@ class TestSchema < Test::Unit::TestCase
       ]}
     SCHEMA
 
-    assert_equal schema.to_avro, {
+    assert_equal({
       'type' => 'record', 'name' => 'OuterRecord', 'namespace' => 'my.name.space',
       'fields' => [
         {'name' => 'definition', 'type' => {
@@ -129,7 +129,7 @@ class TestSchema < Test::Unit::TestCase
         }},
         {'name' => 'reference', 'type' => 'my.name.space.InnerFixed'}
       ]
-    }
+    }, schema.to_avro)
   end
 
   def test_unknown_named_type

http://git-wip-us.apache.org/repos/asf/avro/blob/884fbabd/lang/ruby/test/test_schema_validator.rb
----------------------------------------------------------------------
diff --git a/lang/ruby/test/test_schema_validator.rb b/lang/ruby/test/test_schema_validator.rb
new file mode 100644
index 0000000..93f2ca8
--- /dev/null
+++ b/lang/ruby/test/test_schema_validator.rb
@@ -0,0 +1,402 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+require 'test_help'
+
+class TestSchema < Test::Unit::TestCase
+  def validate!(schema, value)
+    Avro::SchemaValidator.validate!(schema, value)
+  end
+
+  def hash_to_schema(hash)
+    Avro::Schema.parse(hash.to_json)
+  end
+
+  def assert_failed_validation(messages)
+    error = assert_raise(Avro::SchemaValidator::ValidationError) { yield }
+
+    assert_messages = [messages].flatten
+    result_errors = error.result.errors
+    assert_messages.each do |message|
+      assert(result_errors.include?(message), "expected '#{message}' to be in '#{result_errors}'")
+    end
+    assert_equal(assert_messages.size, result_errors.size)
+  end
+
+  def assert_valid_schema(schema, valid, invalid)
+    valid.each do |value|
+      assert_nothing_raised { Avro::SchemaValidator.validate!(schema, value) }
+    end
+
+    invalid.each do |value|
+      assert_raise { Avro::SchemaValidator.validate!(schema, value) }
+    end
+  end
+
+  def test_validate_nil
+    schema = hash_to_schema(type: 'null', name: 'name')
+
+    assert_nothing_raised { validate!(schema, nil) }
+
+    assert_failed_validation('at . expected type null, got int with value 1') do
+      validate!(schema, 1)
+    end
+  end
+
+  def test_validate_boolean
+    schema = hash_to_schema(type: 'boolean', name: 'name')
+
+    assert_nothing_raised { validate!(schema, true) }
+    assert_nothing_raised { validate!(schema, false) }
+
+    assert_failed_validation('at . expected type boolean, got int with value 1') do
+      validate!(schema, 1)
+    end
+
+    assert_failed_validation('at . expected type boolean, got null') do
+      validate!(schema, nil)
+    end
+  end
+
+  def test_fixed_size_string
+    schema = hash_to_schema(type: 'fixed', name: 'some', size: 3)
+
+    assert_nothing_raised { validate!(schema, 'baf') }
+
+    assert_failed_validation('at . expected fixed with size 3, got "some" with size 4') do
+      validate!(schema, 'some')
+    end
+
+    assert_failed_validation('at . expected fixed with size 3, got null') do
+      validate!(schema, nil)
+    end
+  end
+
+  def test_original_validate_nil
+    schema = hash_to_schema(type: 'null', name: 'name')
+
+    assert_valid_schema(schema, [nil], ['something'])
+  end
+
+  def test_original_validate_boolean
+    schema = hash_to_schema(type: 'boolean', name: 'name')
+
+    assert_valid_schema(schema, [true, false], [nil, 1])
+  end
+
+  def test_validate_string
+    schema = hash_to_schema(type: 'string', name: 'name')
+
+    assert_valid_schema(schema, ['string'], [nil, 1])
+  end
+
+  def test_validate_bytes
+    schema = hash_to_schema(type: 'bytes', name: 'name')
+
+    assert_valid_schema(schema, ['string'], [nil, 1])
+  end
+
+  def test_validate_int
+    schema = hash_to_schema(type: 'int', name: 'name')
+
+    assert_valid_schema(
+      schema,
+      [Avro::Schema::INT_MIN_VALUE, Avro::Schema::INT_MAX_VALUE, 1],
+      [Avro::Schema::LONG_MIN_VALUE, Avro::Schema::LONG_MAX_VALUE, 'string']
+    )
+    assert_failed_validation('at . out of bound value 9223372036854775807') do
+      validate!(schema, Avro::Schema::LONG_MAX_VALUE)
+    end
+  end
+
+  def test_validate_long
+    schema = hash_to_schema(type: 'long', name: 'name')
+
+    assert_valid_schema(schema, [Avro::Schema::LONG_MIN_VALUE, Avro::Schema::LONG_MAX_VALUE, 1], [1.1, 'string'])
+  end
+
+  def test_validate_float
+    schema = hash_to_schema(type: 'float', name: 'name')
+
+    assert_valid_schema(schema, [1.1, 1, Avro::Schema::LONG_MAX_VALUE], ['string'])
+  end
+
+  def test_validate_double
+    schema = hash_to_schema(type: 'double', name: 'name')
+
+    assert_valid_schema(schema, [1.1, 1, Avro::Schema::LONG_MAX_VALUE], ['string'])
+  end
+
+  def test_validate_fixed
+    schema = hash_to_schema(type: 'fixed', name: 'name', size: 3)
+
+    assert_valid_schema(schema, ['abc'], ['ab', 1, 1.1, true])
+  end
+
+  def test_validate_original_num
+    schema = hash_to_schema(type: 'enum', name: 'name', symbols: %w(a b))
+
+    assert_valid_schema(schema, ['a', 'b'], ['c'])
+  end
+
+  def test_validate_record
+    schema = hash_to_schema(type: 'record', name: 'name', fields: [{ type: 'null', name: 'sub' }])
+
+    assert_valid_schema(schema, [{ 'sub' => nil }], [{ 'sub' => 1 }])
+  end
+
+  def test_validate_shallow_record
+    schema = hash_to_schema(
+      type: 'record', name: 'name', fields: [{ type: 'int', name: 'sub' }]
+    )
+
+    assert_nothing_raised { validate!(schema, 'sub' => 1) }
+
+    assert_failed_validation('at .sub expected type int, got null') do
+      validate!(schema, {})
+    end
+
+    assert_failed_validation('at . expected type record, got float with value 1.2') do
+      validate!(schema, 1.2)
+    end
+
+    assert_failed_validation('at .sub expected type int, got float with value 1.2') do
+      validate!(schema, 'sub' => 1.2)
+    end
+
+    assert_failed_validation('at .sub expected type int, got null') do
+      validate!(schema, {})
+    end
+  end
+
+  def test_validate_array
+    schema = hash_to_schema(type: 'array',
+                            name: 'person',
+                            items: [{ type: 'int', name: 'height' }])
+
+    assert_nothing_raised { validate!(schema, []) }
+
+    assert_failed_validation 'at . expected type array, got null' do
+      validate!(schema, nil)
+    end
+
+    assert_failed_validation('at .[0] expected type int, got null') do
+      validate!(schema, [nil])
+    end
+
+    assert_failed_validation('at .[3] expected type int, got string with value "so wrong"') do
+      validate!(schema, [1, 3, 9, 'so wrong'])
+    end
+  end
+
+  def test_validate_enum
+    schema = hash_to_schema(type: 'enum',
+                            name: 'person',
+                            symbols: %w(one two three))
+
+    assert_nothing_raised { validate!(schema, 'one') }
+
+    assert_failed_validation('at . expected enum with values ["one", "two", "three"], got string with value "five"') do
+      validate!(schema, 'five')
+    end
+  end
+
+  def test_validate_union_on_primitive_types
+    schema = hash_to_schema(
+      name: 'should_not_matter',
+      type: 'record',
+      fields: [
+        { name: 'what_ever', type: %w(long string) }
+      ]
+    )
+
+    assert_failed_validation('at .what_ever expected union of [\'long\', \'string\'], got null') {
+      validate!(schema, 'what_ever' => nil)
+    }
+  end
+
+  def test_validate_union_of_nil_and_record_inside_array
+    schema = hash_to_schema(
+      name: 'this does not matter',
+      type: 'record',
+      fields: [
+        {
+          name: 'person',
+          type: {
+            name: 'person_entry',
+            type: 'record',
+            fields: [
+              {
+                name: 'houses',
+                type: [
+                  'null',
+                  {
+                    name: 'houses_entry',
+                    type: 'array',
+                    items: [
+                      {
+                        name: 'house_entry',
+                        type: 'record',
+                        fields: [
+                          { name: 'number_of_rooms', type: 'long' }
+                        ]
+                      }
+                    ]
+                  }
+                ],
+              }
+            ]
+          }
+        }
+      ]
+    )
+
+    assert_failed_validation('at .person expected type record, got null') {
+      validate!(schema, 'not at all' => nil)
+    }
+    assert_nothing_raised { validate!(schema, 'person' => {}) }
+    assert_nothing_raised { validate!(schema, 'person' => { houses: [] }) }
+    assert_nothing_raised { validate!(schema, 'person' => { 'houses' => [{ 'number_of_rooms' => 1 }] }) }
+
+    message = 'at .person.houses[1].number_of_rooms expected type long, got string with value "not valid at all"'
+    assert_failed_validation(message) do
+      validate!(
+        schema,
+        'person' => {
+          'houses' => [
+            { 'number_of_rooms' => 2 },
+            { 'number_of_rooms' => 'not valid at all' }
+          ]
+        }
+      )
+    end
+  end
+
+  def test_validate_map
+    schema = hash_to_schema(type: 'map',
+                            name: 'numbers',
+                            values: [
+                              { name: 'some', type: 'int' }
+                            ])
+
+    assert_nothing_raised { validate!(schema, 'some' => 1) }
+
+    assert_failed_validation('at .some expected type int, got string with value "nope"') do
+      validate!(schema, 'some' => 'nope')
+    end
+
+    assert_failed_validation("at . unexpected key type 'Symbol' in map") do
+      validate!(schema, some: 1)
+    end
+  end
+
+  def test_validate_deep_record
+    schema = hash_to_schema(type: 'record',
+                            name: 'person',
+                            fields: [
+                              {
+                                name: 'head',
+                                type: {
+                                  name: 'head',
+                                  type: 'record',
+                                  fields: [
+                                    {
+                                      name: 'hair',
+                                      type: {
+                                        name: 'hair',
+                                        type: 'record',
+                                        fields: [
+                                          {
+                                            name: 'color',
+                                            type: 'string'
+                                          }
+                                        ]
+                                      }
+                                    }
+                                  ]
+                                }
+                              }
+                            ])
+
+    assert_nothing_raised { validate!(schema, 'head' => { 'hair' => { 'color' => 'black' } }) }
+
+    assert_failed_validation('at .head.hair.color expected type string, got null') do
+      validate!(schema, 'head' => { 'hair' => { 'color' => nil } })
+    end
+
+    assert_failed_validation('at .head.hair.color expected type string, got null') do
+      validate!(schema, 'head' => { 'hair' => {} })
+    end
+
+    assert_failed_validation('at .head.hair expected type record, got null') do
+      validate!(schema, 'head' => {})
+    end
+
+    assert_failed_validation('at . expected type record, got null') do
+      validate!(schema, nil)
+    end
+  end
+
+  def test_validate_deep_record_with_array
+    schema = hash_to_schema(type: 'record',
+                            name: 'fruits',
+                            fields: [
+                              {
+                                name: 'fruits',
+                                type: {
+                                  name: 'fruits',
+                                  type: 'array',
+                                  items: [
+                                    {
+                                      name: 'fruit',
+                                      type: 'record',
+                                      fields: [
+                                        { name: 'name', type: 'string' },
+                                        { name: 'weight', type: 'float' }
+                                      ]
+                                    }
+                                  ]
+                                }
+                              }
+                            ])
+    assert_nothing_raised { validate!(schema, 'fruits' => [{ 'name' => 'apple', 'weight' => 30.2 }]) }
+
+    assert_failed_validation('at .fruits[0].name expected type string, got null') do
+      validate!(schema, 'fruits' => [{ 'name' => nil, 'weight' => 30.2 }])
+    end
+
+    assert_failed_validation('at .fruits expected type array, got int with value 1') do
+      validate!(schema, 'fruits' => 1)
+    end
+  end
+
+  def test_validate_multiple_errors
+    schema = hash_to_schema(type: 'array',
+                            name: 'ages',
+                            items: [
+                              { type: 'int', name: 'age' }
+                            ])
+
+    exception = assert_raise(Avro::SchemaValidator::ValidationError) do
+      validate!(schema, [nil, 'e'])
+    end
+    assert_equal 2, exception.result.errors.size
+    assert_equal(
+      "at .[0] expected type int, got null\nat .[1] expected type int, got string with value \"e\"",
+      exception.to_s
+    )
+  end
+end