apache_avro/schema/
mod.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! Logic for parsing and interacting with schemas in Avro format.
19
20mod builders;
21mod name;
22mod parser;
23mod record;
24mod resolve;
25mod union;
26
27pub(crate) use crate::schema::resolve::{
28    ResolvedOwnedSchema, resolve_names, resolve_names_with_schemata,
29};
30pub use crate::schema::{
31    name::{Alias, Aliases, Name, Names, NamesRef, Namespace},
32    record::{
33        RecordField, RecordFieldBuilder, RecordFieldOrder, RecordSchema, RecordSchemaBuilder,
34    },
35    resolve::ResolvedSchema,
36    union::UnionSchema,
37};
38use crate::{
39    AvroResult,
40    error::{Details, Error},
41    schema::{parser::Parser, record::RecordSchemaParseLocation},
42    schema_equality,
43    types::{self, Value},
44};
45use digest::Digest;
46use serde::{
47    Serialize, Serializer,
48    ser::{Error as _, SerializeMap, SerializeSeq},
49};
50use serde_json::{Map, Value as JsonValue};
51use std::{
52    collections::{BTreeMap, HashMap, HashSet},
53    fmt,
54    fmt::Debug,
55    hash::Hash,
56    io::Read,
57};
58use strum::{Display, EnumDiscriminants};
59
60/// Represents documentation for complex Avro schemas.
61pub type Documentation = Option<String>;
62
63/// Represents an Avro schema fingerprint.
64///
65/// More information about Avro schema fingerprints can be found in the
66/// [Avro Schema Fingerprint documentation](https://avro.apache.org/docs/++version++/specification/#schema-fingerprints)
67pub struct SchemaFingerprint {
68    pub bytes: Vec<u8>,
69}
70
71impl fmt::Display for SchemaFingerprint {
72    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
73        write!(
74            f,
75            "{}",
76            self.bytes
77                .iter()
78                .map(|byte| format!("{byte:02x}"))
79                .collect::<Vec<String>>()
80                .join("")
81        )
82    }
83}
84
85/// Represents any valid Avro schema
86/// More information about Avro schemas can be found in the
87/// [Avro Specification](https://avro.apache.org/docs/++version++/specification/#schema-declaration)
88#[derive(Clone, Debug, EnumDiscriminants, Display)]
89#[strum_discriminants(name(SchemaKind), derive(Hash, Ord, PartialOrd))]
90pub enum Schema {
91    /// A `null` Avro schema.
92    Null,
93    /// A `boolean` Avro schema.
94    Boolean,
95    /// An `int` Avro schema.
96    Int,
97    /// A `long` Avro schema.
98    Long,
99    /// A `float` Avro schema.
100    Float,
101    /// A `double` Avro schema.
102    Double,
103    /// A `bytes` Avro schema.
104    ///
105    /// `Bytes` represents a sequence of 8-bit unsigned bytes.
106    Bytes,
107    /// A `string` Avro schema.
108    ///
109    /// `String` represents a unicode character sequence.
110    String,
111    /// An `array` Avro schema.
112    ///
113    /// All items will have the same schema.
114    Array(ArraySchema),
115    /// A `map` Avro schema.
116    ///
117    /// Keys are always a `Schema::String` and all values will have the same schema.
118    Map(MapSchema),
119    /// A `union` Avro schema.
120    Union(UnionSchema),
121    /// A `record` Avro schema.
122    Record(RecordSchema),
123    /// An `enum` Avro schema.
124    Enum(EnumSchema),
125    /// A `fixed` Avro schema.
126    Fixed(FixedSchema),
127    /// Logical type which represents `Decimal` values.
128    ///
129    /// The underlying type is serialized and deserialized as `Schema::Bytes` or `Schema::Fixed`.
130    Decimal(DecimalSchema),
131    /// Logical type which represents `Decimal` values without predefined scale.
132    ///
133    /// The underlying type is serialized and deserialized as `Schema::Bytes`
134    BigDecimal,
135    /// A universally unique identifier, annotating a string, bytes or fixed.
136    Uuid(UuidSchema),
137    /// Logical type which represents the number of days since the unix epoch.
138    ///
139    /// Serialization format is `Schema::Int`.
140    Date,
141    /// The time of day in number of milliseconds after midnight.
142    ///
143    /// This type has no reference to any calendar, time zone or date in particular.
144    TimeMillis,
145    /// The time of day in number of microseconds after midnight.
146    ///
147    /// This type has no reference to any calendar, time zone or date in particular.
148    TimeMicros,
149    /// An instant in time represented as the number of milliseconds after the UNIX epoch.
150    TimestampMillis,
151    /// An instant in time represented as the number of microseconds after the UNIX epoch.
152    TimestampMicros,
153    /// An instant in time represented as the number of nanoseconds after the UNIX epoch.
154    TimestampNanos,
155    /// An instant in localtime represented as the number of milliseconds after the UNIX epoch.
156    LocalTimestampMillis,
157    /// An instant in local time represented as the number of microseconds after the UNIX epoch.
158    LocalTimestampMicros,
159    /// An instant in local time represented as the number of nanoseconds after the UNIX epoch.
160    LocalTimestampNanos,
161    /// An amount of time defined by a number of months, days and milliseconds.
162    Duration(FixedSchema),
163    /// A reference to another schema.
164    Ref { name: Name },
165}
166
167#[derive(Clone, Debug, PartialEq)]
168pub struct MapSchema {
169    pub types: Box<Schema>,
170    pub default: Option<HashMap<String, Value>>,
171    pub attributes: BTreeMap<String, JsonValue>,
172}
173
174#[derive(Clone, Debug, PartialEq)]
175pub struct ArraySchema {
176    pub items: Box<Schema>,
177    pub default: Option<Vec<Value>>,
178    pub attributes: BTreeMap<String, JsonValue>,
179}
180
181impl PartialEq for Schema {
182    /// Assess equality of two `Schema` based on [Parsing Canonical Form].
183    ///
184    /// [Parsing Canonical Form]:
185    /// https://avro.apache.org/docs/1.11.1/specification/#parsing-canonical-form-for-schemas
186    fn eq(&self, other: &Self) -> bool {
187        schema_equality::compare_schemata(self, other)
188    }
189}
190
191impl SchemaKind {
192    pub fn is_primitive(self) -> bool {
193        matches!(
194            self,
195            SchemaKind::Null
196                | SchemaKind::Boolean
197                | SchemaKind::Int
198                | SchemaKind::Long
199                | SchemaKind::Double
200                | SchemaKind::Float
201                | SchemaKind::Bytes
202                | SchemaKind::String,
203        )
204    }
205
206    #[deprecated(since = "0.22.0", note = "Use Schema::is_named instead")]
207    pub fn is_named(self) -> bool {
208        matches!(
209            self,
210            SchemaKind::Record
211                | SchemaKind::Enum
212                | SchemaKind::Fixed
213                | SchemaKind::Ref
214                | SchemaKind::Duration
215        )
216    }
217}
218
219impl From<&types::Value> for SchemaKind {
220    fn from(value: &types::Value) -> Self {
221        use crate::types::Value;
222        match value {
223            Value::Null => Self::Null,
224            Value::Boolean(_) => Self::Boolean,
225            Value::Int(_) => Self::Int,
226            Value::Long(_) => Self::Long,
227            Value::Float(_) => Self::Float,
228            Value::Double(_) => Self::Double,
229            Value::Bytes(_) => Self::Bytes,
230            Value::String(_) => Self::String,
231            Value::Array(_) => Self::Array,
232            Value::Map(_) => Self::Map,
233            Value::Union(_, _) => Self::Union,
234            Value::Record(_) => Self::Record,
235            Value::Enum(_, _) => Self::Enum,
236            Value::Fixed(_, _) => Self::Fixed,
237            Value::Decimal { .. } => Self::Decimal,
238            Value::BigDecimal(_) => Self::BigDecimal,
239            Value::Uuid(_) => Self::Uuid,
240            Value::Date(_) => Self::Date,
241            Value::TimeMillis(_) => Self::TimeMillis,
242            Value::TimeMicros(_) => Self::TimeMicros,
243            Value::TimestampMillis(_) => Self::TimestampMillis,
244            Value::TimestampMicros(_) => Self::TimestampMicros,
245            Value::TimestampNanos(_) => Self::TimestampNanos,
246            Value::LocalTimestampMillis(_) => Self::LocalTimestampMillis,
247            Value::LocalTimestampMicros(_) => Self::LocalTimestampMicros,
248            Value::LocalTimestampNanos(_) => Self::LocalTimestampNanos,
249            Value::Duration { .. } => Self::Duration,
250        }
251    }
252}
253
254/// A description of an Enum schema.
255#[derive(bon::Builder, Debug, Clone)]
256pub struct EnumSchema {
257    /// The name of the schema
258    pub name: Name,
259    /// The aliases of the schema
260    #[builder(default)]
261    pub aliases: Aliases,
262    /// The documentation of the schema
263    #[builder(default)]
264    pub doc: Documentation,
265    /// The set of symbols of the schema
266    pub symbols: Vec<String>,
267    /// An optional default symbol used for compatibility
268    pub default: Option<String>,
269    /// The custom attributes of the schema
270    #[builder(default = BTreeMap::new())]
271    pub attributes: BTreeMap<String, JsonValue>,
272}
273
274/// A description of a Fixed schema.
275#[derive(bon::Builder, Debug, Clone)]
276pub struct FixedSchema {
277    /// The name of the schema
278    pub name: Name,
279    /// The aliases of the schema
280    #[builder(default)]
281    pub aliases: Aliases,
282    /// The documentation of the schema
283    #[builder(default)]
284    pub doc: Documentation,
285    /// The size of the fixed schema
286    pub size: usize,
287    /// The custom attributes of the schema
288    #[builder(default = BTreeMap::new())]
289    pub attributes: BTreeMap<String, JsonValue>,
290}
291
292impl FixedSchema {
293    fn serialize_to_map<S>(&self, mut map: S::SerializeMap) -> Result<S::SerializeMap, S::Error>
294    where
295        S: Serializer,
296    {
297        map.serialize_entry("type", "fixed")?;
298        if let Some(n) = self.name.namespace.as_ref() {
299            map.serialize_entry("namespace", n)?;
300        }
301        map.serialize_entry("name", &self.name.name)?;
302        if let Some(docstr) = self.doc.as_ref() {
303            map.serialize_entry("doc", docstr)?;
304        }
305        map.serialize_entry("size", &self.size)?;
306
307        if let Some(aliases) = self.aliases.as_ref() {
308            map.serialize_entry("aliases", aliases)?;
309        }
310
311        for attr in &self.attributes {
312            map.serialize_entry(attr.0, attr.1)?;
313        }
314
315        Ok(map)
316    }
317
318    /// Create a new `FixedSchema` copying only the size.
319    ///
320    /// All other fields are `None` or empty.
321    pub(crate) fn copy_only_size(&self) -> Self {
322        Self {
323            name: Name {
324                name: String::new(),
325                namespace: None,
326            },
327            aliases: None,
328            doc: None,
329            size: self.size,
330            attributes: Default::default(),
331        }
332    }
333}
334
335/// A description of a Decimal schema.
336///
337/// `scale` defaults to 0 and is an integer greater than or equal to 0 and `precision` is an
338/// integer greater than 0.
339#[derive(Debug, Clone)]
340pub struct DecimalSchema {
341    /// The number of digits in the unscaled value
342    pub precision: DecimalMetadata,
343    /// The number of digits to the right of the decimal point
344    pub scale: DecimalMetadata,
345    /// The inner schema of the decimal (fixed or bytes)
346    pub inner: InnerDecimalSchema,
347}
348
349/// The inner schema of the Decimal type.
350#[derive(Debug, Clone)]
351pub enum InnerDecimalSchema {
352    Bytes,
353    Fixed(FixedSchema),
354}
355
356impl TryFrom<Schema> for InnerDecimalSchema {
357    type Error = Error;
358
359    fn try_from(value: Schema) -> Result<Self, Self::Error> {
360        match value {
361            Schema::Bytes => Ok(InnerDecimalSchema::Bytes),
362            Schema::Fixed(fixed) => Ok(InnerDecimalSchema::Fixed(fixed)),
363            _ => Err(Details::ResolveDecimalSchema(value.into()).into()),
364        }
365    }
366}
367
368/// The inner schema of the Uuid type.
369#[derive(Debug, Clone)]
370pub enum UuidSchema {
371    /// [`Schema::Bytes`] with size of 16.
372    ///
373    /// This is not according to specification, but was what happened in `0.21.0` and earlier when
374    /// a schema with logical type `uuid` and inner type `fixed` was used.
375    Bytes,
376    /// [`Schema::String`].
377    String,
378    /// [`Schema::Fixed`] with size of 16.
379    Fixed(FixedSchema),
380}
381
382type DecimalMetadata = usize;
383pub(crate) type Precision = DecimalMetadata;
384pub(crate) type Scale = DecimalMetadata;
385
386impl Schema {
387    /// Converts `self` into its [Parsing Canonical Form].
388    ///
389    /// [Parsing Canonical Form]:
390    /// https://avro.apache.org/docs/++version++/specification/#parsing-canonical-form-for-schemas
391    pub fn canonical_form(&self) -> String {
392        let json = serde_json::to_value(self)
393            .unwrap_or_else(|e| panic!("Cannot parse Schema from JSON: {e}"));
394        let mut defined_names = HashSet::new();
395        parsing_canonical_form(&json, &mut defined_names)
396    }
397
398    /// Returns the [Parsing Canonical Form] of `self` that is self contained (not dependent on
399    /// any definitions in `schemata`)
400    ///
401    /// If you require a self contained schema including `default` and `doc` attributes, see [`denormalize`][Schema::denormalize].
402    ///
403    /// [Parsing Canonical Form]:
404    /// https://avro.apache.org/docs/++version++/specification/#parsing-canonical-form-for-schemas
405    pub fn independent_canonical_form(&self, schemata: &[Schema]) -> Result<String, Error> {
406        let mut this = self.clone();
407        this.denormalize(schemata)?;
408        Ok(this.canonical_form())
409    }
410
411    /// Generate the [fingerprint] of the schema's [Parsing Canonical Form].
412    ///
413    /// # Example
414    /// ```
415    /// use apache_avro::rabin::Rabin;
416    /// use apache_avro::{Schema, Error};
417    /// use md5::Md5;
418    /// use sha2::Sha256;
419    ///
420    /// fn main() -> Result<(), Error> {
421    ///     let raw_schema = r#"
422    ///         {
423    ///             "type": "record",
424    ///             "name": "test",
425    ///             "fields": [
426    ///                 {"name": "a", "type": "long", "default": 42},
427    ///                 {"name": "b", "type": "string"}
428    ///             ]
429    ///         }
430    ///     "#;
431    ///     let schema = Schema::parse_str(raw_schema)?;
432    ///     println!("{}", schema.fingerprint::<Sha256>());
433    ///     println!("{}", schema.fingerprint::<Md5>());
434    ///     println!("{}", schema.fingerprint::<Rabin>());
435    ///     Ok(())
436    /// }
437    /// ```
438    ///
439    /// [Parsing Canonical Form]:
440    /// https://avro.apache.org/docs/++version++/specification/#parsing-canonical-form-for-schemas
441    /// [fingerprint]:
442    /// https://avro.apache.org/docs/++version++/specification/#schema-fingerprints
443    pub fn fingerprint<D: Digest>(&self) -> SchemaFingerprint {
444        let mut d = D::new();
445        d.update(self.canonical_form());
446        SchemaFingerprint {
447            bytes: d.finalize().to_vec(),
448        }
449    }
450
451    /// Create a `Schema` from a string representing a JSON Avro schema.
452    pub fn parse_str(input: &str) -> Result<Schema, Error> {
453        let mut parser = Parser::default();
454        parser.parse_str(input)
455    }
456
457    /// Create an array of `Schema`'s from a list of named JSON Avro schemas (Record, Enum, and
458    /// Fixed).
459    ///
460    /// It is allowed that the schemas have cross-dependencies; these will be resolved
461    /// during parsing.
462    ///
463    /// If two of the input schemas have the same fullname, an Error will be returned.
464    pub fn parse_list(input: impl IntoIterator<Item = impl AsRef<str>>) -> AvroResult<Vec<Schema>> {
465        let input = input.into_iter();
466        let input_len = input.size_hint().0;
467        let mut input_schemas: HashMap<Name, JsonValue> = HashMap::with_capacity(input_len);
468        let mut input_order: Vec<Name> = Vec::with_capacity(input_len);
469        for json in input {
470            let json = json.as_ref();
471            let schema: JsonValue = serde_json::from_str(json).map_err(Details::ParseSchemaJson)?;
472            if let JsonValue::Object(inner) = &schema {
473                let name = Name::parse(inner, &None)?;
474                let previous_value = input_schemas.insert(name.clone(), schema);
475                if previous_value.is_some() {
476                    return Err(Details::NameCollision(name.fullname(None)).into());
477                }
478                input_order.push(name);
479            } else {
480                return Err(Details::GetNameField.into());
481            }
482        }
483        let mut parser = Parser::new(
484            input_schemas,
485            input_order,
486            HashMap::with_capacity(input_len),
487        );
488        parser.parse_list()
489    }
490
491    /// Create a `Schema` from a string representing a JSON Avro schema,
492    /// along with an array of `Schema`'s from a list of named JSON Avro schemas (Record, Enum, and
493    /// Fixed).
494    ///
495    /// It is allowed that the schemas have cross-dependencies; these will be resolved
496    /// during parsing.
497    ///
498    /// If two of the named input schemas have the same fullname, an Error will be returned.
499    ///
500    /// # Arguments
501    /// * `schema` - the JSON string of the schema to parse
502    /// * `schemata` - a slice of additional schemas that is used to resolve cross-references
503    pub fn parse_str_with_list(
504        schema: &str,
505        schemata: impl IntoIterator<Item = impl AsRef<str>>,
506    ) -> AvroResult<(Schema, Vec<Schema>)> {
507        let schemata = schemata.into_iter();
508        let schemata_len = schemata.size_hint().0;
509        let mut input_schemas: HashMap<Name, JsonValue> = HashMap::with_capacity(schemata_len);
510        let mut input_order: Vec<Name> = Vec::with_capacity(schemata_len);
511        for json in schemata {
512            let json = json.as_ref();
513            let schema: JsonValue = serde_json::from_str(json).map_err(Details::ParseSchemaJson)?;
514            if let JsonValue::Object(inner) = &schema {
515                let name = Name::parse(inner, &None)?;
516                if let Some(_previous) = input_schemas.insert(name.clone(), schema) {
517                    return Err(Details::NameCollision(name.fullname(None)).into());
518                }
519                input_order.push(name);
520            } else {
521                return Err(Details::GetNameField.into());
522            }
523        }
524        let mut parser = Parser::new(
525            input_schemas,
526            input_order,
527            HashMap::with_capacity(schemata_len),
528        );
529        parser.parse_input_schemas()?;
530
531        let value = serde_json::from_str(schema).map_err(Details::ParseSchemaJson)?;
532        let schema = parser.parse(&value, &None)?;
533        let schemata = parser.parse_list()?;
534        Ok((schema, schemata))
535    }
536
537    /// Create a `Schema` from a reader which implements [`Read`].
538    pub fn parse_reader(reader: &mut (impl Read + ?Sized)) -> AvroResult<Schema> {
539        let mut buf = String::new();
540        match reader.read_to_string(&mut buf) {
541            Ok(_) => Self::parse_str(&buf),
542            Err(e) => Err(Details::ReadSchemaFromReader(e).into()),
543        }
544    }
545
546    /// Parses an Avro schema from JSON.
547    pub fn parse(value: &JsonValue) -> AvroResult<Schema> {
548        let mut parser = Parser::default();
549        parser.parse(value, &None)
550    }
551
552    /// Parses an Avro schema from JSON.
553    /// Any `Schema::Ref`s must be known in the `names` map.
554    pub(crate) fn parse_with_names(value: &JsonValue, names: Names) -> AvroResult<Schema> {
555        let mut parser = Parser::new(HashMap::with_capacity(1), Vec::with_capacity(1), names);
556        parser.parse(value, &None)
557    }
558
559    /// Returns the custom attributes (metadata) if the schema supports them.
560    pub fn custom_attributes(&self) -> Option<&BTreeMap<String, JsonValue>> {
561        match self {
562            Schema::Record(RecordSchema { attributes, .. })
563            | Schema::Enum(EnumSchema { attributes, .. })
564            | Schema::Fixed(FixedSchema { attributes, .. })
565            | Schema::Array(ArraySchema { attributes, .. })
566            | Schema::Map(MapSchema { attributes, .. })
567            | Schema::Decimal(DecimalSchema {
568                inner: InnerDecimalSchema::Fixed(FixedSchema { attributes, .. }),
569                ..
570            })
571            | Schema::Uuid(UuidSchema::Fixed(FixedSchema { attributes, .. })) => Some(attributes),
572            Schema::Duration(FixedSchema { attributes, .. }) => Some(attributes),
573            _ => None,
574        }
575    }
576
577    /// Returns whether the schema represents a named type according to the avro specification
578    pub fn is_named(&self) -> bool {
579        matches!(
580            self,
581            Schema::Ref { .. }
582                | Schema::Record(_)
583                | Schema::Enum(_)
584                | Schema::Fixed(_)
585                | Schema::Decimal(DecimalSchema {
586                    inner: InnerDecimalSchema::Fixed(_),
587                    ..
588                })
589                | Schema::Uuid(UuidSchema::Fixed(_))
590                | Schema::Duration(_)
591        )
592    }
593
594    /// Returns the name of the schema if it has one.
595    pub fn name(&self) -> Option<&Name> {
596        match self {
597            Schema::Ref { name, .. }
598            | Schema::Record(RecordSchema { name, .. })
599            | Schema::Enum(EnumSchema { name, .. })
600            | Schema::Fixed(FixedSchema { name, .. })
601            | Schema::Decimal(DecimalSchema {
602                inner: InnerDecimalSchema::Fixed(FixedSchema { name, .. }),
603                ..
604            })
605            | Schema::Uuid(UuidSchema::Fixed(FixedSchema { name, .. }))
606            | Schema::Duration(FixedSchema { name, .. }) => Some(name),
607            _ => None,
608        }
609    }
610
611    /// Returns the namespace of the schema if it has one.
612    pub fn namespace(&self) -> Namespace {
613        self.name().and_then(|n| n.namespace.clone())
614    }
615
616    /// Returns the aliases of the schema if it has ones.
617    pub fn aliases(&self) -> Option<&Vec<Alias>> {
618        match self {
619            Schema::Record(RecordSchema { aliases, .. })
620            | Schema::Enum(EnumSchema { aliases, .. })
621            | Schema::Fixed(FixedSchema { aliases, .. })
622            | Schema::Decimal(DecimalSchema {
623                inner: InnerDecimalSchema::Fixed(FixedSchema { aliases, .. }),
624                ..
625            })
626            | Schema::Uuid(UuidSchema::Fixed(FixedSchema { aliases, .. })) => aliases.as_ref(),
627            Schema::Duration(FixedSchema { aliases, .. }) => aliases.as_ref(),
628            _ => None,
629        }
630    }
631
632    /// Returns the doc of the schema if it has one.
633    pub fn doc(&self) -> Option<&String> {
634        match self {
635            Schema::Record(RecordSchema { doc, .. })
636            | Schema::Enum(EnumSchema { doc, .. })
637            | Schema::Fixed(FixedSchema { doc, .. })
638            | Schema::Decimal(DecimalSchema {
639                inner: InnerDecimalSchema::Fixed(FixedSchema { doc, .. }),
640                ..
641            })
642            | Schema::Uuid(UuidSchema::Fixed(FixedSchema { doc, .. })) => doc.as_ref(),
643            Schema::Duration(FixedSchema { doc, .. }) => doc.as_ref(),
644            _ => None,
645        }
646    }
647
648    /// Remove all external references from the schema.
649    ///
650    /// `schemata` must contain all externally referenced schemas.
651    ///
652    /// # Errors
653    /// Will return a [`Details::SchemaResolutionError`] if it fails to find
654    /// a referenced schema. This will put the schema in a partly denormalized state.
655    pub fn denormalize(&mut self, schemata: &[Schema]) -> AvroResult<()> {
656        self.denormalize_inner(schemata, &mut HashSet::new())
657    }
658
659    fn denormalize_inner(
660        &mut self,
661        schemata: &[Schema],
662        defined_names: &mut HashSet<Name>,
663    ) -> AvroResult<()> {
664        // If this name already exists in this schema we can reference it.
665        // This makes the denormalized form as small as possible and prevent infinite loops for recursive types.
666        if let Some(name) = self.name()
667            && defined_names.contains(name)
668        {
669            *self = Schema::Ref { name: name.clone() };
670            return Ok(());
671        }
672        match self {
673            Schema::Ref { name } => {
674                let replacement_schema = schemata
675                    .iter()
676                    .find(|s| s.name().map(|n| *n == *name).unwrap_or(false));
677                if let Some(schema) = replacement_schema {
678                    let mut denorm = schema.clone();
679                    denorm.denormalize_inner(schemata, defined_names)?;
680                    *self = denorm;
681                } else {
682                    return Err(Details::SchemaResolutionError(name.clone()).into());
683                }
684            }
685            Schema::Record(record_schema) => {
686                defined_names.insert(record_schema.name.clone());
687                for field in &mut record_schema.fields {
688                    field.schema.denormalize_inner(schemata, defined_names)?;
689                }
690            }
691            Schema::Array(array_schema) => {
692                array_schema
693                    .items
694                    .denormalize_inner(schemata, defined_names)?;
695            }
696            Schema::Map(map_schema) => {
697                map_schema
698                    .types
699                    .denormalize_inner(schemata, defined_names)?;
700            }
701            Schema::Union(union_schema) => {
702                for schema in &mut union_schema.schemas {
703                    schema.denormalize_inner(schemata, defined_names)?;
704                }
705            }
706            schema if schema.is_named() => {
707                defined_names.insert(schema.name().expect("Schema is named").clone());
708            }
709            _ => (),
710        }
711        Ok(())
712    }
713}
714
715impl Serialize for Schema {
716    fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
717    where
718        S: Serializer,
719    {
720        match &self {
721            Schema::Ref { name } => serializer.serialize_str(&name.fullname(None)),
722            Schema::Null => serializer.serialize_str("null"),
723            Schema::Boolean => serializer.serialize_str("boolean"),
724            Schema::Int => serializer.serialize_str("int"),
725            Schema::Long => serializer.serialize_str("long"),
726            Schema::Float => serializer.serialize_str("float"),
727            Schema::Double => serializer.serialize_str("double"),
728            Schema::Bytes => serializer.serialize_str("bytes"),
729            Schema::String => serializer.serialize_str("string"),
730            Schema::Array(ArraySchema {
731                items,
732                default,
733                attributes,
734            }) => {
735                let mut map = serializer.serialize_map(Some(
736                    2 + attributes.len() + if default.is_some() { 1 } else { 0 },
737                ))?;
738                map.serialize_entry("type", "array")?;
739                map.serialize_entry("items", items)?;
740                if let Some(default) = default {
741                    let value = JsonValue::try_from(Value::Array(default.clone()))
742                        .map_err(S::Error::custom)?;
743                    map.serialize_entry("default", &value)?;
744                }
745                for (key, value) in attributes {
746                    map.serialize_entry(key, value)?;
747                }
748                map.end()
749            }
750            Schema::Map(MapSchema {
751                types,
752                default,
753                attributes,
754            }) => {
755                let mut map = serializer.serialize_map(Some(
756                    2 + attributes.len() + if default.is_some() { 1 } else { 0 },
757                ))?;
758                map.serialize_entry("type", "map")?;
759                map.serialize_entry("values", types)?;
760                if let Some(default) = default {
761                    let value = JsonValue::try_from(Value::Map(default.clone()))
762                        .map_err(S::Error::custom)?;
763                    map.serialize_entry("default", &value)?;
764                }
765                for (key, value) in attributes {
766                    map.serialize_entry(key, value)?;
767                }
768                map.end()
769            }
770            Schema::Union(inner) => {
771                let variants = inner.variants();
772                let mut seq = serializer.serialize_seq(Some(variants.len()))?;
773                for v in variants {
774                    seq.serialize_element(v)?;
775                }
776                seq.end()
777            }
778            Schema::Record(RecordSchema {
779                name,
780                aliases,
781                doc,
782                fields,
783                attributes,
784                lookup: _lookup,
785            }) => {
786                let mut map = serializer.serialize_map(None)?;
787                map.serialize_entry("type", "record")?;
788                if let Some(ref n) = name.namespace {
789                    map.serialize_entry("namespace", n)?;
790                }
791                map.serialize_entry("name", &name.name)?;
792                if let Some(docstr) = doc {
793                    map.serialize_entry("doc", docstr)?;
794                }
795                if let Some(aliases) = aliases {
796                    map.serialize_entry("aliases", aliases)?;
797                }
798                map.serialize_entry("fields", fields)?;
799                for attr in attributes {
800                    map.serialize_entry(attr.0, attr.1)?;
801                }
802                map.end()
803            }
804            Schema::Enum(EnumSchema {
805                name,
806                symbols,
807                aliases,
808                attributes,
809                default,
810                doc,
811            }) => {
812                let mut map = serializer.serialize_map(None)?;
813                map.serialize_entry("type", "enum")?;
814                if let Some(ref n) = name.namespace {
815                    map.serialize_entry("namespace", n)?;
816                }
817                map.serialize_entry("name", &name.name)?;
818                map.serialize_entry("symbols", symbols)?;
819
820                if let Some(aliases) = aliases {
821                    map.serialize_entry("aliases", aliases)?;
822                }
823                if let Some(default) = default {
824                    map.serialize_entry("default", default)?;
825                }
826                if let Some(doc) = doc {
827                    map.serialize_entry("doc", doc)?;
828                }
829                for attr in attributes {
830                    map.serialize_entry(attr.0, attr.1)?;
831                }
832                map.end()
833            }
834            Schema::Fixed(fixed_schema) => {
835                let mut map = serializer.serialize_map(None)?;
836                map = fixed_schema.serialize_to_map::<S>(map)?;
837                map.end()
838            }
839            Schema::Decimal(DecimalSchema {
840                scale,
841                precision,
842                inner,
843            }) => {
844                let mut map = serializer.serialize_map(None)?;
845                match inner {
846                    InnerDecimalSchema::Fixed(fixed_schema) => {
847                        map = fixed_schema.serialize_to_map::<S>(map)?;
848                    }
849                    InnerDecimalSchema::Bytes => {
850                        map.serialize_entry("type", "bytes")?;
851                    }
852                }
853                map.serialize_entry("logicalType", "decimal")?;
854                map.serialize_entry("scale", scale)?;
855                map.serialize_entry("precision", precision)?;
856                map.end()
857            }
858
859            Schema::BigDecimal => {
860                let mut map = serializer.serialize_map(None)?;
861                map.serialize_entry("type", "bytes")?;
862                map.serialize_entry("logicalType", "big-decimal")?;
863                map.end()
864            }
865            Schema::Uuid(inner) => {
866                let mut map = serializer.serialize_map(None)?;
867                match inner {
868                    UuidSchema::Bytes => {
869                        map.serialize_entry("type", "bytes")?;
870                    }
871                    UuidSchema::String => {
872                        map.serialize_entry("type", "string")?;
873                    }
874                    UuidSchema::Fixed(fixed_schema) => {
875                        map = fixed_schema.serialize_to_map::<S>(map)?;
876                    }
877                }
878                map.serialize_entry("logicalType", "uuid")?;
879                map.end()
880            }
881            Schema::Date => {
882                let mut map = serializer.serialize_map(None)?;
883                map.serialize_entry("type", "int")?;
884                map.serialize_entry("logicalType", "date")?;
885                map.end()
886            }
887            Schema::TimeMillis => {
888                let mut map = serializer.serialize_map(None)?;
889                map.serialize_entry("type", "int")?;
890                map.serialize_entry("logicalType", "time-millis")?;
891                map.end()
892            }
893            Schema::TimeMicros => {
894                let mut map = serializer.serialize_map(None)?;
895                map.serialize_entry("type", "long")?;
896                map.serialize_entry("logicalType", "time-micros")?;
897                map.end()
898            }
899            Schema::TimestampMillis => {
900                let mut map = serializer.serialize_map(None)?;
901                map.serialize_entry("type", "long")?;
902                map.serialize_entry("logicalType", "timestamp-millis")?;
903                map.end()
904            }
905            Schema::TimestampMicros => {
906                let mut map = serializer.serialize_map(None)?;
907                map.serialize_entry("type", "long")?;
908                map.serialize_entry("logicalType", "timestamp-micros")?;
909                map.end()
910            }
911            Schema::TimestampNanos => {
912                let mut map = serializer.serialize_map(None)?;
913                map.serialize_entry("type", "long")?;
914                map.serialize_entry("logicalType", "timestamp-nanos")?;
915                map.end()
916            }
917            Schema::LocalTimestampMillis => {
918                let mut map = serializer.serialize_map(None)?;
919                map.serialize_entry("type", "long")?;
920                map.serialize_entry("logicalType", "local-timestamp-millis")?;
921                map.end()
922            }
923            Schema::LocalTimestampMicros => {
924                let mut map = serializer.serialize_map(None)?;
925                map.serialize_entry("type", "long")?;
926                map.serialize_entry("logicalType", "local-timestamp-micros")?;
927                map.end()
928            }
929            Schema::LocalTimestampNanos => {
930                let mut map = serializer.serialize_map(None)?;
931                map.serialize_entry("type", "long")?;
932                map.serialize_entry("logicalType", "local-timestamp-nanos")?;
933                map.end()
934            }
935            Schema::Duration(fixed) => {
936                let map = serializer.serialize_map(None)?;
937
938                let mut map = fixed.serialize_to_map::<S>(map)?;
939                map.serialize_entry("logicalType", "duration")?;
940                map.end()
941            }
942        }
943    }
944}
945
946/// Parses a valid Avro schema into [the Parsing Canonical Form].
947///
948/// [the Parsing Canonical Form](https://avro.apache.org/docs/++version++/specification/#parsing-canonical-form-for-schemas)
949fn parsing_canonical_form(schema: &JsonValue, defined_names: &mut HashSet<String>) -> String {
950    match schema {
951        JsonValue::Object(map) => pcf_map(map, defined_names),
952        JsonValue::String(s) => pcf_string(s),
953        JsonValue::Array(v) => pcf_array(v, defined_names),
954        json => panic!("got invalid JSON value for canonical form of schema: {json}"),
955    }
956}
957
958fn pcf_map(schema: &Map<String, JsonValue>, defined_names: &mut HashSet<String>) -> String {
959    let typ = schema.get("type").and_then(|v| v.as_str());
960    let name = if is_named_type(typ) {
961        let ns = schema.get("namespace").and_then(|v| v.as_str());
962        let raw_name = schema.get("name").and_then(|v| v.as_str());
963        Some(format!(
964            "{}{}",
965            ns.map_or("".to_string(), |n| { format!("{n}.") }),
966            raw_name.unwrap_or_default()
967        ))
968    } else {
969        None
970    };
971
972    //if this is already a defined type, early return
973    if let Some(ref n) = name {
974        if defined_names.contains(n) {
975            return pcf_string(n);
976        } else {
977            defined_names.insert(n.clone());
978        }
979    }
980
981    let mut fields = Vec::new();
982    for (k, v) in schema {
983        // Reduce primitive types to their simple form. ([PRIMITIVE] rule)
984        if schema.len() == 1 && k == "type" {
985            // Invariant: function is only callable from a valid schema, so this is acceptable.
986            if let JsonValue::String(s) = v {
987                return pcf_string(s);
988            }
989        }
990
991        // Strip out unused fields ([STRIP] rule)
992        if field_ordering_position(k).is_none()
993            || k == "default"
994            || k == "doc"
995            || k == "aliases"
996            || k == "logicalType"
997        {
998            continue;
999        }
1000
1001        // Fully qualify the name, if it isn't already ([FULLNAMES] rule).
1002        if k == "name"
1003            && let Some(ref n) = name
1004        {
1005            fields.push(("name", format!("{}:{}", pcf_string(k), pcf_string(n))));
1006            continue;
1007        }
1008
1009        // Strip off quotes surrounding "size" type, if they exist ([INTEGERS] rule).
1010        if k == "size" || k == "precision" || k == "scale" {
1011            let i = match v.as_str() {
1012                Some(s) => s.parse::<i64>().expect("Only valid schemas are accepted!"),
1013                None => v.as_i64().unwrap(),
1014            };
1015            fields.push((k, format!("{}:{}", pcf_string(k), i)));
1016            continue;
1017        }
1018
1019        // For anything else, recursively process the result.
1020        fields.push((
1021            k,
1022            format!(
1023                "{}:{}",
1024                pcf_string(k),
1025                parsing_canonical_form(v, defined_names)
1026            ),
1027        ));
1028    }
1029
1030    // Sort the fields by their canonical ordering ([ORDER] rule).
1031    fields.sort_unstable_by_key(|(k, _)| field_ordering_position(k).unwrap());
1032    let inter = fields
1033        .into_iter()
1034        .map(|(_, v)| v)
1035        .collect::<Vec<_>>()
1036        .join(",");
1037    format!("{{{inter}}}")
1038}
1039
1040fn is_named_type(typ: Option<&str>) -> bool {
1041    matches!(
1042        typ,
1043        Some("record") | Some("enum") | Some("fixed") | Some("ref")
1044    )
1045}
1046
1047fn pcf_array(arr: &[JsonValue], defined_names: &mut HashSet<String>) -> String {
1048    let inter = arr
1049        .iter()
1050        .map(|a| parsing_canonical_form(a, defined_names))
1051        .collect::<Vec<String>>()
1052        .join(",");
1053    format!("[{inter}]")
1054}
1055
1056fn pcf_string(s: &str) -> String {
1057    format!("\"{s}\"")
1058}
1059
1060const RESERVED_FIELDS: &[&str] = &[
1061    "name",
1062    "type",
1063    "fields",
1064    "symbols",
1065    "items",
1066    "values",
1067    "size",
1068    "logicalType",
1069    "order",
1070    "doc",
1071    "aliases",
1072    "default",
1073    "precision",
1074    "scale",
1075];
1076
1077// Used to define the ordering and inclusion of fields.
1078fn field_ordering_position(field: &str) -> Option<usize> {
1079    RESERVED_FIELDS
1080        .iter()
1081        .position(|&f| f == field)
1082        .map(|pos| pos + 1)
1083}
1084
1085#[cfg(test)]
1086mod tests {
1087    use super::*;
1088    use crate::{error::Details, rabin::Rabin};
1089    use apache_avro_test_helper::{
1090        TestResult,
1091        logger::{assert_logged, assert_not_logged},
1092    };
1093    use serde::{Deserialize, Serialize};
1094    use serde_json::json;
1095
1096    #[test]
1097    fn test_invalid_schema() {
1098        assert!(Schema::parse_str("invalid").is_err());
1099    }
1100
1101    #[test]
1102    fn test_primitive_schema() -> TestResult {
1103        assert_eq!(Schema::Null, Schema::parse_str("\"null\"")?);
1104        assert_eq!(Schema::Int, Schema::parse_str("\"int\"")?);
1105        assert_eq!(Schema::Double, Schema::parse_str("\"double\"")?);
1106        Ok(())
1107    }
1108
1109    #[test]
1110    fn test_array_schema() -> TestResult {
1111        let schema = Schema::parse_str(r#"{"type": "array", "items": "string"}"#)?;
1112        assert_eq!(Schema::array(Schema::String).build(), schema);
1113        Ok(())
1114    }
1115
1116    #[test]
1117    fn test_map_schema() -> TestResult {
1118        let schema = Schema::parse_str(r#"{"type": "map", "values": "double"}"#)?;
1119        assert_eq!(Schema::map(Schema::Double).build(), schema);
1120        Ok(())
1121    }
1122
1123    #[test]
1124    fn test_union_schema() -> TestResult {
1125        let schema = Schema::parse_str(r#"["null", "int"]"#)?;
1126        assert_eq!(
1127            Schema::Union(UnionSchema::new(vec![Schema::Null, Schema::Int])?),
1128            schema
1129        );
1130        Ok(())
1131    }
1132
1133    #[test]
1134    fn test_union_unsupported_schema() {
1135        let schema = Schema::parse_str(r#"["null", ["null", "int"], "string"]"#);
1136        assert!(schema.is_err());
1137    }
1138
1139    #[test]
1140    fn test_multi_union_schema() -> TestResult {
1141        let schema = Schema::parse_str(r#"["null", "int", "float", "string", "bytes"]"#);
1142        assert!(schema.is_ok());
1143        let schema = schema?;
1144        assert_eq!(SchemaKind::from(&schema), SchemaKind::Union);
1145        let union_schema = match schema {
1146            Schema::Union(u) => u,
1147            _ => unreachable!(),
1148        };
1149        assert_eq!(union_schema.variants().len(), 5);
1150        let mut variants = union_schema.variants().iter();
1151        assert_eq!(SchemaKind::from(variants.next().unwrap()), SchemaKind::Null);
1152        assert_eq!(SchemaKind::from(variants.next().unwrap()), SchemaKind::Int);
1153        assert_eq!(
1154            SchemaKind::from(variants.next().unwrap()),
1155            SchemaKind::Float
1156        );
1157        assert_eq!(
1158            SchemaKind::from(variants.next().unwrap()),
1159            SchemaKind::String
1160        );
1161        assert_eq!(
1162            SchemaKind::from(variants.next().unwrap()),
1163            SchemaKind::Bytes
1164        );
1165        assert_eq!(variants.next(), None);
1166
1167        Ok(())
1168    }
1169
1170    // AVRO-3248
1171    #[test]
1172    fn test_union_of_records() -> TestResult {
1173        // A and B are the same except the name.
1174        let schema_str_a = r#"{
1175            "name": "A",
1176            "type": "record",
1177            "fields": [
1178                {"name": "field_one", "type": "float"}
1179            ]
1180        }"#;
1181
1182        let schema_str_b = r#"{
1183            "name": "B",
1184            "type": "record",
1185            "fields": [
1186                {"name": "field_one", "type": "float"}
1187            ]
1188        }"#;
1189
1190        // we get Details::GetNameField if we put ["A", "B"] directly here.
1191        let schema_str_c = r#"{
1192            "name": "C",
1193            "type": "record",
1194            "fields": [
1195                {"name": "field_one",  "type": ["A", "B"]}
1196            ]
1197        }"#;
1198
1199        let schema_c = Schema::parse_list([schema_str_a, schema_str_b, schema_str_c])?
1200            .last()
1201            .unwrap()
1202            .clone();
1203
1204        let schema_c_expected = Schema::Record(
1205            RecordSchema::builder()
1206                .try_name("C")?
1207                .fields(vec![
1208                    RecordField::builder()
1209                        .name("field_one".to_string())
1210                        .schema(Schema::Union(UnionSchema::new(vec![
1211                            Schema::Ref {
1212                                name: Name::new("A")?,
1213                            },
1214                            Schema::Ref {
1215                                name: Name::new("B")?,
1216                            },
1217                        ])?))
1218                        .build(),
1219                ])
1220                .build(),
1221        );
1222
1223        assert_eq!(schema_c, schema_c_expected);
1224        Ok(())
1225    }
1226
1227    #[test]
1228    fn avro_rs_104_test_root_union_of_records() -> TestResult {
1229        // A and B are the same except the name.
1230        let schema_str_a = r#"{
1231            "name": "A",
1232            "type": "record",
1233            "fields": [
1234                {"name": "field_one", "type": "float"}
1235            ]
1236        }"#;
1237
1238        let schema_str_b = r#"{
1239            "name": "B",
1240            "type": "record",
1241            "fields": [
1242                {"name": "field_one", "type": "float"}
1243            ]
1244        }"#;
1245
1246        let schema_str_c = r#"["A", "B"]"#;
1247
1248        let (schema_c, schemata) =
1249            Schema::parse_str_with_list(schema_str_c, [schema_str_a, schema_str_b])?;
1250
1251        let schema_a_expected = Schema::Record(RecordSchema {
1252            name: Name::new("A")?,
1253            aliases: None,
1254            doc: None,
1255            fields: vec![RecordField {
1256                name: "field_one".to_string(),
1257                doc: None,
1258                default: None,
1259                aliases: None,
1260                schema: Schema::Float,
1261                order: RecordFieldOrder::Ignore,
1262                position: 0,
1263                custom_attributes: Default::default(),
1264            }],
1265            lookup: BTreeMap::from_iter(vec![("field_one".to_string(), 0)]),
1266            attributes: Default::default(),
1267        });
1268
1269        let schema_b_expected = Schema::Record(RecordSchema {
1270            name: Name::new("B")?,
1271            aliases: None,
1272            doc: None,
1273            fields: vec![RecordField {
1274                name: "field_one".to_string(),
1275                doc: None,
1276                default: None,
1277                aliases: None,
1278                schema: Schema::Float,
1279                order: RecordFieldOrder::Ignore,
1280                position: 0,
1281                custom_attributes: Default::default(),
1282            }],
1283            lookup: BTreeMap::from_iter(vec![("field_one".to_string(), 0)]),
1284            attributes: Default::default(),
1285        });
1286
1287        let schema_c_expected = Schema::Union(UnionSchema::new(vec![
1288            Schema::Ref {
1289                name: Name::new("A")?,
1290            },
1291            Schema::Ref {
1292                name: Name::new("B")?,
1293            },
1294        ])?);
1295
1296        assert_eq!(schema_c, schema_c_expected);
1297        assert_eq!(schemata[0], schema_a_expected);
1298        assert_eq!(schemata[1], schema_b_expected);
1299
1300        Ok(())
1301    }
1302
1303    #[test]
1304    fn avro_rs_104_test_root_union_of_records_name_collision() -> TestResult {
1305        // A and B are exactly the same.
1306        let schema_str_a1 = r#"{
1307            "name": "A",
1308            "type": "record",
1309            "fields": [
1310                {"name": "field_one", "type": "float"}
1311            ]
1312        }"#;
1313
1314        let schema_str_a2 = r#"{
1315            "name": "A",
1316            "type": "record",
1317            "fields": [
1318                {"name": "field_one", "type": "float"}
1319            ]
1320        }"#;
1321
1322        let schema_str_c = r#"["A", "A"]"#;
1323
1324        match Schema::parse_str_with_list(schema_str_c, [schema_str_a1, schema_str_a2]) {
1325            Ok(_) => unreachable!("Expected an error that the name is already defined"),
1326            Err(e) => assert_eq!(
1327                e.to_string(),
1328                "Two schemas with the same fullname were given: \"A\""
1329            ),
1330        }
1331
1332        Ok(())
1333    }
1334
1335    #[test]
1336    fn avro_rs_104_test_root_union_of_records_no_name() -> TestResult {
1337        let schema_str_a = r#"{
1338            "name": "A",
1339            "type": "record",
1340            "fields": [
1341                {"name": "field_one", "type": "float"}
1342            ]
1343        }"#;
1344
1345        // B has no name field.
1346        let schema_str_b = r#"{
1347            "type": "record",
1348            "fields": [
1349                {"name": "field_one", "type": "float"}
1350            ]
1351        }"#;
1352
1353        let schema_str_c = r#"["A", "A"]"#;
1354
1355        match Schema::parse_str_with_list(schema_str_c, [schema_str_a, schema_str_b]) {
1356            Ok(_) => unreachable!("Expected an error that schema_str_b is missing a name field"),
1357            Err(e) => assert_eq!(e.to_string(), "No `name` field"),
1358        }
1359
1360        Ok(())
1361    }
1362
1363    #[test]
1364    fn avro_3584_test_recursion_records() -> TestResult {
1365        // A and B are the same except the name.
1366        let schema_str_a = r#"{
1367            "name": "A",
1368            "type": "record",
1369            "fields": [ {"name": "field_one", "type": "B"} ]
1370        }"#;
1371
1372        let schema_str_b = r#"{
1373            "name": "B",
1374            "type": "record",
1375            "fields": [ {"name": "field_one", "type": "A"} ]
1376        }"#;
1377
1378        let list = Schema::parse_list([schema_str_a, schema_str_b])?;
1379
1380        let schema_a = list.first().unwrap().clone();
1381
1382        match schema_a {
1383            Schema::Record(RecordSchema { fields, .. }) => {
1384                let f1 = fields.first();
1385
1386                let ref_schema = Schema::Ref {
1387                    name: Name::new("B")?,
1388                };
1389                assert_eq!(ref_schema, f1.unwrap().schema);
1390            }
1391            _ => panic!("Expected a record schema!"),
1392        }
1393
1394        Ok(())
1395    }
1396
1397    #[test]
1398    fn test_avro_3248_nullable_record() -> TestResult {
1399        use std::iter::FromIterator;
1400
1401        let schema_str_a = r#"{
1402            "name": "A",
1403            "type": "record",
1404            "fields": [
1405                {"name": "field_one", "type": "float"}
1406            ]
1407        }"#;
1408
1409        // we get Details::GetNameField if we put ["null", "B"] directly here.
1410        let schema_str_option_a = r#"{
1411            "name": "OptionA",
1412            "type": "record",
1413            "fields": [
1414                {"name": "field_one",  "type": ["null", "A"], "default": null}
1415            ]
1416        }"#;
1417
1418        let schema_option_a = Schema::parse_list([schema_str_a, schema_str_option_a])?
1419            .last()
1420            .unwrap()
1421            .clone();
1422
1423        let schema_option_a_expected = Schema::Record(RecordSchema {
1424            name: Name::new("OptionA")?,
1425            aliases: None,
1426            doc: None,
1427            fields: vec![RecordField {
1428                name: "field_one".to_string(),
1429                doc: None,
1430                default: Some(JsonValue::Null),
1431                aliases: None,
1432                schema: Schema::Union(UnionSchema::new(vec![
1433                    Schema::Null,
1434                    Schema::Ref {
1435                        name: Name::new("A")?,
1436                    },
1437                ])?),
1438                order: RecordFieldOrder::Ignore,
1439                position: 0,
1440                custom_attributes: Default::default(),
1441            }],
1442            lookup: BTreeMap::from_iter(vec![("field_one".to_string(), 0)]),
1443            attributes: Default::default(),
1444        });
1445
1446        assert_eq!(schema_option_a, schema_option_a_expected);
1447
1448        Ok(())
1449    }
1450
1451    #[test]
1452    fn test_record_schema() -> TestResult {
1453        let parsed = Schema::parse_str(
1454            r#"
1455            {
1456                "type": "record",
1457                "name": "test",
1458                "fields": [
1459                    {"name": "a", "type": "long", "default": 42},
1460                    {"name": "b", "type": "string"}
1461                ]
1462            }
1463        "#,
1464        )?;
1465
1466        let mut lookup = BTreeMap::new();
1467        lookup.insert("a".to_owned(), 0);
1468        lookup.insert("b".to_owned(), 1);
1469
1470        let expected = Schema::Record(RecordSchema {
1471            name: Name::new("test")?,
1472            aliases: None,
1473            doc: None,
1474            fields: vec![
1475                RecordField {
1476                    name: "a".to_string(),
1477                    doc: None,
1478                    default: Some(JsonValue::Number(42i64.into())),
1479                    aliases: None,
1480                    schema: Schema::Long,
1481                    order: RecordFieldOrder::Ascending,
1482                    position: 0,
1483                    custom_attributes: Default::default(),
1484                },
1485                RecordField {
1486                    name: "b".to_string(),
1487                    doc: None,
1488                    default: None,
1489                    aliases: None,
1490                    schema: Schema::String,
1491                    order: RecordFieldOrder::Ascending,
1492                    position: 1,
1493                    custom_attributes: Default::default(),
1494                },
1495            ],
1496            lookup,
1497            attributes: Default::default(),
1498        });
1499
1500        assert_eq!(parsed, expected);
1501
1502        Ok(())
1503    }
1504
1505    #[test]
1506    fn test_avro_3302_record_schema_with_currently_parsing_schema() -> TestResult {
1507        let schema = Schema::parse_str(
1508            r#"
1509            {
1510                "type": "record",
1511                "name": "test",
1512                "fields": [{
1513                    "name": "recordField",
1514                    "type": {
1515                        "type": "record",
1516                        "name": "Node",
1517                        "fields": [
1518                            {"name": "label", "type": "string"},
1519                            {"name": "children", "type": {"type": "array", "items": "Node"}}
1520                        ]
1521                    }
1522                }]
1523            }
1524        "#,
1525        )?;
1526
1527        let mut lookup = BTreeMap::new();
1528        lookup.insert("recordField".to_owned(), 0);
1529
1530        let mut node_lookup = BTreeMap::new();
1531        node_lookup.insert("children".to_owned(), 1);
1532        node_lookup.insert("label".to_owned(), 0);
1533
1534        let expected = Schema::Record(RecordSchema {
1535            name: Name::new("test")?,
1536            aliases: None,
1537            doc: None,
1538            fields: vec![RecordField {
1539                name: "recordField".to_string(),
1540                doc: None,
1541                default: None,
1542                aliases: None,
1543                schema: Schema::Record(RecordSchema {
1544                    name: Name::new("Node")?,
1545                    aliases: None,
1546                    doc: None,
1547                    fields: vec![
1548                        RecordField {
1549                            name: "label".to_string(),
1550                            doc: None,
1551                            default: None,
1552                            aliases: None,
1553                            schema: Schema::String,
1554                            order: RecordFieldOrder::Ascending,
1555                            position: 0,
1556                            custom_attributes: Default::default(),
1557                        },
1558                        RecordField {
1559                            name: "children".to_string(),
1560                            doc: None,
1561                            default: None,
1562                            aliases: None,
1563                            schema: Schema::array(Schema::Ref {
1564                                name: Name::new("Node")?,
1565                            })
1566                            .build(),
1567                            order: RecordFieldOrder::Ascending,
1568                            position: 1,
1569                            custom_attributes: Default::default(),
1570                        },
1571                    ],
1572                    lookup: node_lookup,
1573                    attributes: Default::default(),
1574                }),
1575                order: RecordFieldOrder::Ascending,
1576                position: 0,
1577                custom_attributes: Default::default(),
1578            }],
1579            lookup,
1580            attributes: Default::default(),
1581        });
1582        assert_eq!(schema, expected);
1583
1584        let canonical_form = &schema.canonical_form();
1585        let expected = r#"{"name":"test","type":"record","fields":[{"name":"recordField","type":{"name":"Node","type":"record","fields":[{"name":"label","type":"string"},{"name":"children","type":{"type":"array","items":"Node"}}]}}]}"#;
1586        assert_eq!(canonical_form, &expected);
1587
1588        Ok(())
1589    }
1590
1591    // https://github.com/flavray/avro-rs/pull/99#issuecomment-1016948451
1592    #[test]
1593    fn test_parsing_of_recursive_type_enum() -> TestResult {
1594        let schema = r#"
1595    {
1596        "type": "record",
1597        "name": "User",
1598        "namespace": "office",
1599        "fields": [
1600            {
1601              "name": "details",
1602              "type": [
1603                {
1604                  "type": "record",
1605                  "name": "Employee",
1606                  "fields": [
1607                    {
1608                      "name": "gender",
1609                      "type": {
1610                        "type": "enum",
1611                        "name": "Gender",
1612                        "symbols": [
1613                          "male",
1614                          "female"
1615                        ]
1616                      },
1617                      "default": "female"
1618                    }
1619                  ]
1620                },
1621                {
1622                  "type": "record",
1623                  "name": "Manager",
1624                  "fields": [
1625                    {
1626                      "name": "gender",
1627                      "type": "Gender"
1628                    }
1629                  ]
1630                }
1631              ]
1632            }
1633          ]
1634        }
1635        "#;
1636
1637        let schema = Schema::parse_str(schema)?;
1638        let schema_str = schema.canonical_form();
1639        let expected = r#"{"name":"office.User","type":"record","fields":[{"name":"details","type":[{"name":"office.Employee","type":"record","fields":[{"name":"gender","type":{"name":"office.Gender","type":"enum","symbols":["male","female"]}}]},{"name":"office.Manager","type":"record","fields":[{"name":"gender","type":"office.Gender"}]}]}]}"#;
1640        assert_eq!(schema_str, expected);
1641
1642        Ok(())
1643    }
1644
1645    #[test]
1646    fn test_parsing_of_recursive_type_fixed() -> TestResult {
1647        let schema = r#"
1648    {
1649        "type": "record",
1650        "name": "User",
1651        "namespace": "office",
1652        "fields": [
1653            {
1654              "name": "details",
1655              "type": [
1656                {
1657                  "type": "record",
1658                  "name": "Employee",
1659                  "fields": [
1660                    {
1661                      "name": "id",
1662                      "type": {
1663                        "type": "fixed",
1664                        "name": "EmployeeId",
1665                        "size": 16
1666                      },
1667                      "default": "female"
1668                    }
1669                  ]
1670                },
1671                {
1672                  "type": "record",
1673                  "name": "Manager",
1674                  "fields": [
1675                    {
1676                      "name": "id",
1677                      "type": "EmployeeId"
1678                    }
1679                  ]
1680                }
1681              ]
1682            }
1683          ]
1684        }
1685        "#;
1686
1687        let schema = Schema::parse_str(schema)?;
1688        let schema_str = schema.canonical_form();
1689        let expected = r#"{"name":"office.User","type":"record","fields":[{"name":"details","type":[{"name":"office.Employee","type":"record","fields":[{"name":"id","type":{"name":"office.EmployeeId","type":"fixed","size":16}}]},{"name":"office.Manager","type":"record","fields":[{"name":"id","type":"office.EmployeeId"}]}]}]}"#;
1690        assert_eq!(schema_str, expected);
1691
1692        Ok(())
1693    }
1694
1695    #[test]
1696    fn test_avro_3302_record_schema_with_currently_parsing_schema_aliases() -> TestResult {
1697        let schema = Schema::parse_str(
1698            r#"
1699            {
1700              "type": "record",
1701              "name": "LongList",
1702              "aliases": ["LinkedLongs"],
1703              "fields" : [
1704                {"name": "value", "type": "long"},
1705                {"name": "next", "type": ["null", "LinkedLongs"]}
1706              ]
1707            }
1708        "#,
1709        )?;
1710
1711        let mut lookup = BTreeMap::new();
1712        lookup.insert("value".to_owned(), 0);
1713        lookup.insert("next".to_owned(), 1);
1714
1715        let expected = Schema::Record(RecordSchema {
1716            name: Name {
1717                name: "LongList".to_owned(),
1718                namespace: None,
1719            },
1720            aliases: Some(vec![Alias::new("LinkedLongs").unwrap()]),
1721            doc: None,
1722            fields: vec![
1723                RecordField {
1724                    name: "value".to_string(),
1725                    doc: None,
1726                    default: None,
1727                    aliases: None,
1728                    schema: Schema::Long,
1729                    order: RecordFieldOrder::Ascending,
1730                    position: 0,
1731                    custom_attributes: Default::default(),
1732                },
1733                RecordField {
1734                    name: "next".to_string(),
1735                    doc: None,
1736                    default: None,
1737                    aliases: None,
1738                    schema: Schema::Union(UnionSchema::new(vec![
1739                        Schema::Null,
1740                        Schema::Ref {
1741                            name: Name {
1742                                name: "LongList".to_owned(),
1743                                namespace: None,
1744                            },
1745                        },
1746                    ])?),
1747                    order: RecordFieldOrder::Ascending,
1748                    position: 1,
1749                    custom_attributes: Default::default(),
1750                },
1751            ],
1752            lookup,
1753            attributes: Default::default(),
1754        });
1755        assert_eq!(schema, expected);
1756
1757        let canonical_form = &schema.canonical_form();
1758        let expected = r#"{"name":"LongList","type":"record","fields":[{"name":"value","type":"long"},{"name":"next","type":["null","LongList"]}]}"#;
1759        assert_eq!(canonical_form, &expected);
1760
1761        Ok(())
1762    }
1763
1764    #[test]
1765    fn test_avro_3370_record_schema_with_currently_parsing_schema_named_record() -> TestResult {
1766        let schema = Schema::parse_str(
1767            r#"
1768            {
1769              "type" : "record",
1770              "name" : "record",
1771              "fields" : [
1772                 { "name" : "value", "type" : "long" },
1773                 { "name" : "next", "type" : "record" }
1774             ]
1775            }
1776        "#,
1777        )?;
1778
1779        let mut lookup = BTreeMap::new();
1780        lookup.insert("value".to_owned(), 0);
1781        lookup.insert("next".to_owned(), 1);
1782
1783        let expected = Schema::Record(RecordSchema {
1784            name: Name {
1785                name: "record".to_owned(),
1786                namespace: None,
1787            },
1788            aliases: None,
1789            doc: None,
1790            fields: vec![
1791                RecordField {
1792                    name: "value".to_string(),
1793                    doc: None,
1794                    default: None,
1795                    aliases: None,
1796                    schema: Schema::Long,
1797                    order: RecordFieldOrder::Ascending,
1798                    position: 0,
1799                    custom_attributes: Default::default(),
1800                },
1801                RecordField {
1802                    name: "next".to_string(),
1803                    doc: None,
1804                    default: None,
1805                    aliases: None,
1806                    schema: Schema::Ref {
1807                        name: Name {
1808                            name: "record".to_owned(),
1809                            namespace: None,
1810                        },
1811                    },
1812                    order: RecordFieldOrder::Ascending,
1813                    position: 1,
1814                    custom_attributes: Default::default(),
1815                },
1816            ],
1817            lookup,
1818            attributes: Default::default(),
1819        });
1820        assert_eq!(schema, expected);
1821
1822        let canonical_form = &schema.canonical_form();
1823        let expected = r#"{"name":"record","type":"record","fields":[{"name":"value","type":"long"},{"name":"next","type":"record"}]}"#;
1824        assert_eq!(canonical_form, &expected);
1825
1826        Ok(())
1827    }
1828
1829    #[test]
1830    fn test_avro_3370_record_schema_with_currently_parsing_schema_named_enum() -> TestResult {
1831        let schema = Schema::parse_str(
1832            r#"
1833            {
1834              "type" : "record",
1835              "name" : "record",
1836              "fields" : [
1837                 {
1838                    "type" : "enum",
1839                    "name" : "enum",
1840                    "symbols": ["one", "two", "three"]
1841                 },
1842                 { "name" : "next", "type" : "enum" }
1843             ]
1844            }
1845        "#,
1846        )?;
1847
1848        let mut lookup = BTreeMap::new();
1849        lookup.insert("enum".to_owned(), 0);
1850        lookup.insert("next".to_owned(), 1);
1851
1852        let expected = Schema::Record(RecordSchema {
1853            name: Name {
1854                name: "record".to_owned(),
1855                namespace: None,
1856            },
1857            aliases: None,
1858            doc: None,
1859            fields: vec![
1860                RecordField {
1861                    name: "enum".to_string(),
1862                    doc: None,
1863                    default: None,
1864                    aliases: None,
1865                    schema: Schema::Enum(
1866                        EnumSchema::builder()
1867                            .name(Name::new("enum")?)
1868                            .symbols(vec![
1869                                "one".to_string(),
1870                                "two".to_string(),
1871                                "three".to_string(),
1872                            ])
1873                            .build(),
1874                    ),
1875                    order: RecordFieldOrder::Ascending,
1876                    position: 0,
1877                    custom_attributes: Default::default(),
1878                },
1879                RecordField {
1880                    name: "next".to_string(),
1881                    doc: None,
1882                    default: None,
1883                    aliases: None,
1884                    schema: Schema::Enum(EnumSchema {
1885                        name: Name {
1886                            name: "enum".to_owned(),
1887                            namespace: None,
1888                        },
1889                        aliases: None,
1890                        doc: None,
1891                        symbols: vec!["one".to_string(), "two".to_string(), "three".to_string()],
1892                        default: None,
1893                        attributes: Default::default(),
1894                    }),
1895                    order: RecordFieldOrder::Ascending,
1896                    position: 1,
1897                    custom_attributes: Default::default(),
1898                },
1899            ],
1900            lookup,
1901            attributes: Default::default(),
1902        });
1903        assert_eq!(schema, expected);
1904
1905        let canonical_form = &schema.canonical_form();
1906        let expected = r#"{"name":"record","type":"record","fields":[{"name":"enum","type":{"name":"enum","type":"enum","symbols":["one","two","three"]}},{"name":"next","type":"enum"}]}"#;
1907        assert_eq!(canonical_form, &expected);
1908
1909        Ok(())
1910    }
1911
1912    #[test]
1913    fn test_avro_3370_record_schema_with_currently_parsing_schema_named_fixed() -> TestResult {
1914        let schema = Schema::parse_str(
1915            r#"
1916            {
1917              "type" : "record",
1918              "name" : "record",
1919              "fields" : [
1920                 {
1921                    "type" : "fixed",
1922                    "name" : "fixed",
1923                    "size": 456
1924                 },
1925                 { "name" : "next", "type" : "fixed" }
1926             ]
1927            }
1928        "#,
1929        )?;
1930
1931        let mut lookup = BTreeMap::new();
1932        lookup.insert("fixed".to_owned(), 0);
1933        lookup.insert("next".to_owned(), 1);
1934
1935        let expected = Schema::Record(RecordSchema {
1936            name: Name {
1937                name: "record".to_owned(),
1938                namespace: None,
1939            },
1940            aliases: None,
1941            doc: None,
1942            fields: vec![
1943                RecordField {
1944                    name: "fixed".to_string(),
1945                    doc: None,
1946                    default: None,
1947                    aliases: None,
1948                    schema: Schema::Fixed(FixedSchema {
1949                        name: Name {
1950                            name: "fixed".to_owned(),
1951                            namespace: None,
1952                        },
1953                        aliases: None,
1954                        doc: None,
1955                        size: 456,
1956                        attributes: Default::default(),
1957                    }),
1958                    order: RecordFieldOrder::Ascending,
1959                    position: 0,
1960                    custom_attributes: Default::default(),
1961                },
1962                RecordField {
1963                    name: "next".to_string(),
1964                    doc: None,
1965                    default: None,
1966                    aliases: None,
1967                    schema: Schema::Fixed(FixedSchema {
1968                        name: Name {
1969                            name: "fixed".to_owned(),
1970                            namespace: None,
1971                        },
1972                        aliases: None,
1973                        doc: None,
1974                        size: 456,
1975                        attributes: Default::default(),
1976                    }),
1977                    order: RecordFieldOrder::Ascending,
1978                    position: 1,
1979                    custom_attributes: Default::default(),
1980                },
1981            ],
1982            lookup,
1983            attributes: Default::default(),
1984        });
1985        assert_eq!(schema, expected);
1986
1987        let canonical_form = &schema.canonical_form();
1988        let expected = r#"{"name":"record","type":"record","fields":[{"name":"fixed","type":{"name":"fixed","type":"fixed","size":456}},{"name":"next","type":"fixed"}]}"#;
1989        assert_eq!(canonical_form, &expected);
1990
1991        Ok(())
1992    }
1993
1994    #[test]
1995    fn test_enum_schema() -> TestResult {
1996        let schema = Schema::parse_str(
1997            r#"{"type": "enum", "name": "Suit", "symbols": ["diamonds", "spades", "clubs", "hearts"]}"#,
1998        )?;
1999
2000        let expected = Schema::Enum(EnumSchema {
2001            name: Name::new("Suit")?,
2002            aliases: None,
2003            doc: None,
2004            symbols: vec![
2005                "diamonds".to_owned(),
2006                "spades".to_owned(),
2007                "clubs".to_owned(),
2008                "hearts".to_owned(),
2009            ],
2010            default: None,
2011            attributes: Default::default(),
2012        });
2013
2014        assert_eq!(expected, schema);
2015
2016        Ok(())
2017    }
2018
2019    #[test]
2020    fn test_enum_schema_duplicate() -> TestResult {
2021        // Duplicate "diamonds"
2022        let schema = Schema::parse_str(
2023            r#"{"type": "enum", "name": "Suit", "symbols": ["diamonds", "spades", "clubs", "diamonds"]}"#,
2024        );
2025        assert!(schema.is_err());
2026
2027        Ok(())
2028    }
2029
2030    #[test]
2031    fn test_enum_schema_name() -> TestResult {
2032        // Invalid name "0000" does not match [A-Za-z_][A-Za-z0-9_]*
2033        let schema = Schema::parse_str(
2034            r#"{"type": "enum", "name": "Enum", "symbols": ["0000", "variant"]}"#,
2035        );
2036        assert!(schema.is_err());
2037
2038        Ok(())
2039    }
2040
2041    #[test]
2042    fn test_fixed_schema() -> TestResult {
2043        let schema = Schema::parse_str(r#"{"type": "fixed", "name": "test", "size": 16}"#)?;
2044
2045        let expected = Schema::Fixed(FixedSchema {
2046            name: Name::new("test")?,
2047            aliases: None,
2048            doc: None,
2049            size: 16_usize,
2050            attributes: Default::default(),
2051        });
2052
2053        assert_eq!(expected, schema);
2054
2055        Ok(())
2056    }
2057
2058    #[test]
2059    fn test_fixed_schema_with_documentation() -> TestResult {
2060        let schema = Schema::parse_str(
2061            r#"{"type": "fixed", "name": "test", "size": 16, "doc": "FixedSchema documentation"}"#,
2062        )?;
2063
2064        let expected = Schema::Fixed(FixedSchema {
2065            name: Name::new("test")?,
2066            aliases: None,
2067            doc: Some(String::from("FixedSchema documentation")),
2068            size: 16_usize,
2069            attributes: Default::default(),
2070        });
2071
2072        assert_eq!(expected, schema);
2073
2074        Ok(())
2075    }
2076
2077    #[test]
2078    fn test_no_documentation() -> TestResult {
2079        let schema = Schema::parse_str(
2080            r#"{"type": "enum", "name": "Coin", "symbols": ["heads", "tails"]}"#,
2081        )?;
2082
2083        let doc = match schema {
2084            Schema::Enum(EnumSchema { doc, .. }) => doc,
2085            _ => unreachable!(),
2086        };
2087
2088        assert!(doc.is_none());
2089
2090        Ok(())
2091    }
2092
2093    #[test]
2094    fn test_documentation() -> TestResult {
2095        let schema = Schema::parse_str(
2096            r#"{"type": "enum", "name": "Coin", "doc": "Some documentation", "symbols": ["heads", "tails"]}"#,
2097        )?;
2098
2099        let doc = match schema {
2100            Schema::Enum(EnumSchema { doc, .. }) => doc,
2101            _ => None,
2102        };
2103
2104        assert_eq!("Some documentation".to_owned(), doc.unwrap());
2105
2106        Ok(())
2107    }
2108
2109    // Tests to ensure Schema is Send + Sync. These tests don't need to _do_ anything, if they can
2110    // compile, they pass.
2111    #[test]
2112    fn test_schema_is_send() {
2113        fn send<S: Send>(_s: S) {}
2114
2115        let schema = Schema::Null;
2116        send(schema);
2117    }
2118
2119    #[test]
2120    fn test_schema_is_sync() {
2121        fn sync<S: Sync>(_s: S) {}
2122
2123        let schema = Schema::Null;
2124        sync(&schema);
2125        sync(schema);
2126    }
2127
2128    #[test]
2129    fn test_schema_fingerprint() -> TestResult {
2130        use crate::rabin::Rabin;
2131        use md5::Md5;
2132        use sha2::Sha256;
2133
2134        let raw_schema = r#"
2135    {
2136        "type": "record",
2137        "name": "test",
2138        "fields": [
2139            {"name": "a", "type": "long", "default": 42},
2140            {"name": "b", "type": "string"},
2141            {"name": "c", "type": "long", "logicalType": "timestamp-micros"}
2142        ]
2143    }
2144"#;
2145
2146        let schema = Schema::parse_str(raw_schema)?;
2147        assert_eq!(
2148            "7eb3b28d73dfc99bdd9af1848298b40804a2f8ad5d2642be2ecc2ad34842b987",
2149            format!("{}", schema.fingerprint::<Sha256>())
2150        );
2151
2152        assert_eq!(
2153            "cb11615e412ee5d872620d8df78ff6ae",
2154            format!("{}", schema.fingerprint::<Md5>())
2155        );
2156        assert_eq!(
2157            "92f2ccef718c6754",
2158            format!("{}", schema.fingerprint::<Rabin>())
2159        );
2160
2161        Ok(())
2162    }
2163
2164    #[test]
2165    fn test_logical_types() -> TestResult {
2166        let schema = Schema::parse_str(r#"{"type": "int", "logicalType": "date"}"#)?;
2167        assert_eq!(schema, Schema::Date);
2168
2169        let schema = Schema::parse_str(r#"{"type": "long", "logicalType": "timestamp-micros"}"#)?;
2170        assert_eq!(schema, Schema::TimestampMicros);
2171
2172        Ok(())
2173    }
2174
2175    #[test]
2176    fn test_nullable_logical_type() -> TestResult {
2177        let schema = Schema::parse_str(
2178            r#"{"type": ["null", {"type": "long", "logicalType": "timestamp-micros"}]}"#,
2179        )?;
2180        assert_eq!(
2181            schema,
2182            Schema::Union(UnionSchema::new(vec![
2183                Schema::Null,
2184                Schema::TimestampMicros,
2185            ])?)
2186        );
2187
2188        Ok(())
2189    }
2190
2191    #[test]
2192    fn record_field_order_from_str() -> TestResult {
2193        use std::str::FromStr;
2194
2195        assert_eq!(
2196            RecordFieldOrder::from_str("ascending").unwrap(),
2197            RecordFieldOrder::Ascending
2198        );
2199        assert_eq!(
2200            RecordFieldOrder::from_str("descending").unwrap(),
2201            RecordFieldOrder::Descending
2202        );
2203        assert_eq!(
2204            RecordFieldOrder::from_str("ignore").unwrap(),
2205            RecordFieldOrder::Ignore
2206        );
2207        assert!(RecordFieldOrder::from_str("not an ordering").is_err());
2208
2209        Ok(())
2210    }
2211
2212    #[test]
2213    fn test_avro_3374_preserve_namespace_for_primitive() -> TestResult {
2214        let schema = Schema::parse_str(
2215            r#"
2216            {
2217              "type" : "record",
2218              "name" : "ns.int",
2219              "fields" : [
2220                {"name" : "value", "type" : "int"},
2221                {"name" : "next", "type" : [ "null", "ns.int" ]}
2222              ]
2223            }
2224            "#,
2225        )?;
2226
2227        let json = schema.canonical_form();
2228        assert_eq!(
2229            json,
2230            r#"{"name":"ns.int","type":"record","fields":[{"name":"value","type":"int"},{"name":"next","type":["null","ns.int"]}]}"#
2231        );
2232
2233        Ok(())
2234    }
2235
2236    #[test]
2237    fn test_avro_3433_preserve_schema_refs_in_json() -> TestResult {
2238        let schema = r#"
2239    {
2240      "name": "test.test",
2241      "type": "record",
2242      "fields": [
2243        {
2244          "name": "bar",
2245          "type": { "name": "test.foo", "type": "record", "fields": [{ "name": "id", "type": "long" }] }
2246        },
2247        { "name": "baz", "type": "test.foo" }
2248      ]
2249    }
2250    "#;
2251
2252        let schema = Schema::parse_str(schema)?;
2253
2254        let expected = r#"{"name":"test.test","type":"record","fields":[{"name":"bar","type":{"name":"test.foo","type":"record","fields":[{"name":"id","type":"long"}]}},{"name":"baz","type":"test.foo"}]}"#;
2255        assert_eq!(schema.canonical_form(), expected);
2256
2257        Ok(())
2258    }
2259
2260    #[test]
2261    fn test_read_namespace_from_name() -> TestResult {
2262        let schema = r#"
2263    {
2264      "name": "space.name",
2265      "type": "record",
2266      "fields": [
2267        {
2268          "name": "num",
2269          "type": "int"
2270        }
2271      ]
2272    }
2273    "#;
2274
2275        let schema = Schema::parse_str(schema)?;
2276        if let Schema::Record(RecordSchema { name, .. }) = schema {
2277            assert_eq!(name.name, "name");
2278            assert_eq!(name.namespace, Some("space".to_string()));
2279        } else {
2280            panic!("Expected a record schema!");
2281        }
2282
2283        Ok(())
2284    }
2285
2286    #[test]
2287    fn test_namespace_from_name_has_priority_over_from_field() -> TestResult {
2288        let schema = r#"
2289    {
2290      "name": "space1.name",
2291      "namespace": "space2",
2292      "type": "record",
2293      "fields": [
2294        {
2295          "name": "num",
2296          "type": "int"
2297        }
2298      ]
2299    }
2300    "#;
2301
2302        let schema = Schema::parse_str(schema)?;
2303        if let Schema::Record(RecordSchema { name, .. }) = schema {
2304            assert_eq!(name.namespace, Some("space1".to_string()));
2305        } else {
2306            panic!("Expected a record schema!");
2307        }
2308
2309        Ok(())
2310    }
2311
2312    #[test]
2313    fn test_namespace_from_field() -> TestResult {
2314        let schema = r#"
2315    {
2316      "name": "name",
2317      "namespace": "space2",
2318      "type": "record",
2319      "fields": [
2320        {
2321          "name": "num",
2322          "type": "int"
2323        }
2324      ]
2325    }
2326    "#;
2327
2328        let schema = Schema::parse_str(schema)?;
2329        if let Schema::Record(RecordSchema { name, .. }) = schema {
2330            assert_eq!(name.namespace, Some("space2".to_string()));
2331        } else {
2332            panic!("Expected a record schema!");
2333        }
2334
2335        Ok(())
2336    }
2337
2338    fn assert_avro_3512_aliases(aliases: &Aliases) {
2339        match aliases {
2340            Some(aliases) => {
2341                assert_eq!(aliases.len(), 3);
2342                assert_eq!(aliases[0], Alias::new("space.b").unwrap());
2343                assert_eq!(aliases[1], Alias::new("x.y").unwrap());
2344                assert_eq!(aliases[2], Alias::new(".c").unwrap());
2345            }
2346            None => {
2347                panic!("'aliases' must be Some");
2348            }
2349        }
2350    }
2351
2352    #[test]
2353    fn avro_3512_alias_with_null_namespace_record() -> TestResult {
2354        let schema = Schema::parse_str(
2355            r#"
2356            {
2357              "type": "record",
2358              "name": "a",
2359              "namespace": "space",
2360              "aliases": ["b", "x.y", ".c"],
2361              "fields" : [
2362                {"name": "time", "type": "long"}
2363              ]
2364            }
2365        "#,
2366        )?;
2367
2368        if let Schema::Record(RecordSchema { ref aliases, .. }) = schema {
2369            assert_avro_3512_aliases(aliases);
2370        } else {
2371            panic!("The Schema should be a record: {schema:?}");
2372        }
2373
2374        Ok(())
2375    }
2376
2377    #[test]
2378    fn avro_3512_alias_with_null_namespace_enum() -> TestResult {
2379        let schema = Schema::parse_str(
2380            r#"
2381            {
2382              "type": "enum",
2383              "name": "a",
2384              "namespace": "space",
2385              "aliases": ["b", "x.y", ".c"],
2386              "symbols" : [
2387                "symbol1", "symbol2"
2388              ]
2389            }
2390        "#,
2391        )?;
2392
2393        if let Schema::Enum(EnumSchema { ref aliases, .. }) = schema {
2394            assert_avro_3512_aliases(aliases);
2395        } else {
2396            panic!("The Schema should be an enum: {schema:?}");
2397        }
2398
2399        Ok(())
2400    }
2401
2402    #[test]
2403    fn avro_3512_alias_with_null_namespace_fixed() -> TestResult {
2404        let schema = Schema::parse_str(
2405            r#"
2406            {
2407              "type": "fixed",
2408              "name": "a",
2409              "namespace": "space",
2410              "aliases": ["b", "x.y", ".c"],
2411              "size" : 12
2412            }
2413        "#,
2414        )?;
2415
2416        if let Schema::Fixed(FixedSchema { ref aliases, .. }) = schema {
2417            assert_avro_3512_aliases(aliases);
2418        } else {
2419            panic!("The Schema should be a fixed: {schema:?}");
2420        }
2421
2422        Ok(())
2423    }
2424
2425    #[test]
2426    fn avro_3518_serialize_aliases_record() -> TestResult {
2427        let schema = Schema::parse_str(
2428            r#"
2429            {
2430              "type": "record",
2431              "name": "a",
2432              "namespace": "space",
2433              "aliases": ["b", "x.y", ".c"],
2434              "fields" : [
2435                {
2436                    "name": "time",
2437                    "type": "long",
2438                    "doc": "The documentation is not serialized",
2439                    "default": 123,
2440                    "aliases": ["time1", "ns.time2"]
2441                }
2442              ]
2443            }
2444        "#,
2445        )?;
2446
2447        let value = serde_json::to_value(&schema)?;
2448        let serialized = serde_json::to_string(&value)?;
2449        assert_eq!(
2450            r#"{"aliases":["space.b","x.y","c"],"fields":[{"aliases":["time1","ns.time2"],"default":123,"name":"time","type":"long"}],"name":"a","namespace":"space","type":"record"}"#,
2451            &serialized
2452        );
2453        assert_eq!(schema, Schema::parse_str(&serialized)?);
2454
2455        Ok(())
2456    }
2457
2458    #[test]
2459    fn avro_3518_serialize_aliases_enum() -> TestResult {
2460        let schema = Schema::parse_str(
2461            r#"
2462            {
2463              "type": "enum",
2464              "name": "a",
2465              "namespace": "space",
2466              "aliases": ["b", "x.y", ".c"],
2467              "symbols" : [
2468                "symbol1", "symbol2"
2469              ]
2470            }
2471        "#,
2472        )?;
2473
2474        let value = serde_json::to_value(&schema)?;
2475        let serialized = serde_json::to_string(&value)?;
2476        assert_eq!(
2477            r#"{"aliases":["space.b","x.y","c"],"name":"a","namespace":"space","symbols":["symbol1","symbol2"],"type":"enum"}"#,
2478            &serialized
2479        );
2480        assert_eq!(schema, Schema::parse_str(&serialized)?);
2481
2482        Ok(())
2483    }
2484
2485    #[test]
2486    fn avro_3518_serialize_aliases_fixed() -> TestResult {
2487        let schema = Schema::parse_str(
2488            r#"
2489            {
2490              "type": "fixed",
2491              "name": "a",
2492              "namespace": "space",
2493              "aliases": ["b", "x.y", ".c"],
2494              "size" : 12
2495            }
2496        "#,
2497        )?;
2498
2499        let value = serde_json::to_value(&schema)?;
2500        let serialized = serde_json::to_string(&value)?;
2501        assert_eq!(
2502            r#"{"aliases":["space.b","x.y","c"],"name":"a","namespace":"space","size":12,"type":"fixed"}"#,
2503            &serialized
2504        );
2505        assert_eq!(schema, Schema::parse_str(&serialized)?);
2506
2507        Ok(())
2508    }
2509
2510    #[test]
2511    fn avro_3130_parse_anonymous_union_type() -> TestResult {
2512        let schema_str = r#"
2513        {
2514            "type": "record",
2515            "name": "AccountEvent",
2516            "fields": [
2517                {"type":
2518                  ["null",
2519                   { "name": "accountList",
2520                      "type": {
2521                        "type": "array",
2522                        "items": "long"
2523                      }
2524                  }
2525                  ],
2526                 "name":"NullableLongArray"
2527               }
2528            ]
2529        }
2530        "#;
2531        let schema = Schema::parse_str(schema_str)?;
2532
2533        if let Schema::Record(RecordSchema { name, fields, .. }) = schema {
2534            assert_eq!(name, Name::new("AccountEvent")?);
2535
2536            let field = &fields[0];
2537            assert_eq!(&field.name, "NullableLongArray");
2538
2539            if let Schema::Union(ref union) = field.schema {
2540                assert_eq!(union.schemas[0], Schema::Null);
2541
2542                if let Schema::Array(ref array_schema) = union.schemas[1] {
2543                    if let Schema::Long = *array_schema.items {
2544                        // OK
2545                    } else {
2546                        panic!("Expected a Schema::Array of type Long");
2547                    }
2548                } else {
2549                    panic!("Expected Schema::Array");
2550                }
2551            } else {
2552                panic!("Expected Schema::Union");
2553            }
2554        } else {
2555            panic!("Expected Schema::Record");
2556        }
2557
2558        Ok(())
2559    }
2560
2561    #[test]
2562    fn avro_custom_attributes_schema_without_attributes() -> TestResult {
2563        let schemata_str = [
2564            r#"
2565            {
2566                "type": "record",
2567                "name": "Rec",
2568                "doc": "A Record schema without custom attributes",
2569                "fields": []
2570            }
2571            "#,
2572            r#"
2573            {
2574                "type": "enum",
2575                "name": "Enum",
2576                "doc": "An Enum schema without custom attributes",
2577                "symbols": []
2578            }
2579            "#,
2580            r#"
2581            {
2582                "type": "fixed",
2583                "name": "Fixed",
2584                "doc": "A Fixed schema without custom attributes",
2585                "size": 0
2586            }
2587            "#,
2588        ];
2589        for schema_str in schemata_str.iter() {
2590            let schema = Schema::parse_str(schema_str)?;
2591            assert_eq!(schema.custom_attributes(), Some(&Default::default()));
2592        }
2593
2594        Ok(())
2595    }
2596
2597    const CUSTOM_ATTRS_SUFFIX: &str = r#"
2598            "string_key": "value",
2599            "number_key": 1.23,
2600            "null_key": null,
2601            "array_key": [1, 2, 3],
2602            "object_key": {
2603                "key": "value"
2604            }
2605        "#;
2606
2607    #[test]
2608    fn avro_3609_custom_attributes_schema_with_attributes() -> TestResult {
2609        let schemata_str = [
2610            r#"
2611            {
2612                "type": "record",
2613                "name": "Rec",
2614                "namespace": "ns",
2615                "doc": "A Record schema with custom attributes",
2616                "fields": [],
2617                {{{}}}
2618            }
2619            "#,
2620            r#"
2621            {
2622                "type": "enum",
2623                "name": "Enum",
2624                "namespace": "ns",
2625                "doc": "An Enum schema with custom attributes",
2626                "symbols": [],
2627                {{{}}}
2628            }
2629            "#,
2630            r#"
2631            {
2632                "type": "fixed",
2633                "name": "Fixed",
2634                "namespace": "ns",
2635                "doc": "A Fixed schema with custom attributes",
2636                "size": 2,
2637                {{{}}}
2638            }
2639            "#,
2640        ];
2641
2642        for schema_str in schemata_str.iter() {
2643            let schema = Schema::parse_str(
2644                schema_str
2645                    .to_owned()
2646                    .replace("{{{}}}", CUSTOM_ATTRS_SUFFIX)
2647                    .as_str(),
2648            )?;
2649
2650            assert_eq!(
2651                schema.custom_attributes(),
2652                Some(&expected_custom_attributes())
2653            );
2654        }
2655
2656        Ok(())
2657    }
2658
2659    fn expected_custom_attributes() -> BTreeMap<String, JsonValue> {
2660        let mut expected_attributes: BTreeMap<String, JsonValue> = Default::default();
2661        expected_attributes.insert(
2662            "string_key".to_string(),
2663            JsonValue::String("value".to_string()),
2664        );
2665        expected_attributes.insert("number_key".to_string(), json!(1.23));
2666        expected_attributes.insert("null_key".to_string(), JsonValue::Null);
2667        expected_attributes.insert(
2668            "array_key".to_string(),
2669            JsonValue::Array(vec![json!(1), json!(2), json!(3)]),
2670        );
2671        let mut object_value: HashMap<String, JsonValue> = HashMap::new();
2672        object_value.insert("key".to_string(), JsonValue::String("value".to_string()));
2673        expected_attributes.insert("object_key".to_string(), json!(object_value));
2674        expected_attributes
2675    }
2676
2677    #[test]
2678    fn avro_3609_custom_attributes_record_field_without_attributes() -> TestResult {
2679        let schema_str = String::from(
2680            r#"
2681            {
2682                "type": "record",
2683                "name": "Rec",
2684                "doc": "A Record schema without custom attributes",
2685                "fields": [
2686                    {
2687                        "name": "field_one",
2688                        "type": "float",
2689                        {{{}}}
2690                    }
2691                ]
2692            }
2693        "#,
2694        );
2695
2696        let schema = Schema::parse_str(schema_str.replace("{{{}}}", CUSTOM_ATTRS_SUFFIX).as_str())?;
2697
2698        match schema {
2699            Schema::Record(RecordSchema { name, fields, .. }) => {
2700                assert_eq!(name, Name::new("Rec")?);
2701                assert_eq!(fields.len(), 1);
2702                let field = &fields[0];
2703                assert_eq!(&field.name, "field_one");
2704                assert_eq!(field.custom_attributes, expected_custom_attributes());
2705            }
2706            _ => panic!("Expected Schema::Record"),
2707        }
2708
2709        Ok(())
2710    }
2711
2712    #[test]
2713    fn avro_3625_null_is_first() -> TestResult {
2714        let schema_str = String::from(
2715            r#"
2716            {
2717                "type": "record",
2718                "name": "union_schema_test",
2719                "fields": [
2720                    {"name": "a", "type": ["null", "long"], "default": null}
2721                ]
2722            }
2723        "#,
2724        );
2725
2726        let schema = Schema::parse_str(&schema_str)?;
2727
2728        match schema {
2729            Schema::Record(RecordSchema { name, fields, .. }) => {
2730                assert_eq!(name, Name::new("union_schema_test")?);
2731                assert_eq!(fields.len(), 1);
2732                let field = &fields[0];
2733                assert_eq!(&field.name, "a");
2734                assert_eq!(&field.default, &Some(JsonValue::Null));
2735                match &field.schema {
2736                    Schema::Union(union) => {
2737                        assert_eq!(union.variants().len(), 2);
2738                        assert!(union.is_nullable());
2739                        assert_eq!(union.variants()[0], Schema::Null);
2740                        assert_eq!(union.variants()[1], Schema::Long);
2741                    }
2742                    _ => panic!("Expected Schema::Union"),
2743                }
2744            }
2745            _ => panic!("Expected Schema::Record"),
2746        }
2747
2748        Ok(())
2749    }
2750
2751    #[test]
2752    fn avro_3625_null_is_last() -> TestResult {
2753        let schema_str = String::from(
2754            r#"
2755            {
2756                "type": "record",
2757                "name": "union_schema_test",
2758                "fields": [
2759                    {"name": "a", "type": ["long","null"], "default": 123}
2760                ]
2761            }
2762        "#,
2763        );
2764
2765        let schema = Schema::parse_str(&schema_str)?;
2766
2767        match schema {
2768            Schema::Record(RecordSchema { name, fields, .. }) => {
2769                assert_eq!(name, Name::new("union_schema_test")?);
2770                assert_eq!(fields.len(), 1);
2771                let field = &fields[0];
2772                assert_eq!(&field.name, "a");
2773                assert_eq!(&field.default, &Some(json!(123)));
2774                match &field.schema {
2775                    Schema::Union(union) => {
2776                        assert_eq!(union.variants().len(), 2);
2777                        assert_eq!(union.variants()[0], Schema::Long);
2778                        assert_eq!(union.variants()[1], Schema::Null);
2779                    }
2780                    _ => panic!("Expected Schema::Union"),
2781                }
2782            }
2783            _ => panic!("Expected Schema::Record"),
2784        }
2785
2786        Ok(())
2787    }
2788
2789    #[test]
2790    fn avro_3625_null_is_the_middle() -> TestResult {
2791        let schema_str = String::from(
2792            r#"
2793            {
2794                "type": "record",
2795                "name": "union_schema_test",
2796                "fields": [
2797                    {"name": "a", "type": ["long","null","int"], "default": 123}
2798                ]
2799            }
2800        "#,
2801        );
2802
2803        let schema = Schema::parse_str(&schema_str)?;
2804
2805        match schema {
2806            Schema::Record(RecordSchema { name, fields, .. }) => {
2807                assert_eq!(name, Name::new("union_schema_test")?);
2808                assert_eq!(fields.len(), 1);
2809                let field = &fields[0];
2810                assert_eq!(&field.name, "a");
2811                assert_eq!(&field.default, &Some(json!(123)));
2812                match &field.schema {
2813                    Schema::Union(union) => {
2814                        assert_eq!(union.variants().len(), 3);
2815                        assert_eq!(union.variants()[0], Schema::Long);
2816                        assert_eq!(union.variants()[1], Schema::Null);
2817                        assert_eq!(union.variants()[2], Schema::Int);
2818                    }
2819                    _ => panic!("Expected Schema::Union"),
2820                }
2821            }
2822            _ => panic!("Expected Schema::Record"),
2823        }
2824
2825        Ok(())
2826    }
2827
2828    #[test]
2829    fn avro_3649_default_notintfirst() -> TestResult {
2830        let schema_str = String::from(
2831            r#"
2832            {
2833                "type": "record",
2834                "name": "union_schema_test",
2835                "fields": [
2836                    {"name": "a", "type": ["string", "int"], "default": 123}
2837                ]
2838            }
2839        "#,
2840        );
2841
2842        let schema = Schema::parse_str(&schema_str)?;
2843
2844        match schema {
2845            Schema::Record(RecordSchema { name, fields, .. }) => {
2846                assert_eq!(name, Name::new("union_schema_test")?);
2847                assert_eq!(fields.len(), 1);
2848                let field = &fields[0];
2849                assert_eq!(&field.name, "a");
2850                assert_eq!(&field.default, &Some(json!(123)));
2851                match &field.schema {
2852                    Schema::Union(union) => {
2853                        assert_eq!(union.variants().len(), 2);
2854                        assert_eq!(union.variants()[0], Schema::String);
2855                        assert_eq!(union.variants()[1], Schema::Int);
2856                    }
2857                    _ => panic!("Expected Schema::Union"),
2858                }
2859            }
2860            _ => panic!("Expected Schema::Record"),
2861        }
2862
2863        Ok(())
2864    }
2865
2866    #[test]
2867    fn avro_3709_parsing_of_record_field_aliases() -> TestResult {
2868        let schema = r#"
2869        {
2870          "name": "rec",
2871          "type": "record",
2872          "fields": [
2873            {
2874              "name": "num",
2875              "type": "int",
2876              "aliases": ["num1", "num2"]
2877            }
2878          ]
2879        }
2880        "#;
2881
2882        let schema = Schema::parse_str(schema)?;
2883        if let Schema::Record(RecordSchema { fields, .. }) = schema {
2884            let num_field = &fields[0];
2885            assert_eq!(num_field.name, "num");
2886            assert_eq!(num_field.aliases, Some(vec!("num1".into(), "num2".into())));
2887        } else {
2888            panic!("Expected a record schema!");
2889        }
2890
2891        Ok(())
2892    }
2893
2894    #[test]
2895    fn avro_3735_parse_enum_namespace() -> TestResult {
2896        let schema = r#"
2897        {
2898            "type": "record",
2899            "name": "Foo",
2900            "namespace": "name.space",
2901            "fields":
2902            [
2903                {
2904                    "name": "barInit",
2905                    "type":
2906                    {
2907                        "type": "enum",
2908                        "name": "Bar",
2909                        "symbols":
2910                        [
2911                            "bar0",
2912                            "bar1"
2913                        ]
2914                    }
2915                },
2916                {
2917                    "name": "barUse",
2918                    "type": "Bar"
2919                }
2920            ]
2921        }
2922        "#;
2923
2924        #[derive(
2925            Debug, PartialEq, Eq, Hash, PartialOrd, Ord, Clone, serde::Deserialize, serde::Serialize,
2926        )]
2927        pub enum Bar {
2928            #[serde(rename = "bar0")]
2929            Bar0,
2930            #[serde(rename = "bar1")]
2931            Bar1,
2932        }
2933
2934        #[derive(Debug, PartialEq, Eq, Clone, serde::Deserialize, serde::Serialize)]
2935        pub struct Foo {
2936            #[serde(rename = "barInit")]
2937            pub bar_init: Bar,
2938            #[serde(rename = "barUse")]
2939            pub bar_use: Bar,
2940        }
2941
2942        let schema = Schema::parse_str(schema)?;
2943
2944        let foo = Foo {
2945            bar_init: Bar::Bar0,
2946            bar_use: Bar::Bar1,
2947        };
2948
2949        let avro_value = crate::to_value(foo)?;
2950        assert!(avro_value.validate(&schema));
2951
2952        let mut writer = crate::Writer::new(&schema, Vec::new())?;
2953
2954        // schema validation happens here
2955        writer.append_value(avro_value)?;
2956
2957        Ok(())
2958    }
2959
2960    #[test]
2961    fn avro_3755_deserialize() -> TestResult {
2962        #[derive(
2963            Debug, PartialEq, Eq, Hash, PartialOrd, Ord, Clone, serde::Deserialize, serde::Serialize,
2964        )]
2965        pub enum Bar {
2966            #[serde(rename = "bar0")]
2967            Bar0,
2968            #[serde(rename = "bar1")]
2969            Bar1,
2970            #[serde(rename = "bar2")]
2971            Bar2,
2972        }
2973
2974        #[derive(Debug, PartialEq, Eq, Clone, serde::Deserialize, serde::Serialize)]
2975        pub struct Foo {
2976            #[serde(rename = "barInit")]
2977            pub bar_init: Bar,
2978            #[serde(rename = "barUse")]
2979            pub bar_use: Bar,
2980        }
2981
2982        let writer_schema = r#"{
2983            "type": "record",
2984            "name": "Foo",
2985            "fields":
2986            [
2987                {
2988                    "name": "barInit",
2989                    "type":
2990                    {
2991                        "type": "enum",
2992                        "name": "Bar",
2993                        "symbols":
2994                        [
2995                            "bar0",
2996                            "bar1"
2997                        ]
2998                    }
2999                },
3000                {
3001                    "name": "barUse",
3002                    "type": "Bar"
3003                }
3004            ]
3005            }"#;
3006
3007        let reader_schema = r#"{
3008            "type": "record",
3009            "name": "Foo",
3010            "namespace": "name.space",
3011            "fields":
3012            [
3013                {
3014                    "name": "barInit",
3015                    "type":
3016                    {
3017                        "type": "enum",
3018                        "name": "Bar",
3019                        "symbols":
3020                        [
3021                            "bar0",
3022                            "bar1",
3023                            "bar2"
3024                        ]
3025                    }
3026                },
3027                {
3028                    "name": "barUse",
3029                    "type": "Bar"
3030                }
3031            ]
3032            }"#;
3033
3034        let writer_schema = Schema::parse_str(writer_schema)?;
3035        let foo = Foo {
3036            bar_init: Bar::Bar0,
3037            bar_use: Bar::Bar1,
3038        };
3039        let avro_value = crate::to_value(foo)?;
3040        assert!(
3041            avro_value.validate(&writer_schema),
3042            "value is valid for schema",
3043        );
3044        let datum = crate::to_avro_datum(&writer_schema, avro_value)?;
3045        let mut x = &datum[..];
3046        let reader_schema = Schema::parse_str(reader_schema)?;
3047        let deser_value = crate::from_avro_datum(&writer_schema, &mut x, Some(&reader_schema))?;
3048        match deser_value {
3049            types::Value::Record(fields) => {
3050                assert_eq!(fields.len(), 2);
3051                assert_eq!(fields[0].0, "barInit");
3052                assert_eq!(fields[0].1, types::Value::Enum(0, "bar0".to_string()));
3053                assert_eq!(fields[1].0, "barUse");
3054                assert_eq!(fields[1].1, types::Value::Enum(1, "bar1".to_string()));
3055            }
3056            _ => panic!("Expected Value::Record"),
3057        }
3058
3059        Ok(())
3060    }
3061
3062    #[test]
3063    fn test_avro_3780_decimal_schema_type_with_fixed() -> TestResult {
3064        let schema = json!(
3065        {
3066          "type": "record",
3067          "name": "recordWithDecimal",
3068          "fields": [
3069            {
3070                "name": "decimal",
3071                "type": "fixed",
3072                "name": "nestedFixed",
3073                "size": 8,
3074                "logicalType": "decimal",
3075                "precision": 4
3076            }
3077          ]
3078        });
3079
3080        let parse_result = Schema::parse(&schema);
3081        assert!(
3082            parse_result.is_ok(),
3083            "parse result must be ok, got: {parse_result:?}"
3084        );
3085
3086        Ok(())
3087    }
3088
3089    #[test]
3090    fn test_avro_3772_enum_default_wrong_type() -> TestResult {
3091        let schema = r#"
3092        {
3093          "type": "record",
3094          "name": "test",
3095          "fields": [
3096            {"name": "a", "type": "long", "default": 42},
3097            {"name": "b", "type": "string"},
3098            {
3099              "name": "c",
3100              "type": {
3101                "type": "enum",
3102                "name": "suit",
3103                "symbols": ["diamonds", "spades", "clubs", "hearts"],
3104                "default": 123
3105              }
3106            }
3107          ]
3108        }
3109        "#;
3110
3111        match Schema::parse_str(schema) {
3112            Err(err) => {
3113                assert_eq!(
3114                    err.to_string(),
3115                    "Default value for an enum must be a string! Got: 123"
3116                );
3117            }
3118            _ => panic!("Expected an error"),
3119        }
3120        Ok(())
3121    }
3122
3123    #[test]
3124    fn test_avro_3812_handle_null_namespace_properly() -> TestResult {
3125        let schema_str = r#"
3126        {
3127          "namespace": "",
3128          "type": "record",
3129          "name": "my_schema",
3130          "fields": [
3131            {
3132              "name": "a",
3133              "type": {
3134                "type": "enum",
3135                "name": "my_enum",
3136                "namespace": "",
3137                "symbols": ["a", "b"]
3138              }
3139            },  {
3140              "name": "b",
3141              "type": {
3142                "type": "fixed",
3143                "name": "my_fixed",
3144                "namespace": "",
3145                "size": 10
3146              }
3147            }
3148          ]
3149         }
3150         "#;
3151
3152        let expected = r#"{"name":"my_schema","type":"record","fields":[{"name":"a","type":{"name":"my_enum","type":"enum","symbols":["a","b"]}},{"name":"b","type":{"name":"my_fixed","type":"fixed","size":10}}]}"#;
3153        let schema = Schema::parse_str(schema_str)?;
3154        let canonical_form = schema.canonical_form();
3155        assert_eq!(canonical_form, expected);
3156
3157        let name = Name::new("my_name")?;
3158        let fullname = name.fullname(Some("".to_string()));
3159        assert_eq!(fullname, "my_name");
3160        let qname = name.fully_qualified_name(&Some("".to_string())).to_string();
3161        assert_eq!(qname, "my_name");
3162
3163        Ok(())
3164    }
3165
3166    #[test]
3167    fn test_avro_3818_inherit_enclosing_namespace() -> TestResult {
3168        // Enclosing namespace is specified but inner namespaces are not.
3169        let schema_str = r#"
3170        {
3171          "namespace": "my_ns",
3172          "type": "record",
3173          "name": "my_schema",
3174          "fields": [
3175            {
3176              "name": "f1",
3177              "type": {
3178                "name": "enum1",
3179                "type": "enum",
3180                "symbols": ["a"]
3181              }
3182            },  {
3183              "name": "f2",
3184              "type": {
3185                "name": "fixed1",
3186                "type": "fixed",
3187                "size": 1
3188              }
3189            }
3190          ]
3191        }
3192        "#;
3193
3194        let expected = r#"{"name":"my_ns.my_schema","type":"record","fields":[{"name":"f1","type":{"name":"my_ns.enum1","type":"enum","symbols":["a"]}},{"name":"f2","type":{"name":"my_ns.fixed1","type":"fixed","size":1}}]}"#;
3195        let schema = Schema::parse_str(schema_str)?;
3196        let canonical_form = schema.canonical_form();
3197        assert_eq!(canonical_form, expected);
3198
3199        // Enclosing namespace and inner namespaces are specified
3200        // but inner namespaces are ""
3201        let schema_str = r#"
3202        {
3203          "namespace": "my_ns",
3204          "type": "record",
3205          "name": "my_schema",
3206          "fields": [
3207            {
3208              "name": "f1",
3209              "type": {
3210                "name": "enum1",
3211                "type": "enum",
3212                "namespace": "",
3213                "symbols": ["a"]
3214              }
3215            },  {
3216              "name": "f2",
3217              "type": {
3218                "name": "fixed1",
3219                "type": "fixed",
3220                "namespace": "",
3221                "size": 1
3222              }
3223            }
3224          ]
3225        }
3226        "#;
3227
3228        let expected = r#"{"name":"my_ns.my_schema","type":"record","fields":[{"name":"f1","type":{"name":"enum1","type":"enum","symbols":["a"]}},{"name":"f2","type":{"name":"fixed1","type":"fixed","size":1}}]}"#;
3229        let schema = Schema::parse_str(schema_str)?;
3230        let canonical_form = schema.canonical_form();
3231        assert_eq!(canonical_form, expected);
3232
3233        // Enclosing namespace is "" and inner non-empty namespaces are specified.
3234        let schema_str = r#"
3235        {
3236          "namespace": "",
3237          "type": "record",
3238          "name": "my_schema",
3239          "fields": [
3240            {
3241              "name": "f1",
3242              "type": {
3243                "name": "enum1",
3244                "type": "enum",
3245                "namespace": "f1.ns",
3246                "symbols": ["a"]
3247              }
3248            },  {
3249              "name": "f2",
3250              "type": {
3251                "name": "f2.ns.fixed1",
3252                "type": "fixed",
3253                "size": 1
3254              }
3255            }
3256          ]
3257        }
3258        "#;
3259
3260        let expected = r#"{"name":"my_schema","type":"record","fields":[{"name":"f1","type":{"name":"f1.ns.enum1","type":"enum","symbols":["a"]}},{"name":"f2","type":{"name":"f2.ns.fixed1","type":"fixed","size":1}}]}"#;
3261        let schema = Schema::parse_str(schema_str)?;
3262        let canonical_form = schema.canonical_form();
3263        assert_eq!(canonical_form, expected);
3264
3265        // Nested complex types with non-empty enclosing namespace.
3266        let schema_str = r#"
3267        {
3268          "type": "record",
3269          "name": "my_ns.my_schema",
3270          "fields": [
3271            {
3272              "name": "f1",
3273              "type": {
3274                "name": "inner_record1",
3275                "type": "record",
3276                "fields": [
3277                  {
3278                    "name": "f1_1",
3279                    "type": {
3280                      "name": "enum1",
3281                      "type": "enum",
3282                      "symbols": ["a"]
3283                    }
3284                  }
3285                ]
3286              }
3287            },  {
3288              "name": "f2",
3289                "type": {
3290                "name": "inner_record2",
3291                "type": "record",
3292                "namespace": "inner_ns",
3293                "fields": [
3294                  {
3295                    "name": "f2_1",
3296                    "type": {
3297                      "name": "enum2",
3298                      "type": "enum",
3299                      "symbols": ["a"]
3300                    }
3301                  }
3302                ]
3303              }
3304            }
3305          ]
3306        }
3307        "#;
3308
3309        let expected = r#"{"name":"my_ns.my_schema","type":"record","fields":[{"name":"f1","type":{"name":"my_ns.inner_record1","type":"record","fields":[{"name":"f1_1","type":{"name":"my_ns.enum1","type":"enum","symbols":["a"]}}]}},{"name":"f2","type":{"name":"inner_ns.inner_record2","type":"record","fields":[{"name":"f2_1","type":{"name":"inner_ns.enum2","type":"enum","symbols":["a"]}}]}}]}"#;
3310        let schema = Schema::parse_str(schema_str)?;
3311        let canonical_form = schema.canonical_form();
3312        assert_eq!(canonical_form, expected);
3313
3314        Ok(())
3315    }
3316
3317    #[test]
3318    fn test_avro_3779_bigdecimal_schema() -> TestResult {
3319        let schema = json!(
3320            {
3321                "name": "decimal",
3322                "type": "bytes",
3323                "logicalType": "big-decimal"
3324            }
3325        );
3326
3327        let parse_result = Schema::parse(&schema);
3328        assert!(
3329            parse_result.is_ok(),
3330            "parse result must be ok, got: {parse_result:?}"
3331        );
3332        match parse_result? {
3333            Schema::BigDecimal => (),
3334            other => panic!("Expected Schema::BigDecimal but got: {other:?}"),
3335        }
3336
3337        Ok(())
3338    }
3339
3340    #[test]
3341    fn test_avro_3820_deny_invalid_field_names() -> TestResult {
3342        let schema_str = r#"
3343        {
3344          "name": "my_record",
3345          "type": "record",
3346          "fields": [
3347            {
3348              "name": "f1.x",
3349              "type": {
3350                "name": "my_enum",
3351                "type": "enum",
3352                "symbols": ["a"]
3353              }
3354            },  {
3355              "name": "f2",
3356              "type": {
3357                "name": "my_fixed",
3358                "type": "fixed",
3359                "size": 1
3360              }
3361            }
3362          ]
3363        }
3364        "#;
3365
3366        match Schema::parse_str(schema_str).map_err(Error::into_details) {
3367            Err(Details::FieldName(x)) if x == "f1.x" => Ok(()),
3368            other => Err(format!("Expected Details::FieldName, got {other:?}").into()),
3369        }
3370    }
3371
3372    #[test]
3373    fn test_avro_3827_disallow_duplicate_field_names() -> TestResult {
3374        let schema_str = r#"
3375        {
3376          "name": "my_schema",
3377          "type": "record",
3378          "fields": [
3379            {
3380              "name": "f1",
3381              "type": {
3382                "name": "a",
3383                "type": "record",
3384                "fields": []
3385              }
3386            },  {
3387              "name": "f1",
3388              "type": {
3389                "name": "b",
3390                "type": "record",
3391                "fields": []
3392              }
3393            }
3394          ]
3395        }
3396        "#;
3397
3398        match Schema::parse_str(schema_str).map_err(Error::into_details) {
3399            Err(Details::FieldNameDuplicate(_)) => (),
3400            other => {
3401                return Err(format!("Expected Details::FieldNameDuplicate, got {other:?}").into());
3402            }
3403        };
3404
3405        let schema_str = r#"
3406        {
3407          "name": "my_schema",
3408          "type": "record",
3409          "fields": [
3410            {
3411              "name": "f1",
3412              "type": {
3413                "name": "a",
3414                "type": "record",
3415                "fields": [
3416                  {
3417                    "name": "f1",
3418                    "type": {
3419                      "name": "b",
3420                      "type": "record",
3421                      "fields": []
3422                    }
3423                  }
3424                ]
3425              }
3426            }
3427          ]
3428        }
3429        "#;
3430
3431        let expected = r#"{"name":"my_schema","type":"record","fields":[{"name":"f1","type":{"name":"a","type":"record","fields":[{"name":"f1","type":{"name":"b","type":"record","fields":[]}}]}}]}"#;
3432        let schema = Schema::parse_str(schema_str)?;
3433        let canonical_form = schema.canonical_form();
3434        assert_eq!(canonical_form, expected);
3435
3436        Ok(())
3437    }
3438
3439    #[test]
3440    fn test_avro_3830_null_namespace_in_fully_qualified_names() -> TestResult {
3441        // Check whether all the named types don't refer to the namespace field
3442        // if their name starts with a dot.
3443        let schema_str = r#"
3444        {
3445          "name": ".record1",
3446          "namespace": "ns1",
3447          "type": "record",
3448          "fields": [
3449            {
3450              "name": "f1",
3451              "type": {
3452                "name": ".enum1",
3453                "namespace": "ns2",
3454                "type": "enum",
3455                "symbols": ["a"]
3456              }
3457            },  {
3458              "name": "f2",
3459              "type": {
3460                "name": ".fxed1",
3461                "namespace": "ns3",
3462                "type": "fixed",
3463                "size": 1
3464              }
3465            }
3466          ]
3467        }
3468        "#;
3469
3470        let expected = r#"{"name":"record1","type":"record","fields":[{"name":"f1","type":{"name":"enum1","type":"enum","symbols":["a"]}},{"name":"f2","type":{"name":"fxed1","type":"fixed","size":1}}]}"#;
3471        let schema = Schema::parse_str(schema_str)?;
3472        let canonical_form = schema.canonical_form();
3473        assert_eq!(canonical_form, expected);
3474
3475        // Check whether inner types don't inherit ns1.
3476        let schema_str = r#"
3477        {
3478          "name": ".record1",
3479          "namespace": "ns1",
3480          "type": "record",
3481          "fields": [
3482            {
3483              "name": "f1",
3484              "type": {
3485                "name": "enum1",
3486                "type": "enum",
3487                "symbols": ["a"]
3488              }
3489            },  {
3490              "name": "f2",
3491              "type": {
3492                "name": "fxed1",
3493                "type": "fixed",
3494                "size": 1
3495              }
3496            }
3497          ]
3498        }
3499        "#;
3500
3501        let expected = r#"{"name":"record1","type":"record","fields":[{"name":"f1","type":{"name":"enum1","type":"enum","symbols":["a"]}},{"name":"f2","type":{"name":"fxed1","type":"fixed","size":1}}]}"#;
3502        let schema = Schema::parse_str(schema_str)?;
3503        let canonical_form = schema.canonical_form();
3504        assert_eq!(canonical_form, expected);
3505
3506        let name = Name::new(".my_name")?;
3507        let fullname = name.fullname(None);
3508        assert_eq!(fullname, "my_name");
3509        let qname = name.fully_qualified_name(&None).to_string();
3510        assert_eq!(qname, "my_name");
3511
3512        Ok(())
3513    }
3514
3515    #[test]
3516    fn test_avro_3814_schema_resolution_failure() -> TestResult {
3517        // Define a reader schema: a nested record with an optional field.
3518        let reader_schema = json!(
3519            {
3520                "type": "record",
3521                "name": "MyOuterRecord",
3522                "fields": [
3523                    {
3524                        "name": "inner_record",
3525                        "type": [
3526                            "null",
3527                            {
3528                                "type": "record",
3529                                "name": "MyRecord",
3530                                "fields": [
3531                                    {"name": "a", "type": "string"}
3532                                ]
3533                            }
3534                        ],
3535                        "default": null
3536                    }
3537                ]
3538            }
3539        );
3540
3541        // Define a writer schema: a nested record with an optional field, which
3542        // may optionally contain an enum.
3543        let writer_schema = json!(
3544            {
3545                "type": "record",
3546                "name": "MyOuterRecord",
3547                "fields": [
3548                    {
3549                        "name": "inner_record",
3550                        "type": [
3551                            "null",
3552                            {
3553                                "type": "record",
3554                                "name": "MyRecord",
3555                                "fields": [
3556                                    {"name": "a", "type": "string"},
3557                                    {
3558                                        "name": "b",
3559                                        "type": [
3560                                            "null",
3561                                            {
3562                                                "type": "enum",
3563                                                "name": "MyEnum",
3564                                                "symbols": ["A", "B", "C"],
3565                                                "default": "C"
3566                                            }
3567                                        ],
3568                                        "default": null
3569                                    },
3570                                ]
3571                            }
3572                        ]
3573                    }
3574                ],
3575                "default": null
3576            }
3577        );
3578
3579        // Use different structs to represent the "Reader" and the "Writer"
3580        // to mimic two different versions of a producer & consumer application.
3581        #[derive(Serialize, Deserialize, Debug)]
3582        struct MyInnerRecordReader {
3583            a: String,
3584        }
3585
3586        #[derive(Serialize, Deserialize, Debug)]
3587        struct MyRecordReader {
3588            inner_record: Option<MyInnerRecordReader>,
3589        }
3590
3591        #[derive(Serialize, Deserialize, Debug)]
3592        enum MyEnum {
3593            A,
3594            B,
3595            C,
3596        }
3597
3598        #[derive(Serialize, Deserialize, Debug)]
3599        struct MyInnerRecordWriter {
3600            a: String,
3601            b: Option<MyEnum>,
3602        }
3603
3604        #[derive(Serialize, Deserialize, Debug)]
3605        struct MyRecordWriter {
3606            inner_record: Option<MyInnerRecordWriter>,
3607        }
3608
3609        let s = MyRecordWriter {
3610            inner_record: Some(MyInnerRecordWriter {
3611                a: "foo".to_string(),
3612                b: None,
3613            }),
3614        };
3615
3616        // Serialize using the writer schema.
3617        let writer_schema = Schema::parse(&writer_schema)?;
3618        let avro_value = crate::to_value(s)?;
3619        assert!(
3620            avro_value.validate(&writer_schema),
3621            "value is valid for schema",
3622        );
3623        let datum = crate::to_avro_datum(&writer_schema, avro_value)?;
3624
3625        // Now, attempt to deserialize using the reader schema.
3626        let reader_schema = Schema::parse(&reader_schema)?;
3627        let mut x = &datum[..];
3628
3629        // Deserialization should succeed and we should be able to resolve the schema.
3630        let deser_value = crate::from_avro_datum(&writer_schema, &mut x, Some(&reader_schema))?;
3631        assert!(deser_value.validate(&reader_schema));
3632
3633        // Verify that we can read a field from the record.
3634        let d: MyRecordReader = crate::from_value(&deser_value)?;
3635        assert_eq!(d.inner_record.unwrap().a, "foo".to_string());
3636        Ok(())
3637    }
3638
3639    #[test]
3640    fn test_avro_3837_disallow_invalid_namespace() -> TestResult {
3641        // Valid namespace #1 (Single name portion)
3642        let schema_str = r#"
3643        {
3644          "name": "record1",
3645          "namespace": "ns1",
3646          "type": "record",
3647          "fields": []
3648        }
3649        "#;
3650
3651        let expected = r#"{"name":"ns1.record1","type":"record","fields":[]}"#;
3652        let schema = Schema::parse_str(schema_str)?;
3653        let canonical_form = schema.canonical_form();
3654        assert_eq!(canonical_form, expected);
3655
3656        // Valid namespace #2 (multiple name portions).
3657        let schema_str = r#"
3658        {
3659          "name": "enum1",
3660          "namespace": "ns1.foo.bar",
3661          "type": "enum",
3662          "symbols": ["a"]
3663        }
3664        "#;
3665
3666        let expected = r#"{"name":"ns1.foo.bar.enum1","type":"enum","symbols":["a"]}"#;
3667        let schema = Schema::parse_str(schema_str)?;
3668        let canonical_form = schema.canonical_form();
3669        assert_eq!(canonical_form, expected);
3670
3671        // Invalid namespace #1 (a name portion starts with dot)
3672        let schema_str = r#"
3673        {
3674          "name": "fixed1",
3675          "namespace": ".ns1.a.b",
3676          "type": "fixed",
3677          "size": 1
3678        }
3679        "#;
3680
3681        match Schema::parse_str(schema_str).map_err(Error::into_details) {
3682            Err(Details::InvalidNamespace(_, _)) => (),
3683            other => {
3684                return Err(format!("Expected Details::InvalidNamespace, got {other:?}").into());
3685            }
3686        };
3687
3688        // Invalid namespace #2 (invalid character in a name portion)
3689        let schema_str = r#"
3690        {
3691          "name": "record1",
3692          "namespace": "ns1.a*b.c",
3693          "type": "record",
3694          "fields": []
3695        }
3696        "#;
3697
3698        match Schema::parse_str(schema_str).map_err(Error::into_details) {
3699            Err(Details::InvalidNamespace(_, _)) => (),
3700            other => {
3701                return Err(format!("Expected Details::InvalidNamespace, got {other:?}").into());
3702            }
3703        };
3704
3705        // Invalid namespace #3 (a name portion starts with a digit)
3706        let schema_str = r#"
3707        {
3708          "name": "fixed1",
3709          "namespace": "ns1.1a.b",
3710          "type": "fixed",
3711          "size": 1
3712        }
3713        "#;
3714
3715        match Schema::parse_str(schema_str).map_err(Error::into_details) {
3716            Err(Details::InvalidNamespace(_, _)) => (),
3717            other => {
3718                return Err(format!("Expected Details::InvalidNamespace, got {other:?}").into());
3719            }
3720        };
3721
3722        // Invalid namespace #4 (a name portion is missing - two dots in a row)
3723        let schema_str = r#"
3724        {
3725          "name": "fixed1",
3726          "namespace": "ns1..a",
3727          "type": "fixed",
3728          "size": 1
3729        }
3730        "#;
3731
3732        match Schema::parse_str(schema_str).map_err(Error::into_details) {
3733            Err(Details::InvalidNamespace(_, _)) => (),
3734            other => {
3735                return Err(format!("Expected Details::InvalidNamespace, got {other:?}").into());
3736            }
3737        };
3738
3739        // Invalid namespace #5 (a name portion is missing - ends with a dot)
3740        let schema_str = r#"
3741        {
3742          "name": "fixed1",
3743          "namespace": "ns1.a.",
3744          "type": "fixed",
3745          "size": 1
3746        }
3747        "#;
3748
3749        match Schema::parse_str(schema_str).map_err(Error::into_details) {
3750            Err(Details::InvalidNamespace(_, _)) => (),
3751            other => {
3752                return Err(format!("Expected Details::InvalidNamespace, got {other:?}").into());
3753            }
3754        };
3755
3756        Ok(())
3757    }
3758
3759    #[test]
3760    fn test_avro_3851_validate_default_value_of_simple_record_field() -> TestResult {
3761        let schema_str = r#"
3762        {
3763            "name": "record1",
3764            "namespace": "ns",
3765            "type": "record",
3766            "fields": [
3767                {
3768                    "name": "f1",
3769                    "type": "int",
3770                    "default": "invalid"
3771                }
3772            ]
3773        }
3774        "#;
3775        let expected = Details::GetDefaultRecordField(
3776            "f1".to_string(),
3777            "ns.record1".to_string(),
3778            r#""int""#.to_string(),
3779        )
3780        .to_string();
3781        let result = Schema::parse_str(schema_str);
3782        assert!(result.is_err());
3783        let err = result
3784            .map_err(|e| e.to_string())
3785            .err()
3786            .unwrap_or_else(|| "unexpected".to_string());
3787        assert_eq!(expected, err);
3788
3789        Ok(())
3790    }
3791
3792    #[test]
3793    fn test_avro_3851_validate_default_value_of_nested_record_field() -> TestResult {
3794        let schema_str = r#"
3795        {
3796            "name": "record1",
3797            "namespace": "ns",
3798            "type": "record",
3799            "fields": [
3800                {
3801                    "name": "f1",
3802                    "type": {
3803                        "name": "record2",
3804                        "type": "record",
3805                        "fields": [
3806                            {
3807                                "name": "f1_1",
3808                                "type": "int"
3809                            }
3810                        ]
3811                    },
3812                    "default": "invalid"
3813                }
3814            ]
3815        }
3816        "#;
3817        let expected = Details::GetDefaultRecordField(
3818            "f1".to_string(),
3819            "ns.record1".to_string(),
3820            r#"{"name":"ns.record2","type":"record","fields":[{"name":"f1_1","type":"int"}]}"#
3821                .to_string(),
3822        )
3823        .to_string();
3824        let result = Schema::parse_str(schema_str);
3825        assert!(result.is_err());
3826        let err = result
3827            .map_err(|e| e.to_string())
3828            .err()
3829            .unwrap_or_else(|| "unexpected".to_string());
3830        assert_eq!(expected, err);
3831
3832        Ok(())
3833    }
3834
3835    #[test]
3836    fn test_avro_3851_validate_default_value_of_enum_record_field() -> TestResult {
3837        let schema_str = r#"
3838        {
3839            "name": "record1",
3840            "namespace": "ns",
3841            "type": "record",
3842            "fields": [
3843                {
3844                    "name": "f1",
3845                    "type": {
3846                        "name": "enum1",
3847                        "type": "enum",
3848                        "symbols": ["a", "b", "c"]
3849                    },
3850                    "default": "invalid"
3851                }
3852            ]
3853        }
3854        "#;
3855        let expected = Details::GetDefaultRecordField(
3856            "f1".to_string(),
3857            "ns.record1".to_string(),
3858            r#"{"name":"ns.enum1","type":"enum","symbols":["a","b","c"]}"#.to_string(),
3859        )
3860        .to_string();
3861        let result = Schema::parse_str(schema_str);
3862        assert!(result.is_err());
3863        let err = result
3864            .map_err(|e| e.to_string())
3865            .err()
3866            .unwrap_or_else(|| "unexpected".to_string());
3867        assert_eq!(expected, err);
3868
3869        Ok(())
3870    }
3871
3872    #[test]
3873    fn test_avro_3851_validate_default_value_of_fixed_record_field() -> TestResult {
3874        let schema_str = r#"
3875        {
3876            "name": "record1",
3877            "namespace": "ns",
3878            "type": "record",
3879            "fields": [
3880                {
3881                    "name": "f1",
3882                    "type": {
3883                        "name": "fixed1",
3884                        "type": "fixed",
3885                        "size": 3
3886                    },
3887                    "default": 100
3888                }
3889            ]
3890        }
3891        "#;
3892        let expected = Details::GetDefaultRecordField(
3893            "f1".to_string(),
3894            "ns.record1".to_string(),
3895            r#"{"name":"ns.fixed1","type":"fixed","size":3}"#.to_string(),
3896        )
3897        .to_string();
3898        let result = Schema::parse_str(schema_str);
3899        assert!(result.is_err());
3900        let err = result
3901            .map_err(|e| e.to_string())
3902            .err()
3903            .unwrap_or_else(|| "unexpected".to_string());
3904        assert_eq!(expected, err);
3905
3906        Ok(())
3907    }
3908
3909    #[test]
3910    fn test_avro_3851_validate_default_value_of_array_record_field() -> TestResult {
3911        let schema_str = r#"
3912        {
3913            "name": "record1",
3914            "namespace": "ns",
3915            "type": "record",
3916            "fields": [
3917                {
3918                    "name": "f1",
3919                    "type": "array",
3920                    "items": "int",
3921                    "default": "invalid"
3922                }
3923            ]
3924        }
3925        "#;
3926
3927        let result = Schema::parse_str(schema_str);
3928        assert!(result.is_err());
3929        let err = result
3930            .map_err(|e| e.to_string())
3931            .err()
3932            .unwrap_or_else(|| "unexpected".to_string());
3933        assert_eq!(
3934            r#"Default value for an array must be an array! Got: "invalid""#,
3935            err
3936        );
3937
3938        Ok(())
3939    }
3940
3941    #[test]
3942    fn test_avro_3851_validate_default_value_of_map_record_field() -> TestResult {
3943        let schema_str = r#"
3944        {
3945            "name": "record1",
3946            "namespace": "ns",
3947            "type": "record",
3948            "fields": [
3949                {
3950                    "name": "f1",
3951                    "type": "map",
3952                    "values": "string",
3953                    "default": "invalid"
3954                }
3955            ]
3956        }
3957        "#;
3958
3959        let result = Schema::parse_str(schema_str);
3960        assert!(result.is_err());
3961        let err = result
3962            .map_err(|e| e.to_string())
3963            .err()
3964            .unwrap_or_else(|| "unexpected".to_string());
3965        assert_eq!(
3966            r#"Default value for a map must be an object! Got: "invalid""#,
3967            err
3968        );
3969
3970        Ok(())
3971    }
3972
3973    #[test]
3974    fn test_avro_3851_validate_default_value_of_ref_record_field() -> TestResult {
3975        let schema_str = r#"
3976        {
3977            "name": "record1",
3978            "namespace": "ns",
3979            "type": "record",
3980            "fields": [
3981                {
3982                    "name": "f1",
3983                    "type": {
3984                        "name": "record2",
3985                        "type": "record",
3986                        "fields": [
3987                            {
3988                                "name": "f1_1",
3989                                "type": "int"
3990                            }
3991                        ]
3992                    }
3993                },  {
3994                    "name": "f2",
3995                    "type": "ns.record2",
3996                    "default": { "f1_1": true }
3997                }
3998            ]
3999        }
4000        "#;
4001        let expected = Details::GetDefaultRecordField(
4002            "f2".to_string(),
4003            "ns.record1".to_string(),
4004            r#""ns.record2""#.to_string(),
4005        )
4006        .to_string();
4007        let result = Schema::parse_str(schema_str);
4008        assert!(result.is_err());
4009        let err = result
4010            .map_err(|e| e.to_string())
4011            .err()
4012            .unwrap_or_else(|| "unexpected".to_string());
4013        assert_eq!(expected, err);
4014
4015        Ok(())
4016    }
4017
4018    #[test]
4019    fn test_avro_3851_validate_default_value_of_enum() -> TestResult {
4020        let schema_str = r#"
4021        {
4022            "name": "enum1",
4023            "namespace": "ns",
4024            "type": "enum",
4025            "symbols": ["a", "b", "c"],
4026            "default": 100
4027        }
4028        "#;
4029        let expected = Details::EnumDefaultWrongType(100.into()).to_string();
4030        let result = Schema::parse_str(schema_str);
4031        assert!(result.is_err());
4032        let err = result
4033            .map_err(|e| e.to_string())
4034            .err()
4035            .unwrap_or_else(|| "unexpected".to_string());
4036        assert_eq!(expected, err);
4037
4038        let schema_str = r#"
4039        {
4040            "name": "enum1",
4041            "namespace": "ns",
4042            "type": "enum",
4043            "symbols": ["a", "b", "c"],
4044            "default": "d"
4045        }
4046        "#;
4047        let expected = Details::GetEnumDefault {
4048            symbol: "d".to_string(),
4049            symbols: vec!["a".to_string(), "b".to_string(), "c".to_string()],
4050        }
4051        .to_string();
4052        let result = Schema::parse_str(schema_str);
4053        assert!(result.is_err());
4054        let err = result
4055            .map_err(|e| e.to_string())
4056            .err()
4057            .unwrap_or_else(|| "unexpected".to_string());
4058        assert_eq!(expected, err);
4059
4060        Ok(())
4061    }
4062
4063    #[test]
4064    fn test_avro_3862_get_aliases() -> TestResult {
4065        // Test for Record
4066        let schema_str = r#"
4067        {
4068            "name": "record1",
4069            "namespace": "ns1",
4070            "type": "record",
4071            "aliases": ["r1", "ns2.r2"],
4072            "fields": [
4073                { "name": "f1", "type": "int" },
4074                { "name": "f2", "type": "string" }
4075            ]
4076        }
4077        "#;
4078        let schema = Schema::parse_str(schema_str)?;
4079        let expected = vec![Alias::new("ns1.r1")?, Alias::new("ns2.r2")?];
4080        match schema.aliases() {
4081            Some(aliases) => assert_eq!(aliases, &expected),
4082            None => panic!("Expected Some({expected:?}), got None"),
4083        }
4084
4085        let schema_str = r#"
4086        {
4087            "name": "record1",
4088            "namespace": "ns1",
4089            "type": "record",
4090            "fields": [
4091                { "name": "f1", "type": "int" },
4092                { "name": "f2", "type": "string" }
4093            ]
4094        }
4095        "#;
4096        let schema = Schema::parse_str(schema_str)?;
4097        match schema.aliases() {
4098            None => (),
4099            some => panic!("Expected None, got {some:?}"),
4100        }
4101
4102        // Test for Enum
4103        let schema_str = r#"
4104        {
4105            "name": "enum1",
4106            "namespace": "ns1",
4107            "type": "enum",
4108            "aliases": ["en1", "ns2.en2"],
4109            "symbols": ["a", "b", "c"]
4110        }
4111        "#;
4112        let schema = Schema::parse_str(schema_str)?;
4113        let expected = vec![Alias::new("ns1.en1")?, Alias::new("ns2.en2")?];
4114        match schema.aliases() {
4115            Some(aliases) => assert_eq!(aliases, &expected),
4116            None => panic!("Expected Some({expected:?}), got None"),
4117        }
4118
4119        let schema_str = r#"
4120        {
4121            "name": "enum1",
4122            "namespace": "ns1",
4123            "type": "enum",
4124            "symbols": ["a", "b", "c"]
4125        }
4126        "#;
4127        let schema = Schema::parse_str(schema_str)?;
4128        match schema.aliases() {
4129            None => (),
4130            some => panic!("Expected None, got {some:?}"),
4131        }
4132
4133        // Test for Fixed
4134        let schema_str = r#"
4135        {
4136            "name": "fixed1",
4137            "namespace": "ns1",
4138            "type": "fixed",
4139            "aliases": ["fx1", "ns2.fx2"],
4140            "size": 10
4141        }
4142        "#;
4143        let schema = Schema::parse_str(schema_str)?;
4144        let expected = vec![Alias::new("ns1.fx1")?, Alias::new("ns2.fx2")?];
4145        match schema.aliases() {
4146            Some(aliases) => assert_eq!(aliases, &expected),
4147            None => panic!("Expected Some({expected:?}), got None"),
4148        }
4149
4150        let schema_str = r#"
4151        {
4152            "name": "fixed1",
4153            "namespace": "ns1",
4154            "type": "fixed",
4155            "size": 10
4156        }
4157        "#;
4158        let schema = Schema::parse_str(schema_str)?;
4159        match schema.aliases() {
4160            None => (),
4161            some => panic!("Expected None, got {some:?}"),
4162        }
4163
4164        // Test for non-named type
4165        let schema = Schema::Int;
4166        match schema.aliases() {
4167            None => (),
4168            some => panic!("Expected None, got {some:?}"),
4169        }
4170
4171        Ok(())
4172    }
4173
4174    #[test]
4175    fn test_avro_3862_get_doc() -> TestResult {
4176        // Test for Record
4177        let schema_str = r#"
4178        {
4179            "name": "record1",
4180            "type": "record",
4181            "doc": "Record Document",
4182            "fields": [
4183                { "name": "f1", "type": "int" },
4184                { "name": "f2", "type": "string" }
4185            ]
4186        }
4187        "#;
4188        let schema = Schema::parse_str(schema_str)?;
4189        let expected = "Record Document";
4190        match schema.doc() {
4191            Some(doc) => assert_eq!(doc, expected),
4192            None => panic!("Expected Some({expected:?}), got None"),
4193        }
4194
4195        let schema_str = r#"
4196        {
4197            "name": "record1",
4198            "type": "record",
4199            "fields": [
4200                { "name": "f1", "type": "int" },
4201                { "name": "f2", "type": "string" }
4202            ]
4203        }
4204        "#;
4205        let schema = Schema::parse_str(schema_str)?;
4206        match schema.doc() {
4207            None => (),
4208            some => panic!("Expected None, got {some:?}"),
4209        }
4210
4211        // Test for Enum
4212        let schema_str = r#"
4213        {
4214            "name": "enum1",
4215            "type": "enum",
4216            "doc": "Enum Document",
4217            "symbols": ["a", "b", "c"]
4218        }
4219        "#;
4220        let schema = Schema::parse_str(schema_str)?;
4221        let expected = "Enum Document";
4222        match schema.doc() {
4223            Some(doc) => assert_eq!(doc, expected),
4224            None => panic!("Expected Some({expected:?}), got None"),
4225        }
4226
4227        let schema_str = r#"
4228        {
4229            "name": "enum1",
4230            "type": "enum",
4231            "symbols": ["a", "b", "c"]
4232        }
4233        "#;
4234        let schema = Schema::parse_str(schema_str)?;
4235        match schema.doc() {
4236            None => (),
4237            some => panic!("Expected None, got {some:?}"),
4238        }
4239
4240        // Test for Fixed
4241        let schema_str = r#"
4242        {
4243            "name": "fixed1",
4244            "type": "fixed",
4245            "doc": "Fixed Document",
4246            "size": 10
4247        }
4248        "#;
4249        let schema = Schema::parse_str(schema_str)?;
4250        let expected = "Fixed Document";
4251        match schema.doc() {
4252            Some(doc) => assert_eq!(doc, expected),
4253            None => panic!("Expected Some({expected:?}), got None"),
4254        }
4255
4256        let schema_str = r#"
4257        {
4258            "name": "fixed1",
4259            "type": "fixed",
4260            "size": 10
4261        }
4262        "#;
4263        let schema = Schema::parse_str(schema_str)?;
4264        match schema.doc() {
4265            None => (),
4266            some => panic!("Expected None, got {some:?}"),
4267        }
4268
4269        // Test for non-named type
4270        let schema = Schema::Int;
4271        match schema.doc() {
4272            None => (),
4273            some => panic!("Expected None, got {some:?}"),
4274        }
4275
4276        Ok(())
4277    }
4278
4279    #[test]
4280    fn avro_3886_serialize_attributes() -> TestResult {
4281        let attributes = BTreeMap::from([
4282            ("string_key".into(), "value".into()),
4283            ("number_key".into(), 1.23.into()),
4284            ("null_key".into(), JsonValue::Null),
4285            (
4286                "array_key".into(),
4287                JsonValue::Array(vec![1.into(), 2.into(), 3.into()]),
4288            ),
4289            ("object_key".into(), JsonValue::Object(Map::default())),
4290        ]);
4291
4292        // Test serialize enum attributes
4293        let schema = Schema::Enum(EnumSchema {
4294            name: Name::new("a")?,
4295            aliases: None,
4296            doc: None,
4297            symbols: vec![],
4298            default: None,
4299            attributes: attributes.clone(),
4300        });
4301        let serialized = serde_json::to_string(&schema)?;
4302        assert_eq!(
4303            r#"{"type":"enum","name":"a","symbols":[],"array_key":[1,2,3],"null_key":null,"number_key":1.23,"object_key":{},"string_key":"value"}"#,
4304            &serialized
4305        );
4306
4307        // Test serialize fixed custom_attributes
4308        let schema = Schema::Fixed(FixedSchema {
4309            name: Name::new("a")?,
4310            aliases: None,
4311            doc: None,
4312            size: 1,
4313            attributes: attributes.clone(),
4314        });
4315        let serialized = serde_json::to_string(&schema)?;
4316        assert_eq!(
4317            r#"{"type":"fixed","name":"a","size":1,"array_key":[1,2,3],"null_key":null,"number_key":1.23,"object_key":{},"string_key":"value"}"#,
4318            &serialized
4319        );
4320
4321        // Test serialize record custom_attributes
4322        let schema = Schema::Record(RecordSchema {
4323            name: Name::new("a")?,
4324            aliases: None,
4325            doc: None,
4326            fields: vec![],
4327            lookup: BTreeMap::new(),
4328            attributes,
4329        });
4330        let serialized = serde_json::to_string(&schema)?;
4331        assert_eq!(
4332            r#"{"type":"record","name":"a","fields":[],"array_key":[1,2,3],"null_key":null,"number_key":1.23,"object_key":{},"string_key":"value"}"#,
4333            &serialized
4334        );
4335
4336        Ok(())
4337    }
4338
4339    #[test]
4340    fn test_avro_3896_decimal_schema() -> TestResult {
4341        // bytes decimal, represented as native logical type.
4342        let schema = json!(
4343        {
4344          "type": "bytes",
4345          "name": "BytesDecimal",
4346          "logicalType": "decimal",
4347          "size": 38,
4348          "precision": 9,
4349          "scale": 2
4350        });
4351        let parse_result = Schema::parse(&schema)?;
4352        assert!(matches!(
4353            parse_result,
4354            Schema::Decimal(DecimalSchema {
4355                precision: 9,
4356                scale: 2,
4357                ..
4358            })
4359        ));
4360
4361        // long decimal, represents as native complex type.
4362        let schema = json!(
4363        {
4364          "type": "long",
4365          "name": "LongDecimal",
4366          "logicalType": "decimal"
4367        });
4368        let parse_result = Schema::parse(&schema)?;
4369        // assert!(matches!(parse_result, Schema::Long));
4370        assert_eq!(parse_result, Schema::Long);
4371
4372        Ok(())
4373    }
4374
4375    #[test]
4376    fn avro_3896_uuid_schema_for_string() -> TestResult {
4377        // string uuid, represents as native logical type.
4378        let schema = json!(
4379        {
4380          "type": "string",
4381          "name": "StringUUID",
4382          "logicalType": "uuid"
4383        });
4384        let parse_result = Schema::parse(&schema)?;
4385        assert_eq!(parse_result, Schema::Uuid(UuidSchema::String));
4386
4387        Ok(())
4388    }
4389
4390    #[test]
4391    fn avro_3926_uuid_schema_for_fixed_with_size_16() -> TestResult {
4392        let schema = json!(
4393        {
4394            "type": "fixed",
4395            "name": "FixedUUID",
4396            "size": 16,
4397            "logicalType": "uuid"
4398        });
4399        let parse_result = Schema::parse(&schema)?;
4400        assert_eq!(
4401            parse_result,
4402            Schema::Uuid(UuidSchema::Fixed(FixedSchema {
4403                name: Name::new("FixedUUID")?,
4404                aliases: None,
4405                doc: None,
4406                size: 16,
4407                attributes: Default::default(),
4408            }))
4409        );
4410        assert_not_logged(
4411            r#"Ignoring uuid logical type for a Fixed schema because its size (6) is not 16! Schema: Fixed(FixedSchema { name: Name { name: "FixedUUID", namespace: None }, aliases: None, doc: None, size: 6, attributes: {"logicalType": String("uuid")} })"#,
4412        );
4413
4414        Ok(())
4415    }
4416
4417    #[test]
4418    fn uuid_schema_bytes() -> TestResult {
4419        let schema = json!(
4420        {
4421          "type": "bytes",
4422          "name": "BytesUUID",
4423          "logicalType": "uuid"
4424        });
4425        let parse_result = Schema::parse(&schema)?;
4426        assert_eq!(parse_result, Schema::Uuid(UuidSchema::Bytes));
4427
4428        Ok(())
4429    }
4430
4431    #[test]
4432    fn avro_3926_uuid_schema_for_fixed_with_size_different_than_16() -> TestResult {
4433        let schema = json!(
4434        {
4435            "type": "fixed",
4436            "name": "FixedUUID",
4437            "size": 6,
4438            "logicalType": "uuid"
4439        });
4440        let parse_result = Schema::parse(&schema)?;
4441
4442        assert_eq!(
4443            parse_result,
4444            Schema::Fixed(FixedSchema {
4445                name: Name::new("FixedUUID")?,
4446                aliases: None,
4447                doc: None,
4448                size: 6,
4449                attributes: BTreeMap::new(),
4450            })
4451        );
4452        assert_logged(
4453            r#"Ignoring uuid logical type for a Fixed schema because its size (6) is not 16! Schema: Fixed(FixedSchema { name: Name { name: "FixedUUID", namespace: None }, aliases: None, doc: None, size: 6, attributes: {} })"#,
4454        );
4455
4456        Ok(())
4457    }
4458
4459    #[test]
4460    fn test_avro_3896_timestamp_millis_schema() -> TestResult {
4461        // long timestamp-millis, represents as native logical type.
4462        let schema = json!(
4463        {
4464          "type": "long",
4465          "name": "LongTimestampMillis",
4466          "logicalType": "timestamp-millis"
4467        });
4468        let parse_result = Schema::parse(&schema)?;
4469        assert_eq!(parse_result, Schema::TimestampMillis);
4470
4471        // int timestamp-millis, represents as native complex type.
4472        let schema = json!(
4473        {
4474            "type": "int",
4475            "name": "IntTimestampMillis",
4476            "logicalType": "timestamp-millis"
4477        });
4478        let parse_result = Schema::parse(&schema)?;
4479        assert_eq!(parse_result, Schema::Int);
4480
4481        Ok(())
4482    }
4483
4484    #[test]
4485    fn test_avro_3896_custom_bytes_schema() -> TestResult {
4486        // log type, represents as complex type.
4487        let schema = json!(
4488        {
4489            "type": "bytes",
4490            "name": "BytesLog",
4491            "logicalType": "custom"
4492        });
4493        let parse_result = Schema::parse(&schema)?;
4494        assert_eq!(parse_result, Schema::Bytes);
4495        assert_eq!(parse_result.custom_attributes(), None);
4496
4497        Ok(())
4498    }
4499
4500    #[test]
4501    fn test_avro_3899_parse_decimal_type() -> TestResult {
4502        let schema = Schema::parse_str(
4503            r#"{
4504             "name": "InvalidDecimal",
4505             "type": "fixed",
4506             "size": 16,
4507             "logicalType": "decimal",
4508             "precision": 2,
4509             "scale": 3
4510         }"#,
4511        )?;
4512        match schema {
4513            Schema::Fixed(fixed_schema) => {
4514                let attrs = fixed_schema.attributes;
4515                let precision = attrs
4516                    .get("precision")
4517                    .expect("The 'precision' attribute is missing");
4518                let scale = attrs
4519                    .get("scale")
4520                    .expect("The 'scale' attribute is missing");
4521                assert_logged(&format!(
4522                    "Ignoring invalid decimal logical type: The decimal precision ({precision}) must be bigger or equal to the scale ({scale})"
4523                ));
4524            }
4525            _ => unreachable!("Expected Schema::Fixed, got {:?}", schema),
4526        }
4527
4528        let schema = Schema::parse_str(
4529            r#"{
4530            "name": "ValidDecimal",
4531             "type": "bytes",
4532             "logicalType": "decimal",
4533             "precision": 3,
4534             "scale": 2
4535         }"#,
4536        )?;
4537        match schema {
4538            Schema::Decimal(_) => {
4539                assert_not_logged(
4540                    "Ignoring invalid decimal logical type: The decimal precision (2) must be bigger or equal to the scale (3)",
4541                );
4542            }
4543            _ => unreachable!("Expected Schema::Decimal, got {:?}", schema),
4544        }
4545
4546        Ok(())
4547    }
4548
4549    #[test]
4550    fn avro_3920_serialize_record_with_custom_attributes() -> TestResult {
4551        let expected = {
4552            let mut lookup = BTreeMap::new();
4553            lookup.insert("value".to_owned(), 0);
4554            Schema::Record(RecordSchema {
4555                name: Name {
4556                    name: "LongList".to_owned(),
4557                    namespace: None,
4558                },
4559                aliases: Some(vec![Alias::new("LinkedLongs").unwrap()]),
4560                doc: None,
4561                fields: vec![RecordField {
4562                    name: "value".to_string(),
4563                    doc: None,
4564                    default: None,
4565                    aliases: None,
4566                    schema: Schema::Long,
4567                    order: RecordFieldOrder::Ascending,
4568                    position: 0,
4569                    custom_attributes: BTreeMap::from([("field-id".to_string(), 1.into())]),
4570                }],
4571                lookup,
4572                attributes: BTreeMap::from([("custom-attribute".to_string(), "value".into())]),
4573            })
4574        };
4575
4576        let value = serde_json::to_value(&expected)?;
4577        let serialized = serde_json::to_string(&value)?;
4578        assert_eq!(
4579            r#"{"aliases":["LinkedLongs"],"custom-attribute":"value","fields":[{"field-id":1,"name":"value","type":"long"}],"name":"LongList","type":"record"}"#,
4580            &serialized
4581        );
4582        assert_eq!(expected, Schema::parse_str(&serialized)?);
4583
4584        Ok(())
4585    }
4586
4587    #[test]
4588    fn test_avro_3925_serialize_decimal_inner_fixed() -> TestResult {
4589        let schema = Schema::Decimal(DecimalSchema {
4590            precision: 36,
4591            scale: 10,
4592            inner: InnerDecimalSchema::Fixed(FixedSchema {
4593                name: Name::new("decimal_36_10").unwrap(),
4594                aliases: None,
4595                doc: None,
4596                size: 16,
4597                attributes: Default::default(),
4598            }),
4599        });
4600
4601        let serialized_json = serde_json::to_string_pretty(&schema)?;
4602
4603        let expected_json = r#"{
4604  "type": "fixed",
4605  "name": "decimal_36_10",
4606  "size": 16,
4607  "logicalType": "decimal",
4608  "scale": 10,
4609  "precision": 36
4610}"#;
4611
4612        assert_eq!(serialized_json, expected_json);
4613
4614        Ok(())
4615    }
4616
4617    #[test]
4618    fn test_avro_3925_serialize_decimal_inner_bytes() -> TestResult {
4619        let schema = Schema::Decimal(DecimalSchema {
4620            precision: 36,
4621            scale: 10,
4622            inner: InnerDecimalSchema::Bytes,
4623        });
4624
4625        let serialized_json = serde_json::to_string_pretty(&schema)?;
4626
4627        let expected_json = r#"{
4628  "type": "bytes",
4629  "logicalType": "decimal",
4630  "scale": 10,
4631  "precision": 36
4632}"#;
4633
4634        assert_eq!(serialized_json, expected_json);
4635
4636        Ok(())
4637    }
4638
4639    #[test]
4640    fn test_avro_3927_serialize_array_with_custom_attributes() -> TestResult {
4641        let expected = Schema::array(Schema::Long)
4642            .attributes(BTreeMap::from([("field-id".to_string(), "1".into())]))
4643            .build();
4644
4645        let value = serde_json::to_value(&expected)?;
4646        let serialized = serde_json::to_string(&value)?;
4647        assert_eq!(
4648            r#"{"field-id":"1","items":"long","type":"array"}"#,
4649            &serialized
4650        );
4651        let actual_schema = Schema::parse_str(&serialized)?;
4652        assert_eq!(expected, actual_schema);
4653        assert_eq!(
4654            expected.custom_attributes(),
4655            actual_schema.custom_attributes()
4656        );
4657
4658        Ok(())
4659    }
4660
4661    #[test]
4662    fn test_avro_3927_serialize_map_with_custom_attributes() -> TestResult {
4663        let expected = Schema::map(Schema::Long)
4664            .attributes(BTreeMap::from([("field-id".to_string(), "1".into())]))
4665            .build();
4666
4667        let value = serde_json::to_value(&expected)?;
4668        let serialized = serde_json::to_string(&value)?;
4669        assert_eq!(
4670            r#"{"field-id":"1","type":"map","values":"long"}"#,
4671            &serialized
4672        );
4673        let actual_schema = Schema::parse_str(&serialized)?;
4674        assert_eq!(expected, actual_schema);
4675        assert_eq!(
4676            expected.custom_attributes(),
4677            actual_schema.custom_attributes()
4678        );
4679
4680        Ok(())
4681    }
4682
4683    #[test]
4684    fn avro_3928_parse_int_based_schema_with_default() -> TestResult {
4685        let schema = r#"
4686        {
4687          "type": "record",
4688          "name": "DateLogicalType",
4689          "fields": [ {
4690            "name": "birthday",
4691            "type": {"type": "int", "logicalType": "date"},
4692            "default": 1681601653
4693          } ]
4694        }"#;
4695
4696        match Schema::parse_str(schema)? {
4697            Schema::Record(record_schema) => {
4698                assert_eq!(record_schema.fields.len(), 1);
4699                let field = record_schema.fields.first().unwrap();
4700                assert_eq!(field.name, "birthday");
4701                assert_eq!(field.schema, Schema::Date);
4702                assert_eq!(
4703                    types::Value::try_from(field.default.clone().unwrap())?,
4704                    types::Value::Int(1681601653)
4705                );
4706            }
4707            _ => unreachable!("Expected Schema::Record"),
4708        }
4709
4710        Ok(())
4711    }
4712
4713    #[test]
4714    fn avro_3946_union_with_single_type() -> TestResult {
4715        let schema = r#"
4716        {
4717          "type": "record",
4718          "name": "Issue",
4719          "namespace": "invalid.example",
4720          "fields": [
4721            {
4722              "name": "myField",
4723              "type": ["long"]
4724            }
4725          ]
4726        }"#;
4727
4728        let _ = Schema::parse_str(schema)?;
4729
4730        assert_logged(
4731            "Union schema with just one member! Consider dropping the union! \
4732                    Please enable debug logging to find out which Record schema \
4733                    declares the union with 'RUST_LOG=apache_avro::schema=debug'.",
4734        );
4735
4736        Ok(())
4737    }
4738
4739    #[test]
4740    fn avro_3946_union_without_any_types() -> TestResult {
4741        let schema = r#"
4742        {
4743          "type": "record",
4744          "name": "Issue",
4745          "namespace": "invalid.example",
4746          "fields": [
4747            {
4748              "name": "myField",
4749              "type": []
4750            }
4751          ]
4752        }"#;
4753
4754        let _ = Schema::parse_str(schema)?;
4755
4756        assert_logged(
4757            "Union schemas should have at least two members! \
4758                    Please enable debug logging to find out which Record schema \
4759                    declares the union with 'RUST_LOG=apache_avro::schema=debug'.",
4760        );
4761
4762        Ok(())
4763    }
4764
4765    #[test]
4766    fn avro_4004_canonical_form_strip_logical_types() -> TestResult {
4767        let schema_str = r#"
4768      {
4769        "type": "record",
4770        "name": "test",
4771        "fields": [
4772            {"name": "a", "type": "long", "default": 42, "doc": "The field a"},
4773            {"name": "b", "type": "string", "namespace": "test.a"},
4774            {"name": "c", "type": "long", "logicalType": "timestamp-micros"}
4775        ]
4776    }"#;
4777
4778        let schema = Schema::parse_str(schema_str)?;
4779        let canonical_form = schema.canonical_form();
4780        let fp_rabin = schema.fingerprint::<Rabin>();
4781        assert_eq!(
4782            r#"{"name":"test","type":"record","fields":[{"name":"a","type":"long"},{"name":"b","type":"string"},{"name":"c","type":{"type":"long"}}]}"#,
4783            canonical_form
4784        );
4785        assert_eq!("92f2ccef718c6754", fp_rabin.to_string());
4786        Ok(())
4787    }
4788
4789    #[test]
4790    fn avro_4055_should_fail_to_parse_invalid_schema() -> TestResult {
4791        // This is invalid because the record type should be inside the type field.
4792        let invalid_schema_str = r#"
4793        {
4794        "type": "record",
4795        "name": "SampleSchema",
4796        "fields": [
4797            {
4798            "name": "order",
4799            "type": "record",
4800            "fields": [
4801                {
4802                "name": "order_number",
4803                "type": ["null", "string"],
4804                "default": null
4805                },
4806                { "name": "order_date", "type": "string" }
4807            ]
4808            }
4809        ]
4810        }"#;
4811
4812        let schema = Schema::parse_str(invalid_schema_str);
4813        assert!(schema.is_err());
4814        assert_eq!(
4815            schema.unwrap_err().to_string(),
4816            "Invalid schema: There is no type called 'record', if you meant to define a non-primitive schema, it should be defined inside `type` attribute. Please review the specification"
4817        );
4818
4819        let valid_schema = r#"
4820        {
4821            "type": "record",
4822            "name": "SampleSchema",
4823            "fields": [
4824                {
4825                "name": "order",
4826                "type": {
4827                    "type": "record",
4828                    "name": "Order",
4829                    "fields": [
4830                    {
4831                        "name": "order_number",
4832                        "type": ["null", "string"],
4833                        "default": null
4834                    },
4835                    { "name": "order_date", "type": "string" }
4836                    ]
4837                }
4838                }
4839            ]
4840        }"#;
4841        let schema = Schema::parse_str(valid_schema);
4842        assert!(schema.is_ok());
4843
4844        Ok(())
4845    }
4846
4847    #[test]
4848    fn avro_rs_292_array_items_should_be_ignored_in_custom_attributes() -> TestResult {
4849        let raw_schema = r#"{
4850                    "type": "array",
4851                    "items": {
4852                        "name": "foo",
4853                        "type": "record",
4854                        "fields": [
4855                            {
4856                                "name": "bar",
4857                                "type": "array",
4858                                "items": {
4859                                    "type": "record",
4860                                    "name": "baz",
4861                                    "fields": [
4862                                        {
4863                                            "name": "quux",
4864                                            "type": "int"
4865                                        }
4866                                    ]
4867                                }
4868                            }
4869                        ]
4870                    }
4871                }"#;
4872
4873        let schema1 = Schema::parse_str(raw_schema)?;
4874        match &schema1 {
4875            Schema::Array(ArraySchema {
4876                items,
4877                default: _,
4878                attributes,
4879            }) => {
4880                assert!(attributes.is_empty());
4881
4882                match **items {
4883                    Schema::Record(RecordSchema {
4884                        ref name,
4885                        aliases: _,
4886                        doc: _,
4887                        ref fields,
4888                        lookup: _,
4889                        ref attributes,
4890                    }) => {
4891                        assert_eq!(name.to_string(), "foo");
4892                        assert_eq!(fields.len(), 1);
4893                        assert!(attributes.is_empty());
4894
4895                        match &fields[0].schema {
4896                            Schema::Array(ArraySchema {
4897                                items: _,
4898                                default: _,
4899                                attributes,
4900                            }) => {
4901                                assert!(attributes.is_empty());
4902                            }
4903                            _ => panic!("Expected ArraySchema. got: {}", &fields[0].schema),
4904                        }
4905                    }
4906                    _ => panic!("Expected RecordSchema. got: {}", &items),
4907                }
4908            }
4909            _ => panic!("Expected ArraySchema. got: {}", &schema1),
4910        }
4911        let canonical_form1 = schema1.canonical_form();
4912        let schema2 = Schema::parse_str(&canonical_form1)?;
4913        let canonical_form2 = schema2.canonical_form();
4914
4915        assert_eq!(canonical_form1, canonical_form2);
4916
4917        Ok(())
4918    }
4919
4920    #[test]
4921    fn avro_rs_292_map_values_should_be_ignored_in_custom_attributes() -> TestResult {
4922        let raw_schema = r#"{
4923                    "type": "array",
4924                    "items": {
4925                        "name": "foo",
4926                        "type": "record",
4927                        "fields": [
4928                            {
4929                                "name": "bar",
4930                                "type": "map",
4931                                "values": {
4932                                    "type": "record",
4933                                    "name": "baz",
4934                                    "fields": [
4935                                        {
4936                                            "name": "quux",
4937                                            "type": "int"
4938                                        }
4939                                    ]
4940                                }
4941                            }
4942                        ]
4943                    }
4944                }"#;
4945
4946        let schema1 = Schema::parse_str(raw_schema)?;
4947        match &schema1 {
4948            Schema::Array(ArraySchema {
4949                items,
4950                default: _,
4951                attributes,
4952            }) => {
4953                assert!(attributes.is_empty());
4954
4955                match **items {
4956                    Schema::Record(RecordSchema {
4957                        ref name,
4958                        aliases: _,
4959                        doc: _,
4960                        ref fields,
4961                        lookup: _,
4962                        ref attributes,
4963                    }) => {
4964                        assert_eq!(name.to_string(), "foo");
4965                        assert_eq!(fields.len(), 1);
4966                        assert!(attributes.is_empty());
4967
4968                        match &fields[0].schema {
4969                            Schema::Map(MapSchema {
4970                                types: _,
4971                                default: _,
4972                                attributes,
4973                            }) => {
4974                                assert!(attributes.is_empty());
4975                            }
4976                            _ => panic!("Expected MapSchema. got: {}", &fields[0].schema),
4977                        }
4978                    }
4979                    _ => panic!("Expected RecordSchema. got: {}", &items),
4980                }
4981            }
4982            _ => panic!("Expected ArraySchema. got: {}", &schema1),
4983        }
4984        let canonical_form1 = schema1.canonical_form();
4985        println!("Canonical Form 1: {}", &canonical_form1);
4986        let schema2 = Schema::parse_str(&canonical_form1)?;
4987        let canonical_form2 = schema2.canonical_form();
4988
4989        assert_eq!(canonical_form1, canonical_form2);
4990
4991        Ok(())
4992    }
4993
4994    #[test]
4995    fn avro_rs_382_serialize_duration_schema() -> TestResult {
4996        let schema = Schema::Duration(FixedSchema {
4997            name: Name::try_from("Duration")?,
4998            aliases: None,
4999            doc: None,
5000            size: 12,
5001            attributes: BTreeMap::new(),
5002        });
5003
5004        let expected_schema_json = json!({
5005            "type": "fixed",
5006            "logicalType": "duration",
5007            "name": "Duration",
5008            "size": 12
5009        });
5010
5011        let schema_json = serde_json::to_value(&schema)?;
5012
5013        assert_eq!(&schema_json, &expected_schema_json);
5014
5015        Ok(())
5016    }
5017
5018    #[test]
5019    fn avro_rs_395_logical_type_written_once_for_duration() -> TestResult {
5020        let schema = Schema::parse_str(
5021            r#"{
5022            "type": "fixed",
5023            "logicalType": "duration",
5024            "name": "Duration",
5025            "size": 12
5026        }"#,
5027        )?;
5028
5029        let schema_json_str = serde_json::to_string(&schema)?;
5030
5031        assert_eq!(
5032            schema_json_str.matches("logicalType").count(),
5033            1,
5034            "Expected serialized schema to contain only one logicalType key: {schema_json_str}"
5035        );
5036
5037        Ok(())
5038    }
5039
5040    #[test]
5041    fn avro_rs_395_logical_type_written_once_for_uuid_fixed() -> TestResult {
5042        let schema = Schema::parse_str(
5043            r#"{
5044            "type": "fixed",
5045            "logicalType": "uuid",
5046            "name": "UUID",
5047            "size": 16
5048        }"#,
5049        )?;
5050
5051        let schema_json_str = serde_json::to_string(&schema)?;
5052
5053        assert_eq!(
5054            schema_json_str.matches("logicalType").count(),
5055            1,
5056            "Expected serialized schema to contain only one logicalType key: {schema_json_str}"
5057        );
5058
5059        Ok(())
5060    }
5061
5062    #[test]
5063    fn avro_rs_395_logical_type_written_once_for_decimal_fixed() -> TestResult {
5064        let schema = Schema::parse_str(
5065            r#"{
5066            "type": "fixed",
5067            "logicalType": "decimal",
5068            "scale": 4,
5069            "precision": 8,
5070            "name": "FixedDecimal16",
5071            "size": 16
5072        }"#,
5073        )?;
5074
5075        let schema_json_str = serde_json::to_string(&schema)?;
5076
5077        assert_eq!(
5078            schema_json_str.matches("logicalType").count(),
5079            1,
5080            "Expected serialized schema to contain only one logicalType key: {schema_json_str}"
5081        );
5082
5083        Ok(())
5084    }
5085
5086    #[test]
5087    fn avro_rs_420_independent_canonical_form() -> TestResult {
5088        let (record, schemata) = Schema::parse_str_with_list(
5089            r#"{
5090            "name": "root",
5091            "type": "record",
5092            "fields": [{
5093                "name": "node",
5094                "type": "node"
5095            }]
5096        }"#,
5097            [r#"{
5098            "name": "node",
5099            "type": "record",
5100            "fields": [{
5101                "name": "children",
5102                "type": ["null", "node"]
5103            }]
5104        }"#],
5105        )?;
5106        let icf = record.independent_canonical_form(&schemata)?;
5107        assert_eq!(
5108            icf,
5109            r#"{"name":"root","type":"record","fields":[{"name":"node","type":{"name":"node","type":"record","fields":[{"name":"children","type":["null","node"]}]}}]}"#
5110        );
5111        Ok(())
5112    }
5113
5114    #[test]
5115    fn avro_rs_456_bool_instead_of_boolean() -> TestResult {
5116        let error = Schema::parse_str(
5117            r#"{
5118            "type": "record",
5119            "name": "defaults",
5120            "fields": [
5121                {"name": "boolean", "type": "bool", "default": true}
5122            ]
5123        }"#,
5124        )
5125        .unwrap_err()
5126        .into_details()
5127        .to_string();
5128        assert_eq!(
5129            error,
5130            Details::ParsePrimitiveSimilar("bool".to_string(), "boolean").to_string()
5131        );
5132
5133        Ok(())
5134    }
5135
5136    #[test]
5137    fn avro_rs_460_fixed_default_in_custom_attributes() -> TestResult {
5138        let schema = Schema::parse_str(
5139            r#"{
5140            "name": "fixed_with_default",
5141            "type": "fixed",
5142            "size": 1,
5143            "default": "\u0000",
5144            "doc": "a docstring"
5145        }"#,
5146        )?;
5147
5148        assert_eq!(schema.custom_attributes().unwrap().len(), 1);
5149
5150        let json = serde_json::to_string(&schema)?;
5151        let schema2 = Schema::parse_str(&json)?;
5152
5153        assert_eq!(schema2.custom_attributes().unwrap().len(), 1);
5154
5155        Ok(())
5156    }
5157
5158    #[test]
5159    fn avro_rs_460_enum_default_not_in_custom_attributes() -> TestResult {
5160        let schema = Schema::parse_str(
5161            r#"{
5162            "name": "enum_with_default",
5163            "type": "enum",
5164            "symbols": ["A", "B", "C"],
5165            "default": "A",
5166            "doc": "a docstring"
5167        }"#,
5168        )?;
5169
5170        assert_eq!(schema.custom_attributes().unwrap(), &BTreeMap::new());
5171
5172        let json = serde_json::to_string(&schema)?;
5173        let schema2 = Schema::parse_str(&json)?;
5174
5175        let Schema::Enum(enum_schema) = schema2 else {
5176            panic!("Expected Schema::Enum, got {schema2:?}");
5177        };
5178        assert!(enum_schema.default.is_some());
5179        assert!(enum_schema.doc.is_some());
5180
5181        Ok(())
5182    }
5183
5184    #[test]
5185    fn avro_rs_467_array_default() -> TestResult {
5186        let schema = Schema::parse_str(
5187            r#"{
5188            "type": "array",
5189            "items": "string",
5190            "default": []
5191        }"#,
5192        )?;
5193
5194        let Schema::Array(array) = schema else {
5195            panic!("Expected Schema::Array, got {schema:?}");
5196        };
5197
5198        assert_eq!(array.attributes, BTreeMap::new());
5199        assert_eq!(array.default, Some(Vec::new()));
5200
5201        let json = serde_json::to_string(&Schema::Array(array))?;
5202        assert!(json.contains(r#""default":[]"#));
5203
5204        Ok(())
5205    }
5206
5207    #[test]
5208    fn avro_rs_467_array_default_with_actual_values() -> TestResult {
5209        let schema = Schema::parse_str(
5210            r#"{
5211            "type": "array",
5212            "items": "string",
5213            "default": ["foo", "bar"]
5214        }"#,
5215        )?;
5216
5217        let Schema::Array(array) = schema else {
5218            panic!("Expected Schema::Array, got {schema:?}");
5219        };
5220
5221        assert_eq!(array.attributes, BTreeMap::new());
5222        assert_eq!(
5223            array.default,
5224            Some(vec![
5225                Value::String("foo".into()),
5226                Value::String("bar".into())
5227            ])
5228        );
5229
5230        let json = serde_json::to_string(&Schema::Array(array))?;
5231        assert!(json.contains(r#""default":["foo","bar"]"#));
5232
5233        Ok(())
5234    }
5235
5236    #[test]
5237    fn avro_rs_467_array_default_with_invalid_values() -> TestResult {
5238        let err = Schema::parse_str(
5239            r#"{
5240            "type": "array",
5241            "items": "string",
5242            "default": [false, true]
5243        }"#,
5244        )
5245        .unwrap_err();
5246
5247        assert_eq!(
5248            err.to_string(),
5249            "Default value for an array must be an array of String! Found: Boolean(false)"
5250        );
5251
5252        Ok(())
5253    }
5254
5255    #[test]
5256    fn avro_rs_467_array_default_with_mixed_values() -> TestResult {
5257        let err = Schema::parse_str(
5258            r#"{
5259            "type": "array",
5260            "items": "string",
5261            "default": ["foo", true]
5262        }"#,
5263        )
5264        .unwrap_err();
5265
5266        assert_eq!(
5267            err.to_string(),
5268            "Default value for an array must be an array of String! Found: Boolean(true)"
5269        );
5270
5271        Ok(())
5272    }
5273
5274    #[test]
5275    fn avro_rs_467_array_default_with_reference() -> TestResult {
5276        let schema = Schema::parse_str(
5277            r#"{
5278            "type": "record",
5279            "name": "Something",
5280            "fields": [
5281                {
5282                    "name": "one",
5283                    "type": "enum",
5284                    "name": "ABC",
5285                    "symbols": ["A", "B", "C"]
5286                },
5287                {
5288                    "name": "two",
5289                    "type": "array",
5290                    "items": "ABC",
5291                    "default": ["A", "B", "C"]
5292                }
5293            ]
5294        }"#,
5295        )?;
5296
5297        let Schema::Record(record) = schema else {
5298            panic!("Expected Schema::Record, got {schema:?}");
5299        };
5300        let Schema::Array(array) = &record.fields[1].schema else {
5301            panic!("Expected Schema::Array, got {:?}", record.fields[1].schema);
5302        };
5303
5304        assert_eq!(array.attributes, BTreeMap::new());
5305        assert_eq!(
5306            array.default,
5307            Some(vec![
5308                Value::String("A".into()),
5309                Value::String("B".into()),
5310                Value::String("C".into())
5311            ])
5312        );
5313
5314        Ok(())
5315    }
5316
5317    #[test]
5318    fn avro_rs_467_map_default() -> TestResult {
5319        let schema = Schema::parse_str(
5320            r#"{
5321            "type": "map",
5322            "values": "string",
5323            "default": {}
5324        }"#,
5325        )?;
5326
5327        let Schema::Map(map) = schema else {
5328            panic!("Expected Schema::Map, got {schema:?}");
5329        };
5330
5331        assert_eq!(map.attributes, BTreeMap::new());
5332        assert_eq!(map.default, Some(HashMap::new()));
5333
5334        let json = serde_json::to_string(&Schema::Map(map))?;
5335        assert!(json.contains(r#""default":{}"#));
5336
5337        Ok(())
5338    }
5339
5340    #[test]
5341    fn avro_rs_467_map_default_with_actual_values() -> TestResult {
5342        let schema = Schema::parse_str(
5343            r#"{
5344            "type": "map",
5345            "values": "string",
5346            "default": {"foo": "bar"}
5347        }"#,
5348        )?;
5349
5350        let Schema::Map(map) = schema else {
5351            panic!("Expected Schema::Map, got {schema:?}");
5352        };
5353
5354        let mut hashmap = HashMap::new();
5355        hashmap.insert("foo".to_string(), Value::String("bar".into()));
5356        assert_eq!(map.attributes, BTreeMap::new());
5357        assert_eq!(map.default, Some(hashmap));
5358
5359        let json = serde_json::to_string(&Schema::Map(map))?;
5360        assert!(json.contains(r#""default":{"foo":"bar"}"#));
5361
5362        Ok(())
5363    }
5364
5365    #[test]
5366    fn avro_rs_467_map_default_with_invalid_values() -> TestResult {
5367        let err = Schema::parse_str(
5368            r#"{
5369            "type": "map",
5370            "values": "string",
5371            "default": {"foo": true}
5372        }"#,
5373        )
5374        .unwrap_err();
5375
5376        assert_eq!(
5377            err.to_string(),
5378            "Default value for a map must be an object with (String, String)! Found: (String, Boolean(true))"
5379        );
5380
5381        Ok(())
5382    }
5383
5384    #[test]
5385    fn avro_rs_467_map_default_with_mixed_values() -> TestResult {
5386        let err = Schema::parse_str(
5387            r#"{
5388            "type": "map",
5389            "values": "string",
5390            "default": {"foo": "bar", "spam": true}
5391        }"#,
5392        )
5393        .unwrap_err();
5394
5395        assert_eq!(
5396            err.to_string(),
5397            "Default value for a map must be an object with (String, String)! Found: (String, Boolean(true))"
5398        );
5399
5400        Ok(())
5401    }
5402
5403    #[test]
5404    fn avro_rs_467_map_default_with_reference() -> TestResult {
5405        let schema = Schema::parse_str(
5406            r#"{
5407            "type": "record",
5408            "name": "Something",
5409            "fields": [
5410                {
5411                    "name": "one",
5412                    "type": "enum",
5413                    "name": "ABC",
5414                    "symbols": ["A", "B", "C"]
5415                },
5416                {
5417                    "name": "two",
5418                    "type": "map",
5419                    "values": "ABC",
5420                    "default": {"foo": "A"}
5421                }
5422            ]
5423        }"#,
5424        )?;
5425
5426        let Schema::Record(record) = schema else {
5427            panic!("Expected Schema::Record, got {schema:?}");
5428        };
5429        let Schema::Map(map) = &record.fields[1].schema else {
5430            panic!("Expected Schema::Map, got {:?}", record.fields[1].schema);
5431        };
5432
5433        let mut hashmap = HashMap::new();
5434        hashmap.insert("foo".to_string(), Value::String("A".into()));
5435        assert_eq!(map.attributes, BTreeMap::new());
5436        assert_eq!(map.default, Some(hashmap));
5437
5438        Ok(())
5439    }
5440}