apache_avro/
schema.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.
19use crate::{
20    error::Error,
21    schema_equality, types,
22    util::MapHelper,
23    validator::{
24        validate_enum_symbol_name, validate_namespace, validate_record_field_name,
25        validate_schema_name,
26    },
27    AvroResult,
28};
29use digest::Digest;
30use serde::{
31    ser::{SerializeMap, SerializeSeq},
32    Deserialize, Serialize, Serializer,
33};
34use serde_json::{Map, Value};
35use std::{
36    borrow::{Borrow, Cow},
37    collections::{BTreeMap, HashMap, HashSet},
38    fmt,
39    fmt::Debug,
40    hash::Hash,
41    io::Read,
42    str::FromStr,
43};
44use strum_macros::{Display, EnumDiscriminants, EnumString};
45
46/// Represents an Avro schema fingerprint
47/// More information about Avro schema fingerprints can be found in the
48/// [Avro Schema Fingerprint documentation](https://avro.apache.org/docs/current/spec.html#schema_fingerprints)
49pub struct SchemaFingerprint {
50    pub bytes: Vec<u8>,
51}
52
53impl fmt::Display for SchemaFingerprint {
54    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
55        write!(
56            f,
57            "{}",
58            self.bytes
59                .iter()
60                .map(|byte| format!("{byte:02x}"))
61                .collect::<Vec<String>>()
62                .join("")
63        )
64    }
65}
66
67/// Represents any valid Avro schema
68/// More information about Avro schemas can be found in the
69/// [Avro Specification](https://avro.apache.org/docs/current/spec.html#schemas)
70#[derive(Clone, Debug, EnumDiscriminants, Display)]
71#[strum_discriminants(name(SchemaKind), derive(Hash, Ord, PartialOrd))]
72pub enum Schema {
73    /// A `null` Avro schema.
74    Null,
75    /// A `boolean` Avro schema.
76    Boolean,
77    /// An `int` Avro schema.
78    Int,
79    /// A `long` Avro schema.
80    Long,
81    /// A `float` Avro schema.
82    Float,
83    /// A `double` Avro schema.
84    Double,
85    /// A `bytes` Avro schema.
86    /// `Bytes` represents a sequence of 8-bit unsigned bytes.
87    Bytes,
88    /// A `string` Avro schema.
89    /// `String` represents a unicode character sequence.
90    String,
91    /// A `array` Avro schema. Avro arrays are required to have the same type for each element.
92    /// This variant holds the `Schema` for the array element type.
93    Array(ArraySchema),
94    /// A `map` Avro schema.
95    /// `Map` holds a pointer to the `Schema` of its values, which must all be the same schema.
96    /// `Map` keys are assumed to be `string`.
97    Map(MapSchema),
98    /// A `union` Avro schema.
99    Union(UnionSchema),
100    /// A `record` Avro schema.
101    Record(RecordSchema),
102    /// An `enum` Avro schema.
103    Enum(EnumSchema),
104    /// A `fixed` Avro schema.
105    Fixed(FixedSchema),
106    /// Logical type which represents `Decimal` values. The underlying type is serialized and
107    /// deserialized as `Schema::Bytes` or `Schema::Fixed`.
108    Decimal(DecimalSchema),
109    /// Logical type which represents `Decimal` values without predefined scale.
110    /// The underlying type is serialized and deserialized as `Schema::Bytes`
111    BigDecimal,
112    /// A universally unique identifier, annotating a string.
113    Uuid,
114    /// Logical type which represents the number of days since the unix epoch.
115    /// Serialization format is `Schema::Int`.
116    Date,
117    /// The time of day in number of milliseconds after midnight with no reference any calendar,
118    /// time zone or date in particular.
119    TimeMillis,
120    /// The time of day in number of microseconds after midnight with no reference any calendar,
121    /// time zone or date in particular.
122    TimeMicros,
123    /// An instant in time represented as the number of milliseconds after the UNIX epoch.
124    TimestampMillis,
125    /// An instant in time represented as the number of microseconds after the UNIX epoch.
126    TimestampMicros,
127    /// An instant in time represented as the number of nanoseconds after the UNIX epoch.
128    TimestampNanos,
129    /// An instant in localtime represented as the number of milliseconds after the UNIX epoch.
130    LocalTimestampMillis,
131    /// An instant in local time represented as the number of microseconds after the UNIX epoch.
132    LocalTimestampMicros,
133    /// An instant in local time represented as the number of nanoseconds after the UNIX epoch.
134    LocalTimestampNanos,
135    /// An amount of time defined by a number of months, days and milliseconds.
136    Duration,
137    /// A reference to another schema.
138    Ref { name: Name },
139}
140
141#[derive(Clone, Debug, PartialEq)]
142pub struct MapSchema {
143    pub types: Box<Schema>,
144    pub attributes: BTreeMap<String, Value>,
145}
146
147#[derive(Clone, Debug, PartialEq)]
148pub struct ArraySchema {
149    pub items: Box<Schema>,
150    pub attributes: BTreeMap<String, Value>,
151}
152
153impl PartialEq for Schema {
154    /// Assess equality of two `Schema` based on [Parsing Canonical Form].
155    ///
156    /// [Parsing Canonical Form]:
157    /// https://avro.apache.org/docs/1.11.1/specification/#parsing-canonical-form-for-schemas
158    fn eq(&self, other: &Self) -> bool {
159        schema_equality::compare_schemata(self, other)
160    }
161}
162
163impl SchemaKind {
164    pub fn is_primitive(self) -> bool {
165        matches!(
166            self,
167            SchemaKind::Null
168                | SchemaKind::Boolean
169                | SchemaKind::Int
170                | SchemaKind::Long
171                | SchemaKind::Double
172                | SchemaKind::Float
173                | SchemaKind::Bytes
174                | SchemaKind::String,
175        )
176    }
177
178    pub fn is_named(self) -> bool {
179        matches!(
180            self,
181            SchemaKind::Record | SchemaKind::Enum | SchemaKind::Fixed | SchemaKind::Ref
182        )
183    }
184}
185
186impl From<&types::Value> for SchemaKind {
187    fn from(value: &types::Value) -> Self {
188        use crate::types::Value;
189        match value {
190            Value::Null => Self::Null,
191            Value::Boolean(_) => Self::Boolean,
192            Value::Int(_) => Self::Int,
193            Value::Long(_) => Self::Long,
194            Value::Float(_) => Self::Float,
195            Value::Double(_) => Self::Double,
196            Value::Bytes(_) => Self::Bytes,
197            Value::String(_) => Self::String,
198            Value::Array(_) => Self::Array,
199            Value::Map(_) => Self::Map,
200            Value::Union(_, _) => Self::Union,
201            Value::Record(_) => Self::Record,
202            Value::Enum(_, _) => Self::Enum,
203            Value::Fixed(_, _) => Self::Fixed,
204            Value::Decimal { .. } => Self::Decimal,
205            Value::BigDecimal(_) => Self::BigDecimal,
206            Value::Uuid(_) => Self::Uuid,
207            Value::Date(_) => Self::Date,
208            Value::TimeMillis(_) => Self::TimeMillis,
209            Value::TimeMicros(_) => Self::TimeMicros,
210            Value::TimestampMillis(_) => Self::TimestampMillis,
211            Value::TimestampMicros(_) => Self::TimestampMicros,
212            Value::TimestampNanos(_) => Self::TimestampNanos,
213            Value::LocalTimestampMillis(_) => Self::LocalTimestampMillis,
214            Value::LocalTimestampMicros(_) => Self::LocalTimestampMicros,
215            Value::LocalTimestampNanos(_) => Self::LocalTimestampNanos,
216            Value::Duration { .. } => Self::Duration,
217        }
218    }
219}
220
221/// Represents names for `record`, `enum` and `fixed` Avro schemas.
222///
223/// Each of these `Schema`s have a `fullname` composed of two parts:
224///   * a name
225///   * a namespace
226///
227/// `aliases` can also be defined, to facilitate schema evolution.
228///
229/// More information about schema names can be found in the
230/// [Avro specification](https://avro.apache.org/docs/current/spec.html#names)
231#[derive(Clone, Debug, Hash, PartialEq, Eq)]
232pub struct Name {
233    pub name: String,
234    pub namespace: Namespace,
235}
236
237/// Represents documentation for complex Avro schemas.
238pub type Documentation = Option<String>;
239/// Represents the aliases for Named Schema
240pub type Aliases = Option<Vec<Alias>>;
241/// Represents Schema lookup within a schema env
242pub(crate) type Names = HashMap<Name, Schema>;
243/// Represents Schema lookup within a schema
244pub type NamesRef<'a> = HashMap<Name, &'a Schema>;
245/// Represents the namespace for Named Schema
246pub type Namespace = Option<String>;
247
248impl Name {
249    /// Create a new `Name`.
250    /// Parses the optional `namespace` from the `name` string.
251    /// `aliases` will not be defined.
252    pub fn new(name: &str) -> AvroResult<Self> {
253        let (name, namespace) = Name::get_name_and_namespace(name)?;
254        Ok(Self {
255            name,
256            namespace: namespace.filter(|ns| !ns.is_empty()),
257        })
258    }
259
260    fn get_name_and_namespace(name: &str) -> AvroResult<(String, Namespace)> {
261        validate_schema_name(name)
262    }
263
264    /// Parse a `serde_json::Value` into a `Name`.
265    pub(crate) fn parse(
266        complex: &Map<String, Value>,
267        enclosing_namespace: &Namespace,
268    ) -> AvroResult<Self> {
269        let (name, namespace_from_name) = complex
270            .name()
271            .map(|name| Name::get_name_and_namespace(name.as_str()).unwrap())
272            .ok_or(Error::GetNameField)?;
273        // FIXME Reading name from the type is wrong ! The name there is just a metadata (AVRO-3430)
274        let type_name = match complex.get("type") {
275            Some(Value::Object(complex_type)) => complex_type.name().or(None),
276            _ => None,
277        };
278
279        let namespace = namespace_from_name
280            .or_else(|| {
281                complex
282                    .string("namespace")
283                    .or_else(|| enclosing_namespace.clone())
284            })
285            .filter(|ns| !ns.is_empty());
286
287        if let Some(ref ns) = namespace {
288            validate_namespace(ns)?;
289        }
290
291        Ok(Self {
292            name: type_name.unwrap_or(name),
293            namespace,
294        })
295    }
296
297    /// Return the `fullname` of this `Name`
298    ///
299    /// More information about fullnames can be found in the
300    /// [Avro specification](https://avro.apache.org/docs/current/spec.html#names)
301    pub fn fullname(&self, default_namespace: Namespace) -> String {
302        if self.name.contains('.') {
303            self.name.clone()
304        } else {
305            let namespace = self.namespace.clone().or(default_namespace);
306
307            match namespace {
308                Some(ref namespace) if !namespace.is_empty() => {
309                    format!("{}.{}", namespace, self.name)
310                }
311                _ => self.name.clone(),
312            }
313        }
314    }
315
316    /// Return the fully qualified name needed for indexing or searching for the schema within a schema/schema env context. Puts the enclosing namespace into the name's namespace for clarity in schema/schema env parsing
317    /// ```ignore
318    /// use apache_avro::schema::Name;
319    ///
320    /// assert_eq!(
321    /// Name::new("some_name")?.fully_qualified_name(&Some("some_namespace".into())),
322    /// Name::new("some_namespace.some_name")?
323    /// );
324    /// assert_eq!(
325    /// Name::new("some_namespace.some_name")?.fully_qualified_name(&Some("other_namespace".into())),
326    /// Name::new("some_namespace.some_name")?
327    /// );
328    /// ```
329    pub fn fully_qualified_name(&self, enclosing_namespace: &Namespace) -> Name {
330        Name {
331            name: self.name.clone(),
332            namespace: self
333                .namespace
334                .clone()
335                .or_else(|| enclosing_namespace.clone().filter(|ns| !ns.is_empty())),
336        }
337    }
338}
339
340impl From<&str> for Name {
341    fn from(name: &str) -> Self {
342        Name::new(name).unwrap()
343    }
344}
345
346impl fmt::Display for Name {
347    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
348        f.write_str(&self.fullname(None)[..])
349    }
350}
351
352impl<'de> Deserialize<'de> for Name {
353    fn deserialize<D>(deserializer: D) -> Result<Self, D::Error>
354    where
355        D: serde::de::Deserializer<'de>,
356    {
357        Value::deserialize(deserializer).and_then(|value| {
358            use serde::de::Error;
359            if let Value::Object(json) = value {
360                Name::parse(&json, &None).map_err(Error::custom)
361            } else {
362                Err(Error::custom(format!("Expected a JSON object: {value:?}")))
363            }
364        })
365    }
366}
367
368/// Newtype pattern for `Name` to better control the `serde_json::Value` representation.
369/// Aliases are serialized as an array of plain strings in the JSON representation.
370#[derive(Clone, Debug, Hash, PartialEq, Eq)]
371pub struct Alias(Name);
372
373impl Alias {
374    pub fn new(name: &str) -> AvroResult<Self> {
375        Name::new(name).map(Self)
376    }
377
378    pub fn name(&self) -> String {
379        self.0.name.clone()
380    }
381
382    pub fn namespace(&self) -> Namespace {
383        self.0.namespace.clone()
384    }
385
386    pub fn fullname(&self, default_namespace: Namespace) -> String {
387        self.0.fullname(default_namespace)
388    }
389
390    pub fn fully_qualified_name(&self, default_namespace: &Namespace) -> Name {
391        self.0.fully_qualified_name(default_namespace)
392    }
393}
394
395impl From<&str> for Alias {
396    fn from(name: &str) -> Self {
397        Alias::new(name).unwrap()
398    }
399}
400
401impl Serialize for Alias {
402    fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
403    where
404        S: Serializer,
405    {
406        serializer.serialize_str(&self.fullname(None))
407    }
408}
409
410#[derive(Debug)]
411pub struct ResolvedSchema<'s> {
412    names_ref: NamesRef<'s>,
413    schemata: Vec<&'s Schema>,
414}
415
416impl<'s> TryFrom<&'s Schema> for ResolvedSchema<'s> {
417    type Error = Error;
418
419    fn try_from(schema: &'s Schema) -> AvroResult<Self> {
420        let names = HashMap::new();
421        let mut rs = ResolvedSchema {
422            names_ref: names,
423            schemata: vec![schema],
424        };
425        rs.resolve(rs.get_schemata(), &None, None)?;
426        Ok(rs)
427    }
428}
429
430impl<'s> TryFrom<Vec<&'s Schema>> for ResolvedSchema<'s> {
431    type Error = Error;
432
433    fn try_from(schemata: Vec<&'s Schema>) -> AvroResult<Self> {
434        let names = HashMap::new();
435        let mut rs = ResolvedSchema {
436            names_ref: names,
437            schemata,
438        };
439        rs.resolve(rs.get_schemata(), &None, None)?;
440        Ok(rs)
441    }
442}
443
444impl<'s> ResolvedSchema<'s> {
445    pub fn get_schemata(&self) -> Vec<&'s Schema> {
446        self.schemata.clone()
447    }
448
449    pub fn get_names(&self) -> &NamesRef<'s> {
450        &self.names_ref
451    }
452
453    /// Creates `ResolvedSchema` with some already known schemas.
454    ///
455    /// Those schemata would be used to resolve references if needed.
456    pub fn new_with_known_schemata<'n>(
457        schemata_to_resolve: Vec<&'s Schema>,
458        enclosing_namespace: &Namespace,
459        known_schemata: &'n NamesRef<'n>,
460    ) -> AvroResult<Self> {
461        let names = HashMap::new();
462        let mut rs = ResolvedSchema {
463            names_ref: names,
464            schemata: schemata_to_resolve,
465        };
466        rs.resolve(rs.get_schemata(), enclosing_namespace, Some(known_schemata))?;
467        Ok(rs)
468    }
469
470    fn resolve<'n>(
471        &mut self,
472        schemata: Vec<&'s Schema>,
473        enclosing_namespace: &Namespace,
474        known_schemata: Option<&'n NamesRef<'n>>,
475    ) -> AvroResult<()> {
476        for schema in schemata {
477            match schema {
478                Schema::Array(schema) => {
479                    self.resolve(vec![&schema.items], enclosing_namespace, known_schemata)?
480                }
481                Schema::Map(schema) => {
482                    self.resolve(vec![&schema.types], enclosing_namespace, known_schemata)?
483                }
484                Schema::Union(UnionSchema { schemas, .. }) => {
485                    for schema in schemas {
486                        self.resolve(vec![schema], enclosing_namespace, known_schemata)?
487                    }
488                }
489                Schema::Enum(EnumSchema { name, .. }) | Schema::Fixed(FixedSchema { name, .. }) => {
490                    let fully_qualified_name = name.fully_qualified_name(enclosing_namespace);
491                    if self
492                        .names_ref
493                        .insert(fully_qualified_name.clone(), schema)
494                        .is_some()
495                    {
496                        return Err(Error::AmbiguousSchemaDefinition(fully_qualified_name));
497                    }
498                }
499                Schema::Record(RecordSchema { name, fields, .. }) => {
500                    let fully_qualified_name = name.fully_qualified_name(enclosing_namespace);
501                    if self
502                        .names_ref
503                        .insert(fully_qualified_name.clone(), schema)
504                        .is_some()
505                    {
506                        return Err(Error::AmbiguousSchemaDefinition(fully_qualified_name));
507                    } else {
508                        let record_namespace = fully_qualified_name.namespace;
509                        for field in fields {
510                            self.resolve(vec![&field.schema], &record_namespace, known_schemata)?
511                        }
512                    }
513                }
514                Schema::Ref { name } => {
515                    let fully_qualified_name = name.fully_qualified_name(enclosing_namespace);
516                    // first search for reference in current schemata, then look into external references.
517                    if !self.names_ref.contains_key(&fully_qualified_name) {
518                        let is_resolved_with_known_schemas = known_schemata
519                            .as_ref()
520                            .map(|names| names.contains_key(&fully_qualified_name))
521                            .unwrap_or(false);
522                        if !is_resolved_with_known_schemas {
523                            return Err(Error::SchemaResolutionError(fully_qualified_name));
524                        }
525                    }
526                }
527                _ => (),
528            }
529        }
530        Ok(())
531    }
532}
533
534pub(crate) struct ResolvedOwnedSchema {
535    names: Names,
536    root_schema: Schema,
537}
538
539impl TryFrom<Schema> for ResolvedOwnedSchema {
540    type Error = Error;
541
542    fn try_from(schema: Schema) -> AvroResult<Self> {
543        let names = HashMap::new();
544        let mut rs = ResolvedOwnedSchema {
545            names,
546            root_schema: schema,
547        };
548        resolve_names(&rs.root_schema, &mut rs.names, &None)?;
549        Ok(rs)
550    }
551}
552
553impl ResolvedOwnedSchema {
554    pub(crate) fn get_root_schema(&self) -> &Schema {
555        &self.root_schema
556    }
557    pub(crate) fn get_names(&self) -> &Names {
558        &self.names
559    }
560}
561
562pub(crate) fn resolve_names(
563    schema: &Schema,
564    names: &mut Names,
565    enclosing_namespace: &Namespace,
566) -> AvroResult<()> {
567    match schema {
568        Schema::Array(schema) => resolve_names(&schema.items, names, enclosing_namespace),
569        Schema::Map(schema) => resolve_names(&schema.types, names, enclosing_namespace),
570        Schema::Union(UnionSchema { schemas, .. }) => {
571            for schema in schemas {
572                resolve_names(schema, names, enclosing_namespace)?
573            }
574            Ok(())
575        }
576        Schema::Enum(EnumSchema { name, .. }) | Schema::Fixed(FixedSchema { name, .. }) => {
577            let fully_qualified_name = name.fully_qualified_name(enclosing_namespace);
578            if names
579                .insert(fully_qualified_name.clone(), schema.clone())
580                .is_some()
581            {
582                Err(Error::AmbiguousSchemaDefinition(fully_qualified_name))
583            } else {
584                Ok(())
585            }
586        }
587        Schema::Record(RecordSchema { name, fields, .. }) => {
588            let fully_qualified_name = name.fully_qualified_name(enclosing_namespace);
589            if names
590                .insert(fully_qualified_name.clone(), schema.clone())
591                .is_some()
592            {
593                Err(Error::AmbiguousSchemaDefinition(fully_qualified_name))
594            } else {
595                let record_namespace = fully_qualified_name.namespace;
596                for field in fields {
597                    resolve_names(&field.schema, names, &record_namespace)?
598                }
599                Ok(())
600            }
601        }
602        Schema::Ref { name } => {
603            let fully_qualified_name = name.fully_qualified_name(enclosing_namespace);
604            names
605                .get(&fully_qualified_name)
606                .map(|_| ())
607                .ok_or(Error::SchemaResolutionError(fully_qualified_name))
608        }
609        _ => Ok(()),
610    }
611}
612
613pub(crate) fn resolve_names_with_schemata(
614    schemata: &Vec<&Schema>,
615    names: &mut Names,
616    enclosing_namespace: &Namespace,
617) -> AvroResult<()> {
618    for schema in schemata {
619        resolve_names(schema, names, enclosing_namespace)?;
620    }
621    Ok(())
622}
623
624/// Represents a `field` in a `record` Avro schema.
625#[derive(Clone, Debug, PartialEq)]
626pub struct RecordField {
627    /// Name of the field.
628    pub name: String,
629    /// Documentation of the field.
630    pub doc: Documentation,
631    /// Aliases of the field's name. They have no namespace.
632    pub aliases: Option<Vec<String>>,
633    /// Default value of the field.
634    /// This value will be used when reading Avro datum if schema resolution
635    /// is enabled.
636    pub default: Option<Value>,
637    /// Schema of the field.
638    pub schema: Schema,
639    /// Order of the field.
640    ///
641    /// **NOTE** This currently has no effect.
642    pub order: RecordFieldOrder,
643    /// Position of the field in the list of `field` of its parent `Schema`
644    pub position: usize,
645    /// A collection of all unknown fields in the record field.
646    pub custom_attributes: BTreeMap<String, Value>,
647}
648
649/// Represents any valid order for a `field` in a `record` Avro schema.
650#[derive(Clone, Debug, Eq, PartialEq, EnumString)]
651#[strum(serialize_all = "kebab_case")]
652pub enum RecordFieldOrder {
653    Ascending,
654    Descending,
655    Ignore,
656}
657
658impl RecordField {
659    /// Parse a `serde_json::Value` into a `RecordField`.
660    fn parse(
661        field: &Map<String, Value>,
662        position: usize,
663        parser: &mut Parser,
664        enclosing_record: &Name,
665    ) -> AvroResult<Self> {
666        let name = field.name().ok_or(Error::GetNameFieldFromRecord)?;
667
668        validate_record_field_name(&name)?;
669
670        // TODO: "type" = "<record name>"
671        let schema = parser.parse_complex(field, &enclosing_record.namespace)?;
672
673        let default = field.get("default").cloned();
674        Self::resolve_default_value(
675            &schema,
676            &name,
677            &enclosing_record.fullname(None),
678            &parser.parsed_schemas,
679            &default,
680        )?;
681
682        let aliases = field.get("aliases").and_then(|aliases| {
683            aliases.as_array().map(|aliases| {
684                aliases
685                    .iter()
686                    .flat_map(|alias| alias.as_str())
687                    .map(|alias| alias.to_string())
688                    .collect::<Vec<String>>()
689            })
690        });
691
692        let order = field
693            .get("order")
694            .and_then(|order| order.as_str())
695            .and_then(|order| RecordFieldOrder::from_str(order).ok())
696            .unwrap_or(RecordFieldOrder::Ascending);
697
698        Ok(RecordField {
699            name,
700            doc: field.doc(),
701            default,
702            aliases,
703            order,
704            position,
705            custom_attributes: RecordField::get_field_custom_attributes(field, &schema),
706            schema,
707        })
708    }
709
710    fn resolve_default_value(
711        field_schema: &Schema,
712        field_name: &str,
713        record_name: &str,
714        names: &Names,
715        default: &Option<Value>,
716    ) -> AvroResult<()> {
717        if let Some(value) = default {
718            let avro_value = types::Value::from(value.clone());
719            match field_schema {
720                Schema::Union(union_schema) => {
721                    let schemas = &union_schema.schemas;
722                    let resolved = schemas.iter().any(|schema| {
723                        avro_value
724                            .to_owned()
725                            .resolve_internal(schema, names, &schema.namespace(), &None)
726                            .is_ok()
727                    });
728
729                    if !resolved {
730                        let schema: Option<&Schema> = schemas.first();
731                        return match schema {
732                            Some(first_schema) => Err(Error::GetDefaultUnion(
733                                SchemaKind::from(first_schema),
734                                types::ValueKind::from(avro_value),
735                            )),
736                            None => Err(Error::EmptyUnion),
737                        };
738                    }
739                }
740                _ => {
741                    let resolved = avro_value
742                        .resolve_internal(field_schema, names, &field_schema.namespace(), &None)
743                        .is_ok();
744
745                    if !resolved {
746                        return Err(Error::GetDefaultRecordField(
747                            field_name.to_string(),
748                            record_name.to_string(),
749                            field_schema.canonical_form(),
750                        ));
751                    }
752                }
753            };
754        }
755
756        Ok(())
757    }
758
759    fn get_field_custom_attributes(
760        field: &Map<String, Value>,
761        schema: &Schema,
762    ) -> BTreeMap<String, Value> {
763        let mut custom_attributes: BTreeMap<String, Value> = BTreeMap::new();
764        for (key, value) in field {
765            match key.as_str() {
766                "type" | "name" | "doc" | "default" | "order" | "position" | "aliases"
767                | "logicalType" => continue,
768                key if key == "symbols" && matches!(schema, Schema::Enum(_)) => continue,
769                key if key == "size" && matches!(schema, Schema::Fixed(_)) => continue,
770                _ => custom_attributes.insert(key.clone(), value.clone()),
771            };
772        }
773        custom_attributes
774    }
775
776    /// Returns true if this `RecordField` is nullable, meaning the schema is a `UnionSchema` where the first variant is `Null`.
777    pub fn is_nullable(&self) -> bool {
778        match self.schema {
779            Schema::Union(ref inner) => inner.is_nullable(),
780            _ => false,
781        }
782    }
783}
784
785/// A description of an Enum schema.
786#[derive(Debug, Clone)]
787pub struct RecordSchema {
788    /// The name of the schema
789    pub name: Name,
790    /// The aliases of the schema
791    pub aliases: Aliases,
792    /// The documentation of the schema
793    pub doc: Documentation,
794    /// The set of fields of the schema
795    pub fields: Vec<RecordField>,
796    /// The `lookup` table maps field names to their position in the `Vec`
797    /// of `fields`.
798    pub lookup: BTreeMap<String, usize>,
799    /// The custom attributes of the schema
800    pub attributes: BTreeMap<String, Value>,
801}
802
803/// A description of an Enum schema.
804#[derive(Debug, Clone)]
805pub struct EnumSchema {
806    /// The name of the schema
807    pub name: Name,
808    /// The aliases of the schema
809    pub aliases: Aliases,
810    /// The documentation of the schema
811    pub doc: Documentation,
812    /// The set of symbols of the schema
813    pub symbols: Vec<String>,
814    /// An optional default symbol used for compatibility
815    pub default: Option<String>,
816    /// The custom attributes of the schema
817    pub attributes: BTreeMap<String, Value>,
818}
819
820/// A description of a Union schema.
821#[derive(Debug, Clone)]
822pub struct FixedSchema {
823    /// The name of the schema
824    pub name: Name,
825    /// The aliases of the schema
826    pub aliases: Aliases,
827    /// The documentation of the schema
828    pub doc: Documentation,
829    /// The size of the fixed schema
830    pub size: usize,
831    /// An optional default symbol used for compatibility
832    pub default: Option<String>,
833    /// The custom attributes of the schema
834    pub attributes: BTreeMap<String, Value>,
835}
836
837impl FixedSchema {
838    fn serialize_to_map<S>(&self, mut map: S::SerializeMap) -> Result<S::SerializeMap, S::Error>
839    where
840        S: Serializer,
841    {
842        map.serialize_entry("type", "fixed")?;
843        if let Some(ref n) = self.name.namespace {
844            map.serialize_entry("namespace", n)?;
845        }
846        map.serialize_entry("name", &self.name.name)?;
847        if let Some(ref docstr) = self.doc {
848            map.serialize_entry("doc", docstr)?;
849        }
850        map.serialize_entry("size", &self.size)?;
851
852        if let Some(ref aliases) = self.aliases {
853            map.serialize_entry("aliases", aliases)?;
854        }
855
856        for attr in &self.attributes {
857            map.serialize_entry(attr.0, attr.1)?;
858        }
859
860        Ok(map)
861    }
862}
863
864/// A description of a Union schema.
865///
866/// `scale` defaults to 0 and is an integer greater than or equal to 0 and `precision` is an
867/// integer greater than 0.
868#[derive(Debug, Clone)]
869pub struct DecimalSchema {
870    /// The number of digits in the unscaled value
871    pub precision: DecimalMetadata,
872    /// The number of digits to the right of the decimal point
873    pub scale: DecimalMetadata,
874    /// The inner schema of the decimal (fixed or bytes)
875    pub inner: Box<Schema>,
876}
877
878/// A description of a Union schema
879#[derive(Debug, Clone)]
880pub struct UnionSchema {
881    /// The schemas that make up this union
882    pub(crate) schemas: Vec<Schema>,
883    // Used to ensure uniqueness of schema inputs, and provide constant time finding of the
884    // schema index given a value.
885    // **NOTE** that this approach does not work for named types, and will have to be modified
886    // to support that. A simple solution is to also keep a mapping of the names used.
887    variant_index: BTreeMap<SchemaKind, usize>,
888}
889
890impl UnionSchema {
891    /// Creates a new UnionSchema from a vector of schemas.
892    pub fn new(schemas: Vec<Schema>) -> AvroResult<Self> {
893        let mut vindex = BTreeMap::new();
894        for (i, schema) in schemas.iter().enumerate() {
895            if let Schema::Union(_) = schema {
896                return Err(Error::GetNestedUnion);
897            }
898            let kind = SchemaKind::from(schema);
899            if !kind.is_named() && vindex.insert(kind, i).is_some() {
900                return Err(Error::GetUnionDuplicate);
901            }
902        }
903        Ok(UnionSchema {
904            schemas,
905            variant_index: vindex,
906        })
907    }
908
909    /// Returns a slice to all variants of this schema.
910    pub fn variants(&self) -> &[Schema] {
911        &self.schemas
912    }
913
914    /// Returns true if the any of the variants of this `UnionSchema` is `Null`.
915    pub fn is_nullable(&self) -> bool {
916        self.schemas.iter().any(|x| matches!(x, Schema::Null))
917    }
918
919    /// Optionally returns a reference to the schema matched by this value, as well as its position
920    /// within this union.
921    #[deprecated(
922        since = "0.15.0",
923        note = "Please use `find_schema_with_known_schemata` instead"
924    )]
925    pub fn find_schema(&self, value: &types::Value) -> Option<(usize, &Schema)> {
926        self.find_schema_with_known_schemata::<Schema>(value, None, &None)
927    }
928
929    /// Optionally returns a reference to the schema matched by this value, as well as its position
930    /// within this union.
931    ///
932    /// Extra arguments:
933    /// - `known_schemata` - mapping between `Name` and `Schema` - if passed, additional external schemas would be used to resolve references.
934    pub fn find_schema_with_known_schemata<S: Borrow<Schema> + Debug>(
935        &self,
936        value: &types::Value,
937        known_schemata: Option<&HashMap<Name, S>>,
938        enclosing_namespace: &Namespace,
939    ) -> Option<(usize, &Schema)> {
940        let schema_kind = SchemaKind::from(value);
941        if let Some(&i) = self.variant_index.get(&schema_kind) {
942            // fast path
943            Some((i, &self.schemas[i]))
944        } else {
945            // slow path (required for matching logical or named types)
946
947            // first collect what schemas we already know
948            let mut collected_names: HashMap<Name, &Schema> = known_schemata
949                .map(|names| {
950                    names
951                        .iter()
952                        .map(|(name, schema)| (name.clone(), schema.borrow()))
953                        .collect()
954                })
955                .unwrap_or_default();
956
957            self.schemas.iter().enumerate().find(|(_, schema)| {
958                let resolved_schema = ResolvedSchema::new_with_known_schemata(
959                    vec![*schema],
960                    enclosing_namespace,
961                    &collected_names,
962                )
963                .expect("Schema didn't successfully parse");
964                let resolved_names = resolved_schema.names_ref;
965
966                // extend known schemas with just resolved names
967                collected_names.extend(resolved_names);
968                let namespace = &schema.namespace().or_else(|| enclosing_namespace.clone());
969
970                value
971                    .clone()
972                    .resolve_internal(schema, &collected_names, namespace, &None)
973                    .is_ok()
974            })
975        }
976    }
977}
978
979// No need to compare variant_index, it is derivative of schemas.
980impl PartialEq for UnionSchema {
981    fn eq(&self, other: &UnionSchema) -> bool {
982        self.schemas.eq(&other.schemas)
983    }
984}
985
986type DecimalMetadata = usize;
987pub(crate) type Precision = DecimalMetadata;
988pub(crate) type Scale = DecimalMetadata;
989
990fn parse_json_integer_for_decimal(value: &serde_json::Number) -> Result<DecimalMetadata, Error> {
991    Ok(if value.is_u64() {
992        let num = value
993            .as_u64()
994            .ok_or_else(|| Error::GetU64FromJson(value.clone()))?;
995        num.try_into()
996            .map_err(|e| Error::ConvertU64ToUsize(e, num))?
997    } else if value.is_i64() {
998        let num = value
999            .as_i64()
1000            .ok_or_else(|| Error::GetI64FromJson(value.clone()))?;
1001        num.try_into()
1002            .map_err(|e| Error::ConvertI64ToUsize(e, num))?
1003    } else {
1004        return Err(Error::GetPrecisionOrScaleFromJson(value.clone()));
1005    })
1006}
1007
1008#[derive(Default)]
1009struct Parser {
1010    input_schemas: HashMap<Name, Value>,
1011    /// A map of name -> Schema::Ref
1012    /// Used to resolve cyclic references, i.e. when a
1013    /// field's type is a reference to its record's type
1014    resolving_schemas: Names,
1015    input_order: Vec<Name>,
1016    /// A map of name -> fully parsed Schema
1017    /// Used to avoid parsing the same schema twice
1018    parsed_schemas: Names,
1019}
1020
1021impl Schema {
1022    /// Converts `self` into its [Parsing Canonical Form].
1023    ///
1024    /// [Parsing Canonical Form]:
1025    /// https://avro.apache.org/docs/1.8.2/spec.html#Parsing+Canonical+Form+for+Schemas
1026    pub fn canonical_form(&self) -> String {
1027        let json = serde_json::to_value(self)
1028            .unwrap_or_else(|e| panic!("Cannot parse Schema from JSON: {e}"));
1029        parsing_canonical_form(&json)
1030    }
1031
1032    /// Generate [fingerprint] of Schema's [Parsing Canonical Form].
1033    ///
1034    /// [Parsing Canonical Form]:
1035    /// https://avro.apache.org/docs/1.8.2/spec.html#Parsing+Canonical+Form+for+Schemas
1036    /// [fingerprint]:
1037    /// https://avro.apache.org/docs/current/spec.html#schema_fingerprints
1038    pub fn fingerprint<D: Digest>(&self) -> SchemaFingerprint {
1039        let mut d = D::new();
1040        d.update(self.canonical_form());
1041        SchemaFingerprint {
1042            bytes: d.finalize().to_vec(),
1043        }
1044    }
1045
1046    /// Create a `Schema` from a string representing a JSON Avro schema.
1047    pub fn parse_str(input: &str) -> Result<Schema, Error> {
1048        let mut parser = Parser::default();
1049        parser.parse_str(input)
1050    }
1051
1052    /// Create a array of `Schema`'s from a list of named JSON Avro schemas (Record, Enum, and
1053    /// Fixed).
1054    ///
1055    /// It is allowed that the schemas have cross-dependencies; these will be resolved
1056    /// during parsing.
1057    ///
1058    /// If two of the input schemas have the same fullname, an Error will be returned.
1059    pub fn parse_list(input: &[&str]) -> AvroResult<Vec<Schema>> {
1060        let mut input_schemas: HashMap<Name, Value> = HashMap::with_capacity(input.len());
1061        let mut input_order: Vec<Name> = Vec::with_capacity(input.len());
1062        for js in input {
1063            let schema: Value = serde_json::from_str(js).map_err(Error::ParseSchemaJson)?;
1064            if let Value::Object(inner) = &schema {
1065                let name = Name::parse(inner, &None)?;
1066                let previous_value = input_schemas.insert(name.clone(), schema);
1067                if previous_value.is_some() {
1068                    return Err(Error::NameCollision(name.fullname(None)));
1069                }
1070                input_order.push(name);
1071            } else {
1072                return Err(Error::GetNameField);
1073            }
1074        }
1075        let mut parser = Parser {
1076            input_schemas,
1077            resolving_schemas: HashMap::default(),
1078            input_order,
1079            parsed_schemas: HashMap::with_capacity(input.len()),
1080        };
1081        parser.parse_list()
1082    }
1083
1084    /// Create a `Schema` from a reader which implements [`Read`].
1085    pub fn parse_reader(reader: &mut (impl Read + ?Sized)) -> AvroResult<Schema> {
1086        let mut buf = String::new();
1087        match reader.read_to_string(&mut buf) {
1088            Ok(_) => Self::parse_str(&buf),
1089            Err(e) => Err(Error::ReadSchemaFromReader(e)),
1090        }
1091    }
1092
1093    /// Parses an Avro schema from JSON.
1094    pub fn parse(value: &Value) -> AvroResult<Schema> {
1095        let mut parser = Parser::default();
1096        parser.parse(value, &None)
1097    }
1098
1099    /// Parses an Avro schema from JSON.
1100    /// Any `Schema::Ref`s must be known in the `names` map.
1101    pub(crate) fn parse_with_names(value: &Value, names: Names) -> AvroResult<Schema> {
1102        let mut parser = Parser {
1103            input_schemas: HashMap::with_capacity(1),
1104            resolving_schemas: Names::default(),
1105            input_order: Vec::with_capacity(1),
1106            parsed_schemas: names,
1107        };
1108        parser.parse(value, &None)
1109    }
1110
1111    /// Returns the custom attributes (metadata) if the schema supports them.
1112    pub fn custom_attributes(&self) -> Option<&BTreeMap<String, Value>> {
1113        match self {
1114            Schema::Record(RecordSchema { attributes, .. })
1115            | Schema::Enum(EnumSchema { attributes, .. })
1116            | Schema::Fixed(FixedSchema { attributes, .. })
1117            | Schema::Array(ArraySchema { attributes, .. })
1118            | Schema::Map(MapSchema { attributes, .. }) => Some(attributes),
1119            _ => None,
1120        }
1121    }
1122
1123    /// Returns the name of the schema if it has one.
1124    pub fn name(&self) -> Option<&Name> {
1125        match self {
1126            Schema::Ref { name, .. }
1127            | Schema::Record(RecordSchema { name, .. })
1128            | Schema::Enum(EnumSchema { name, .. })
1129            | Schema::Fixed(FixedSchema { name, .. }) => Some(name),
1130            _ => None,
1131        }
1132    }
1133
1134    /// Returns the namespace of the schema if it has one.
1135    pub fn namespace(&self) -> Namespace {
1136        self.name().and_then(|n| n.namespace.clone())
1137    }
1138
1139    /// Returns the aliases of the schema if it has ones.
1140    pub fn aliases(&self) -> Option<&Vec<Alias>> {
1141        match self {
1142            Schema::Record(RecordSchema { aliases, .. })
1143            | Schema::Enum(EnumSchema { aliases, .. })
1144            | Schema::Fixed(FixedSchema { aliases, .. }) => aliases.as_ref(),
1145            _ => None,
1146        }
1147    }
1148
1149    /// Returns the doc of the schema if it has one.
1150    pub fn doc(&self) -> Option<&String> {
1151        match self {
1152            Schema::Record(RecordSchema { doc, .. })
1153            | Schema::Enum(EnumSchema { doc, .. })
1154            | Schema::Fixed(FixedSchema { doc, .. }) => doc.as_ref(),
1155            _ => None,
1156        }
1157    }
1158
1159    /// Returns a Schema::Map with the given types.
1160    pub fn map(types: Schema) -> Self {
1161        Schema::Map(MapSchema {
1162            types: Box::new(types),
1163            attributes: Default::default(),
1164        })
1165    }
1166
1167    /// Returns a Schema::Map with the given types and custom attributes.
1168    pub fn map_with_attributes(types: Schema, attributes: BTreeMap<String, Value>) -> Self {
1169        Schema::Map(MapSchema {
1170            types: Box::new(types),
1171            attributes,
1172        })
1173    }
1174
1175    /// Returns a Schema::Array with the given items.
1176    pub fn array(items: Schema) -> Self {
1177        Schema::Array(ArraySchema {
1178            items: Box::new(items),
1179            attributes: Default::default(),
1180        })
1181    }
1182
1183    /// Returns a Schema::Array with the given items and custom attributes.
1184    pub fn array_with_attributes(items: Schema, attributes: BTreeMap<String, Value>) -> Self {
1185        Schema::Array(ArraySchema {
1186            items: Box::new(items),
1187            attributes,
1188        })
1189    }
1190}
1191
1192impl Parser {
1193    /// Create a `Schema` from a string representing a JSON Avro schema.
1194    fn parse_str(&mut self, input: &str) -> Result<Schema, Error> {
1195        let value = serde_json::from_str(input).map_err(Error::ParseSchemaJson)?;
1196        self.parse(&value, &None)
1197    }
1198
1199    /// Create an array of `Schema`'s from an iterator of JSON Avro schemas. It is allowed that
1200    /// the schemas have cross-dependencies; these will be resolved during parsing.
1201    fn parse_list(&mut self) -> Result<Vec<Schema>, Error> {
1202        while !self.input_schemas.is_empty() {
1203            let next_name = self
1204                .input_schemas
1205                .keys()
1206                .next()
1207                .expect("Input schemas unexpectedly empty")
1208                .to_owned();
1209            let (name, value) = self
1210                .input_schemas
1211                .remove_entry(&next_name)
1212                .expect("Key unexpectedly missing");
1213            let parsed = self.parse(&value, &None)?;
1214            self.parsed_schemas
1215                .insert(get_schema_type_name(name, value), parsed);
1216        }
1217
1218        let mut parsed_schemas = Vec::with_capacity(self.parsed_schemas.len());
1219        for name in self.input_order.drain(0..) {
1220            let parsed = self
1221                .parsed_schemas
1222                .remove(&name)
1223                .expect("One of the input schemas was unexpectedly not parsed");
1224            parsed_schemas.push(parsed);
1225        }
1226        Ok(parsed_schemas)
1227    }
1228
1229    /// Create a `Schema` from a `serde_json::Value` representing a JSON Avro
1230    /// schema.
1231    fn parse(&mut self, value: &Value, enclosing_namespace: &Namespace) -> AvroResult<Schema> {
1232        match *value {
1233            Value::String(ref t) => self.parse_known_schema(t.as_str(), enclosing_namespace),
1234            Value::Object(ref data) => self.parse_complex(data, enclosing_namespace),
1235            Value::Array(ref data) => self.parse_union(data, enclosing_namespace),
1236            _ => Err(Error::ParseSchemaFromValidJson),
1237        }
1238    }
1239
1240    /// Parse a `serde_json::Value` representing an Avro type whose Schema is known into a
1241    /// `Schema`. A Schema for a `serde_json::Value` is known if it is primitive or has
1242    /// been parsed previously by the parsed and stored in its map of parsed_schemas.
1243    fn parse_known_schema(
1244        &mut self,
1245        name: &str,
1246        enclosing_namespace: &Namespace,
1247    ) -> AvroResult<Schema> {
1248        match name {
1249            "null" => Ok(Schema::Null),
1250            "boolean" => Ok(Schema::Boolean),
1251            "int" => Ok(Schema::Int),
1252            "long" => Ok(Schema::Long),
1253            "double" => Ok(Schema::Double),
1254            "float" => Ok(Schema::Float),
1255            "bytes" => Ok(Schema::Bytes),
1256            "string" => Ok(Schema::String),
1257            _ => self.fetch_schema_ref(name, enclosing_namespace),
1258        }
1259    }
1260
1261    /// Given a name, tries to retrieve the parsed schema from `parsed_schemas`.
1262    /// If a parsed schema is not found, it checks if a currently resolving
1263    /// schema with that name exists.
1264    /// If a resolving schema is not found, it checks if a json with that name exists
1265    /// in `input_schemas` and then parses it (removing it from `input_schemas`)
1266    /// and adds the parsed schema to `parsed_schemas`.
1267    ///
1268    /// This method allows schemas definitions that depend on other types to
1269    /// parse their dependencies (or look them up if already parsed).
1270    fn fetch_schema_ref(
1271        &mut self,
1272        name: &str,
1273        enclosing_namespace: &Namespace,
1274    ) -> AvroResult<Schema> {
1275        fn get_schema_ref(parsed: &Schema) -> Schema {
1276            match &parsed {
1277                Schema::Record(RecordSchema { ref name, .. })
1278                | Schema::Enum(EnumSchema { ref name, .. })
1279                | Schema::Fixed(FixedSchema { ref name, .. }) => Schema::Ref { name: name.clone() },
1280                _ => parsed.clone(),
1281            }
1282        }
1283
1284        let name = Name::new(name)?;
1285        let fully_qualified_name = name.fully_qualified_name(enclosing_namespace);
1286
1287        if self.parsed_schemas.contains_key(&fully_qualified_name) {
1288            return Ok(Schema::Ref {
1289                name: fully_qualified_name,
1290            });
1291        }
1292        if let Some(resolving_schema) = self.resolving_schemas.get(&fully_qualified_name) {
1293            return Ok(resolving_schema.clone());
1294        }
1295
1296        let value = self
1297            .input_schemas
1298            .remove(&fully_qualified_name)
1299            // TODO make a better descriptive error message here that conveys that a named schema cannot be found
1300            .ok_or_else(|| Error::ParsePrimitive(fully_qualified_name.fullname(None)))?;
1301
1302        // parsing a full schema from inside another schema. Other full schema will not inherit namespace
1303        let parsed = self.parse(&value, &None)?;
1304        self.parsed_schemas
1305            .insert(get_schema_type_name(name, value), parsed.clone());
1306
1307        Ok(get_schema_ref(&parsed))
1308    }
1309
1310    fn parse_precision_and_scale(
1311        complex: &Map<String, Value>,
1312    ) -> Result<(Precision, Scale), Error> {
1313        fn get_decimal_integer(
1314            complex: &Map<String, Value>,
1315            key: &'static str,
1316        ) -> Result<DecimalMetadata, Error> {
1317            match complex.get(key) {
1318                Some(Value::Number(value)) => parse_json_integer_for_decimal(value),
1319                None => {
1320                    if key == "scale" {
1321                        Ok(0)
1322                    } else {
1323                        Err(Error::GetDecimalMetadataFromJson(key))
1324                    }
1325                }
1326                Some(value) => Err(Error::GetDecimalMetadataValueFromJson {
1327                    key: key.into(),
1328                    value: value.clone(),
1329                }),
1330            }
1331        }
1332        let precision = get_decimal_integer(complex, "precision")?;
1333        let scale = get_decimal_integer(complex, "scale")?;
1334
1335        if precision < 1 {
1336            return Err(Error::DecimalPrecisionMuBePositive { precision });
1337        }
1338
1339        if precision < scale {
1340            Err(Error::DecimalPrecisionLessThanScale { precision, scale })
1341        } else {
1342            Ok((precision, scale))
1343        }
1344    }
1345
1346    /// Parse a `serde_json::Value` representing a complex Avro type into a
1347    /// `Schema`.
1348    ///
1349    /// Avro supports "recursive" definition of types.
1350    /// e.g: {"type": {"type": "string"}}
1351    fn parse_complex(
1352        &mut self,
1353        complex: &Map<String, Value>,
1354        enclosing_namespace: &Namespace,
1355    ) -> AvroResult<Schema> {
1356        // Try to parse this as a native complex type.
1357        fn parse_as_native_complex(
1358            complex: &Map<String, Value>,
1359            parser: &mut Parser,
1360            enclosing_namespace: &Namespace,
1361        ) -> AvroResult<Schema> {
1362            match complex.get("type") {
1363                Some(value) => match value {
1364                    Value::String(s) if s == "fixed" => {
1365                        parser.parse_fixed(complex, enclosing_namespace)
1366                    }
1367                    _ => parser.parse(value, enclosing_namespace),
1368                },
1369                None => Err(Error::GetLogicalTypeField),
1370            }
1371        }
1372
1373        // This crate support some logical types natively, and this function tries to convert
1374        // a native complex type with a logical type attribute to these logical types.
1375        // This function:
1376        // 1. Checks whether the native complex type is in the supported kinds.
1377        // 2. If it is, using the convert function to convert the native complex type to
1378        // a logical type.
1379        fn try_convert_to_logical_type<F>(
1380            logical_type: &str,
1381            schema: Schema,
1382            supported_schema_kinds: &[SchemaKind],
1383            convert: F,
1384        ) -> AvroResult<Schema>
1385        where
1386            F: Fn(Schema) -> AvroResult<Schema>,
1387        {
1388            let kind = SchemaKind::from(schema.clone());
1389            if supported_schema_kinds.contains(&kind) {
1390                convert(schema)
1391            } else {
1392                warn!(
1393                    "Ignoring unknown logical type '{}' for schema of type: {:?}!",
1394                    logical_type, schema
1395                );
1396                Ok(schema)
1397            }
1398        }
1399
1400        match complex.get("logicalType") {
1401            Some(Value::String(t)) => match t.as_str() {
1402                "decimal" => {
1403                    return try_convert_to_logical_type(
1404                        "decimal",
1405                        parse_as_native_complex(complex, self, enclosing_namespace)?,
1406                        &[SchemaKind::Fixed, SchemaKind::Bytes],
1407                        |inner| -> AvroResult<Schema> {
1408                            match Self::parse_precision_and_scale(complex) {
1409                                Ok((precision, scale)) => Ok(Schema::Decimal(DecimalSchema {
1410                                    precision,
1411                                    scale,
1412                                    inner: Box::new(inner),
1413                                })),
1414                                Err(err) => {
1415                                    warn!("Ignoring invalid decimal logical type: {}", err);
1416                                    Ok(inner)
1417                                }
1418                            }
1419                        },
1420                    );
1421                }
1422                "big-decimal" => {
1423                    return try_convert_to_logical_type(
1424                        "big-decimal",
1425                        parse_as_native_complex(complex, self, enclosing_namespace)?,
1426                        &[SchemaKind::Bytes],
1427                        |_| -> AvroResult<Schema> { Ok(Schema::BigDecimal) },
1428                    );
1429                }
1430                "uuid" => {
1431                    return try_convert_to_logical_type(
1432                        "uuid",
1433                        parse_as_native_complex(complex, self, enclosing_namespace)?,
1434                        &[SchemaKind::String, SchemaKind::Fixed],
1435                        |schema| match schema {
1436                            Schema::String => Ok(Schema::Uuid),
1437                            Schema::Fixed(FixedSchema { size: 16, .. }) => Ok(Schema::Uuid),
1438                            Schema::Fixed(FixedSchema { size, .. }) => {
1439                                warn!("Ignoring uuid logical type for a Fixed schema because its size ({size:?}) is not 16! Schema: {:?}", schema);
1440                                Ok(schema)
1441                            }
1442                            _ => {
1443                                warn!(
1444                                    "Ignoring invalid uuid logical type for schema: {:?}",
1445                                    schema
1446                                );
1447                                Ok(schema)
1448                            }
1449                        },
1450                    );
1451                }
1452                "date" => {
1453                    return try_convert_to_logical_type(
1454                        "date",
1455                        parse_as_native_complex(complex, self, enclosing_namespace)?,
1456                        &[SchemaKind::Int],
1457                        |_| -> AvroResult<Schema> { Ok(Schema::Date) },
1458                    );
1459                }
1460                "time-millis" => {
1461                    return try_convert_to_logical_type(
1462                        "date",
1463                        parse_as_native_complex(complex, self, enclosing_namespace)?,
1464                        &[SchemaKind::Int],
1465                        |_| -> AvroResult<Schema> { Ok(Schema::TimeMillis) },
1466                    );
1467                }
1468                "time-micros" => {
1469                    return try_convert_to_logical_type(
1470                        "time-micros",
1471                        parse_as_native_complex(complex, self, enclosing_namespace)?,
1472                        &[SchemaKind::Long],
1473                        |_| -> AvroResult<Schema> { Ok(Schema::TimeMicros) },
1474                    );
1475                }
1476                "timestamp-millis" => {
1477                    return try_convert_to_logical_type(
1478                        "timestamp-millis",
1479                        parse_as_native_complex(complex, self, enclosing_namespace)?,
1480                        &[SchemaKind::Long],
1481                        |_| -> AvroResult<Schema> { Ok(Schema::TimestampMillis) },
1482                    );
1483                }
1484                "timestamp-micros" => {
1485                    return try_convert_to_logical_type(
1486                        "timestamp-micros",
1487                        parse_as_native_complex(complex, self, enclosing_namespace)?,
1488                        &[SchemaKind::Long],
1489                        |_| -> AvroResult<Schema> { Ok(Schema::TimestampMicros) },
1490                    );
1491                }
1492                "local-timestamp-millis" => {
1493                    return try_convert_to_logical_type(
1494                        "local-timestamp-millis",
1495                        parse_as_native_complex(complex, self, enclosing_namespace)?,
1496                        &[SchemaKind::Long],
1497                        |_| -> AvroResult<Schema> { Ok(Schema::LocalTimestampMillis) },
1498                    );
1499                }
1500                "local-timestamp-micros" => {
1501                    return try_convert_to_logical_type(
1502                        "local-timestamp-micros",
1503                        parse_as_native_complex(complex, self, enclosing_namespace)?,
1504                        &[SchemaKind::Long],
1505                        |_| -> AvroResult<Schema> { Ok(Schema::LocalTimestampMicros) },
1506                    );
1507                }
1508                "duration" => {
1509                    return try_convert_to_logical_type(
1510                        "duration",
1511                        parse_as_native_complex(complex, self, enclosing_namespace)?,
1512                        &[SchemaKind::Fixed],
1513                        |_| -> AvroResult<Schema> { Ok(Schema::Duration) },
1514                    );
1515                }
1516                // In this case, of an unknown logical type, we just pass through the underlying
1517                // type.
1518                _ => {}
1519            },
1520            // The spec says to ignore invalid logical types and just pass through the
1521            // underlying type. It is unclear whether that applies to this case or not, where the
1522            // `logicalType` is not a string.
1523            Some(value) => return Err(Error::GetLogicalTypeFieldType(value.clone())),
1524            _ => {}
1525        }
1526        match complex.get("type") {
1527            Some(Value::String(t)) => match t.as_str() {
1528                "record" => self.parse_record(complex, enclosing_namespace),
1529                "enum" => self.parse_enum(complex, enclosing_namespace),
1530                "array" => self.parse_array(complex, enclosing_namespace),
1531                "map" => self.parse_map(complex, enclosing_namespace),
1532                "fixed" => self.parse_fixed(complex, enclosing_namespace),
1533                other => self.parse_known_schema(other, enclosing_namespace),
1534            },
1535            Some(Value::Object(data)) => self.parse_complex(data, enclosing_namespace),
1536            Some(Value::Array(variants)) => self.parse_union(variants, enclosing_namespace),
1537            Some(unknown) => Err(Error::GetComplexType(unknown.clone())),
1538            None => Err(Error::GetComplexTypeField),
1539        }
1540    }
1541
1542    fn register_resolving_schema(&mut self, name: &Name, aliases: &Aliases) {
1543        let resolving_schema = Schema::Ref { name: name.clone() };
1544        self.resolving_schemas
1545            .insert(name.clone(), resolving_schema.clone());
1546
1547        let namespace = &name.namespace;
1548
1549        if let Some(ref aliases) = aliases {
1550            aliases.iter().for_each(|alias| {
1551                let alias_fullname = alias.fully_qualified_name(namespace);
1552                self.resolving_schemas
1553                    .insert(alias_fullname, resolving_schema.clone());
1554            });
1555        }
1556    }
1557
1558    fn register_parsed_schema(
1559        &mut self,
1560        fully_qualified_name: &Name,
1561        schema: &Schema,
1562        aliases: &Aliases,
1563    ) {
1564        // FIXME, this should be globally aware, so if there is something overwriting something
1565        // else then there is an ambiguous schema definition. An appropriate error should be thrown
1566        self.parsed_schemas
1567            .insert(fully_qualified_name.clone(), schema.clone());
1568        self.resolving_schemas.remove(fully_qualified_name);
1569
1570        let namespace = &fully_qualified_name.namespace;
1571
1572        if let Some(ref aliases) = aliases {
1573            aliases.iter().for_each(|alias| {
1574                let alias_fullname = alias.fully_qualified_name(namespace);
1575                self.resolving_schemas.remove(&alias_fullname);
1576                self.parsed_schemas.insert(alias_fullname, schema.clone());
1577            });
1578        }
1579    }
1580
1581    /// Returns already parsed schema or a schema that is currently being resolved.
1582    fn get_already_seen_schema(
1583        &self,
1584        complex: &Map<String, Value>,
1585        enclosing_namespace: &Namespace,
1586    ) -> Option<&Schema> {
1587        match complex.get("type") {
1588            Some(Value::String(ref typ)) => {
1589                let name = Name::new(typ.as_str())
1590                    .unwrap()
1591                    .fully_qualified_name(enclosing_namespace);
1592                self.resolving_schemas
1593                    .get(&name)
1594                    .or_else(|| self.parsed_schemas.get(&name))
1595            }
1596            _ => None,
1597        }
1598    }
1599
1600    /// Parse a `serde_json::Value` representing a Avro record type into a
1601    /// `Schema`.
1602    fn parse_record(
1603        &mut self,
1604        complex: &Map<String, Value>,
1605        enclosing_namespace: &Namespace,
1606    ) -> AvroResult<Schema> {
1607        let fields_opt = complex.get("fields");
1608
1609        if fields_opt.is_none() {
1610            if let Some(seen) = self.get_already_seen_schema(complex, enclosing_namespace) {
1611                return Ok(seen.clone());
1612            }
1613        }
1614
1615        let fully_qualified_name = Name::parse(complex, enclosing_namespace)?;
1616        let aliases = fix_aliases_namespace(complex.aliases(), &fully_qualified_name.namespace);
1617
1618        let mut lookup = BTreeMap::new();
1619
1620        self.register_resolving_schema(&fully_qualified_name, &aliases);
1621
1622        debug!("Going to parse record schema: {:?}", &fully_qualified_name);
1623
1624        let fields: Vec<RecordField> = fields_opt
1625            .and_then(|fields| fields.as_array())
1626            .ok_or(Error::GetRecordFieldsJson)
1627            .and_then(|fields| {
1628                fields
1629                    .iter()
1630                    .filter_map(|field| field.as_object())
1631                    .enumerate()
1632                    .map(|(position, field)| {
1633                        RecordField::parse(field, position, self, &fully_qualified_name)
1634                    })
1635                    .collect::<Result<_, _>>()
1636            })?;
1637
1638        for field in &fields {
1639            if let Some(_old) = lookup.insert(field.name.clone(), field.position) {
1640                return Err(Error::FieldNameDuplicate(field.name.clone()));
1641            }
1642
1643            if let Some(ref field_aliases) = field.aliases {
1644                for alias in field_aliases {
1645                    lookup.insert(alias.clone(), field.position);
1646                }
1647            }
1648        }
1649
1650        let schema = Schema::Record(RecordSchema {
1651            name: fully_qualified_name.clone(),
1652            aliases: aliases.clone(),
1653            doc: complex.doc(),
1654            fields,
1655            lookup,
1656            attributes: self.get_custom_attributes(complex, vec!["fields"]),
1657        });
1658
1659        self.register_parsed_schema(&fully_qualified_name, &schema, &aliases);
1660        Ok(schema)
1661    }
1662
1663    fn get_custom_attributes(
1664        &self,
1665        complex: &Map<String, Value>,
1666        excluded: Vec<&'static str>,
1667    ) -> BTreeMap<String, Value> {
1668        let mut custom_attributes: BTreeMap<String, Value> = BTreeMap::new();
1669        for (key, value) in complex {
1670            match key.as_str() {
1671                "type" | "name" | "namespace" | "doc" | "aliases" => continue,
1672                candidate if excluded.contains(&candidate) => continue,
1673                _ => custom_attributes.insert(key.clone(), value.clone()),
1674            };
1675        }
1676        custom_attributes
1677    }
1678
1679    /// Parse a `serde_json::Value` representing a Avro enum type into a
1680    /// `Schema`.
1681    fn parse_enum(
1682        &mut self,
1683        complex: &Map<String, Value>,
1684        enclosing_namespace: &Namespace,
1685    ) -> AvroResult<Schema> {
1686        let symbols_opt = complex.get("symbols");
1687
1688        if symbols_opt.is_none() {
1689            if let Some(seen) = self.get_already_seen_schema(complex, enclosing_namespace) {
1690                return Ok(seen.clone());
1691            }
1692        }
1693
1694        let fully_qualified_name = Name::parse(complex, enclosing_namespace)?;
1695        let aliases = fix_aliases_namespace(complex.aliases(), &fully_qualified_name.namespace);
1696
1697        let symbols: Vec<String> = symbols_opt
1698            .and_then(|v| v.as_array())
1699            .ok_or(Error::GetEnumSymbolsField)
1700            .and_then(|symbols| {
1701                symbols
1702                    .iter()
1703                    .map(|symbol| symbol.as_str().map(|s| s.to_string()))
1704                    .collect::<Option<_>>()
1705                    .ok_or(Error::GetEnumSymbols)
1706            })?;
1707
1708        let mut existing_symbols: HashSet<&String> = HashSet::with_capacity(symbols.len());
1709        for symbol in symbols.iter() {
1710            validate_enum_symbol_name(symbol)?;
1711
1712            // Ensure there are no duplicate symbols
1713            if existing_symbols.contains(&symbol) {
1714                return Err(Error::EnumSymbolDuplicate(symbol.to_string()));
1715            }
1716
1717            existing_symbols.insert(symbol);
1718        }
1719
1720        let mut default: Option<String> = None;
1721        if let Some(value) = complex.get("default") {
1722            if let Value::String(ref s) = *value {
1723                default = Some(s.clone());
1724            } else {
1725                return Err(Error::EnumDefaultWrongType(value.clone()));
1726            }
1727        }
1728
1729        if let Some(ref value) = default {
1730            let resolved = types::Value::from(value.clone())
1731                .resolve_enum(&symbols, &Some(value.to_string()), &None)
1732                .is_ok();
1733            if !resolved {
1734                return Err(Error::GetEnumDefault {
1735                    symbol: value.to_string(),
1736                    symbols,
1737                });
1738            }
1739        }
1740
1741        let schema = Schema::Enum(EnumSchema {
1742            name: fully_qualified_name.clone(),
1743            aliases: aliases.clone(),
1744            doc: complex.doc(),
1745            symbols,
1746            default,
1747            attributes: self.get_custom_attributes(complex, vec!["symbols"]),
1748        });
1749
1750        self.register_parsed_schema(&fully_qualified_name, &schema, &aliases);
1751
1752        Ok(schema)
1753    }
1754
1755    /// Parse a `serde_json::Value` representing a Avro array type into a
1756    /// `Schema`.
1757    fn parse_array(
1758        &mut self,
1759        complex: &Map<String, Value>,
1760        enclosing_namespace: &Namespace,
1761    ) -> AvroResult<Schema> {
1762        complex
1763            .get("items")
1764            .ok_or(Error::GetArrayItemsField)
1765            .and_then(|items| self.parse(items, enclosing_namespace))
1766            .map(|items| {
1767                Schema::array_with_attributes(
1768                    items,
1769                    self.get_custom_attributes(complex, vec!["items"]),
1770                )
1771            })
1772    }
1773
1774    /// Parse a `serde_json::Value` representing a Avro map type into a
1775    /// `Schema`.
1776    fn parse_map(
1777        &mut self,
1778        complex: &Map<String, Value>,
1779        enclosing_namespace: &Namespace,
1780    ) -> AvroResult<Schema> {
1781        complex
1782            .get("values")
1783            .ok_or(Error::GetMapValuesField)
1784            .and_then(|items| self.parse(items, enclosing_namespace))
1785            .map(|items| {
1786                Schema::map_with_attributes(
1787                    items,
1788                    self.get_custom_attributes(complex, vec!["values"]),
1789                )
1790            })
1791    }
1792
1793    /// Parse a `serde_json::Value` representing a Avro union type into a
1794    /// `Schema`.
1795    fn parse_union(
1796        &mut self,
1797        items: &[Value],
1798        enclosing_namespace: &Namespace,
1799    ) -> AvroResult<Schema> {
1800        items
1801            .iter()
1802            .map(|v| self.parse(v, enclosing_namespace))
1803            .collect::<Result<Vec<_>, _>>()
1804            .and_then(|schemas| {
1805                if schemas.is_empty() {
1806                    error!(
1807                        "Union schemas should have at least two members! \
1808                    Please enable debug logging to find out which Record schema \
1809                    declares the union with 'RUST_LOG=apache_avro::schema=debug'."
1810                    );
1811                } else if schemas.len() == 1 {
1812                    warn!(
1813                        "Union schema with just one member! Consider dropping the union! \
1814                    Please enable debug logging to find out which Record schema \
1815                    declares the union with 'RUST_LOG=apache_avro::schema=debug'."
1816                    );
1817                }
1818                Ok(Schema::Union(UnionSchema::new(schemas)?))
1819            })
1820    }
1821
1822    /// Parse a `serde_json::Value` representing a Avro fixed type into a
1823    /// `Schema`.
1824    fn parse_fixed(
1825        &mut self,
1826        complex: &Map<String, Value>,
1827        enclosing_namespace: &Namespace,
1828    ) -> AvroResult<Schema> {
1829        let size_opt = complex.get("size");
1830        if size_opt.is_none() {
1831            if let Some(seen) = self.get_already_seen_schema(complex, enclosing_namespace) {
1832                return Ok(seen.clone());
1833            }
1834        }
1835
1836        let doc = complex.get("doc").and_then(|v| match &v {
1837            Value::String(ref docstr) => Some(docstr.clone()),
1838            _ => None,
1839        });
1840
1841        let size = match size_opt {
1842            Some(size) => size
1843                .as_u64()
1844                .ok_or_else(|| Error::GetFixedSizeFieldPositive(size.clone())),
1845            None => Err(Error::GetFixedSizeField),
1846        }?;
1847
1848        let default = complex.get("default").and_then(|v| match &v {
1849            Value::String(ref default) => Some(default.clone()),
1850            _ => None,
1851        });
1852
1853        if default.is_some() {
1854            let len = default.clone().unwrap().len();
1855            if len != size as usize {
1856                return Err(Error::FixedDefaultLenSizeMismatch(len, size));
1857            }
1858        }
1859
1860        let fully_qualified_name = Name::parse(complex, enclosing_namespace)?;
1861        let aliases = fix_aliases_namespace(complex.aliases(), &fully_qualified_name.namespace);
1862
1863        let schema = Schema::Fixed(FixedSchema {
1864            name: fully_qualified_name.clone(),
1865            aliases: aliases.clone(),
1866            doc,
1867            size: size as usize,
1868            default,
1869            attributes: self.get_custom_attributes(complex, vec!["size"]),
1870        });
1871
1872        self.register_parsed_schema(&fully_qualified_name, &schema, &aliases);
1873
1874        Ok(schema)
1875    }
1876}
1877
1878// A type alias may be specified either as a fully namespace-qualified, or relative
1879// to the namespace of the name it is an alias for. For example, if a type named "a.b"
1880// has aliases of "c" and "x.y", then the fully qualified names of its aliases are "a.c"
1881// and "x.y".
1882// https://avro.apache.org/docs/current/spec.html#Aliases
1883fn fix_aliases_namespace(aliases: Option<Vec<String>>, namespace: &Namespace) -> Aliases {
1884    aliases.map(|aliases| {
1885        aliases
1886            .iter()
1887            .map(|alias| {
1888                if alias.find('.').is_none() {
1889                    match namespace {
1890                        Some(ref ns) => format!("{ns}.{alias}"),
1891                        None => alias.clone(),
1892                    }
1893                } else {
1894                    alias.clone()
1895                }
1896            })
1897            .map(|alias| Alias::new(alias.as_str()).unwrap())
1898            .collect()
1899    })
1900}
1901
1902fn get_schema_type_name(name: Name, value: Value) -> Name {
1903    match value.get("type") {
1904        Some(Value::Object(complex_type)) => match complex_type.name() {
1905            Some(name) => Name::new(name.as_str()).unwrap(),
1906            _ => name,
1907        },
1908        _ => name,
1909    }
1910}
1911
1912impl Serialize for Schema {
1913    fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
1914    where
1915        S: Serializer,
1916    {
1917        match *self {
1918            Schema::Ref { ref name } => serializer.serialize_str(&name.fullname(None)),
1919            Schema::Null => serializer.serialize_str("null"),
1920            Schema::Boolean => serializer.serialize_str("boolean"),
1921            Schema::Int => serializer.serialize_str("int"),
1922            Schema::Long => serializer.serialize_str("long"),
1923            Schema::Float => serializer.serialize_str("float"),
1924            Schema::Double => serializer.serialize_str("double"),
1925            Schema::Bytes => serializer.serialize_str("bytes"),
1926            Schema::String => serializer.serialize_str("string"),
1927            Schema::Array(ref inner) => {
1928                let mut map = serializer.serialize_map(Some(2 + inner.attributes.len()))?;
1929                map.serialize_entry("type", "array")?;
1930                map.serialize_entry("items", &*inner.items.clone())?;
1931                for attr in &inner.attributes {
1932                    map.serialize_entry(attr.0, attr.1)?;
1933                }
1934                map.end()
1935            }
1936            Schema::Map(ref inner) => {
1937                let mut map = serializer.serialize_map(Some(2 + inner.attributes.len()))?;
1938                map.serialize_entry("type", "map")?;
1939                map.serialize_entry("values", &*inner.types.clone())?;
1940                for attr in &inner.attributes {
1941                    map.serialize_entry(attr.0, attr.1)?;
1942                }
1943                map.end()
1944            }
1945            Schema::Union(ref inner) => {
1946                let variants = inner.variants();
1947                let mut seq = serializer.serialize_seq(Some(variants.len()))?;
1948                for v in variants {
1949                    seq.serialize_element(v)?;
1950                }
1951                seq.end()
1952            }
1953            Schema::Record(RecordSchema {
1954                ref name,
1955                ref aliases,
1956                ref doc,
1957                ref fields,
1958                ref attributes,
1959                ..
1960            }) => {
1961                let mut map = serializer.serialize_map(None)?;
1962                map.serialize_entry("type", "record")?;
1963                if let Some(ref n) = name.namespace {
1964                    map.serialize_entry("namespace", n)?;
1965                }
1966                map.serialize_entry("name", &name.name)?;
1967                if let Some(ref docstr) = doc {
1968                    map.serialize_entry("doc", docstr)?;
1969                }
1970                if let Some(ref aliases) = aliases {
1971                    map.serialize_entry("aliases", aliases)?;
1972                }
1973                map.serialize_entry("fields", fields)?;
1974                for attr in attributes {
1975                    map.serialize_entry(attr.0, attr.1)?;
1976                }
1977                map.end()
1978            }
1979            Schema::Enum(EnumSchema {
1980                ref name,
1981                ref symbols,
1982                ref aliases,
1983                ref attributes,
1984                ..
1985            }) => {
1986                let mut map = serializer.serialize_map(None)?;
1987                map.serialize_entry("type", "enum")?;
1988                if let Some(ref n) = name.namespace {
1989                    map.serialize_entry("namespace", n)?;
1990                }
1991                map.serialize_entry("name", &name.name)?;
1992                map.serialize_entry("symbols", symbols)?;
1993
1994                if let Some(ref aliases) = aliases {
1995                    map.serialize_entry("aliases", aliases)?;
1996                }
1997                for attr in attributes {
1998                    map.serialize_entry(attr.0, attr.1)?;
1999                }
2000                map.end()
2001            }
2002            Schema::Fixed(ref fixed_schema) => {
2003                let mut map = serializer.serialize_map(None)?;
2004                map = fixed_schema.serialize_to_map::<S>(map)?;
2005                map.end()
2006            }
2007            Schema::Decimal(DecimalSchema {
2008                ref scale,
2009                ref precision,
2010                ref inner,
2011            }) => {
2012                let mut map = serializer.serialize_map(None)?;
2013                match inner.as_ref() {
2014                    Schema::Fixed(fixed_schema) => {
2015                        map = fixed_schema.serialize_to_map::<S>(map)?;
2016                    }
2017                    Schema::Bytes => {
2018                        map.serialize_entry("type", "bytes")?;
2019                    }
2020                    others => {
2021                        return Err(serde::ser::Error::custom(format!(
2022                            "DecimalSchema inner type must be Fixed or Bytes, got {:?}",
2023                            SchemaKind::from(others)
2024                        )));
2025                    }
2026                }
2027                map.serialize_entry("logicalType", "decimal")?;
2028                map.serialize_entry("scale", scale)?;
2029                map.serialize_entry("precision", precision)?;
2030                map.end()
2031            }
2032
2033            Schema::BigDecimal => {
2034                let mut map = serializer.serialize_map(None)?;
2035                map.serialize_entry("type", "bytes")?;
2036                map.serialize_entry("logicalType", "big-decimal")?;
2037                map.end()
2038            }
2039            Schema::Uuid => {
2040                let mut map = serializer.serialize_map(None)?;
2041                map.serialize_entry("type", "string")?;
2042                map.serialize_entry("logicalType", "uuid")?;
2043                map.end()
2044            }
2045            Schema::Date => {
2046                let mut map = serializer.serialize_map(None)?;
2047                map.serialize_entry("type", "int")?;
2048                map.serialize_entry("logicalType", "date")?;
2049                map.end()
2050            }
2051            Schema::TimeMillis => {
2052                let mut map = serializer.serialize_map(None)?;
2053                map.serialize_entry("type", "int")?;
2054                map.serialize_entry("logicalType", "time-millis")?;
2055                map.end()
2056            }
2057            Schema::TimeMicros => {
2058                let mut map = serializer.serialize_map(None)?;
2059                map.serialize_entry("type", "long")?;
2060                map.serialize_entry("logicalType", "time-micros")?;
2061                map.end()
2062            }
2063            Schema::TimestampMillis => {
2064                let mut map = serializer.serialize_map(None)?;
2065                map.serialize_entry("type", "long")?;
2066                map.serialize_entry("logicalType", "timestamp-millis")?;
2067                map.end()
2068            }
2069            Schema::TimestampMicros => {
2070                let mut map = serializer.serialize_map(None)?;
2071                map.serialize_entry("type", "long")?;
2072                map.serialize_entry("logicalType", "timestamp-micros")?;
2073                map.end()
2074            }
2075            Schema::TimestampNanos => {
2076                let mut map = serializer.serialize_map(None)?;
2077                map.serialize_entry("type", "long")?;
2078                map.serialize_entry("logicalType", "timestamp-nanos")?;
2079                map.end()
2080            }
2081            Schema::LocalTimestampMillis => {
2082                let mut map = serializer.serialize_map(None)?;
2083                map.serialize_entry("type", "long")?;
2084                map.serialize_entry("logicalType", "local-timestamp-millis")?;
2085                map.end()
2086            }
2087            Schema::LocalTimestampMicros => {
2088                let mut map = serializer.serialize_map(None)?;
2089                map.serialize_entry("type", "long")?;
2090                map.serialize_entry("logicalType", "local-timestamp-micros")?;
2091                map.end()
2092            }
2093            Schema::LocalTimestampNanos => {
2094                let mut map = serializer.serialize_map(None)?;
2095                map.serialize_entry("type", "long")?;
2096                map.serialize_entry("logicalType", "local-timestamp-nanos")?;
2097                map.end()
2098            }
2099            Schema::Duration => {
2100                let mut map = serializer.serialize_map(None)?;
2101
2102                // the Avro doesn't indicate what the name of the underlying fixed type of a
2103                // duration should be or typically is.
2104                let inner = Schema::Fixed(FixedSchema {
2105                    name: Name::new("duration").unwrap(),
2106                    aliases: None,
2107                    doc: None,
2108                    size: 12,
2109                    default: None,
2110                    attributes: Default::default(),
2111                });
2112                map.serialize_entry("type", &inner)?;
2113                map.serialize_entry("logicalType", "duration")?;
2114                map.end()
2115            }
2116        }
2117    }
2118}
2119
2120impl Serialize for RecordField {
2121    fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
2122    where
2123        S: Serializer,
2124    {
2125        let mut map = serializer.serialize_map(None)?;
2126        map.serialize_entry("name", &self.name)?;
2127        map.serialize_entry("type", &self.schema)?;
2128
2129        if let Some(ref default) = self.default {
2130            map.serialize_entry("default", default)?;
2131        }
2132
2133        if let Some(ref aliases) = self.aliases {
2134            map.serialize_entry("aliases", aliases)?;
2135        }
2136
2137        for attr in &self.custom_attributes {
2138            map.serialize_entry(attr.0, attr.1)?;
2139        }
2140
2141        map.end()
2142    }
2143}
2144
2145/// Parses a **valid** avro schema into the Parsing Canonical Form.
2146/// https://avro.apache.org/docs/1.8.2/spec.html#Parsing+Canonical+Form+for+Schemas
2147fn parsing_canonical_form(schema: &Value) -> String {
2148    match schema {
2149        Value::Object(map) => pcf_map(map),
2150        Value::String(s) => pcf_string(s),
2151        Value::Array(v) => pcf_array(v),
2152        json => panic!("got invalid JSON value for canonical form of schema: {json}"),
2153    }
2154}
2155
2156fn pcf_map(schema: &Map<String, Value>) -> String {
2157    // Look for the namespace variant up front.
2158    let ns = schema.get("namespace").and_then(|v| v.as_str());
2159    let typ = schema.get("type").and_then(|v| v.as_str());
2160    let mut fields = Vec::new();
2161    for (k, v) in schema {
2162        // Reduce primitive types to their simple form. ([PRIMITIVE] rule)
2163        if schema.len() == 1 && k == "type" {
2164            // Invariant: function is only callable from a valid schema, so this is acceptable.
2165            if let Value::String(s) = v {
2166                return pcf_string(s);
2167            }
2168        }
2169
2170        // Strip out unused fields ([STRIP] rule)
2171        if field_ordering_position(k).is_none()
2172            || k == "default"
2173            || k == "doc"
2174            || k == "aliases"
2175            || k == "logicalType"
2176        {
2177            continue;
2178        }
2179
2180        // Fully qualify the name, if it isn't already ([FULLNAMES] rule).
2181        if k == "name" {
2182            // Invariant: Only valid schemas. Must be a string.
2183            let name = v.as_str().unwrap();
2184            let n = match ns {
2185                Some(namespace) if is_named_type(typ) && !name.contains('.') => {
2186                    Cow::Owned(format!("{namespace}.{name}"))
2187                }
2188                _ => Cow::Borrowed(name),
2189            };
2190
2191            fields.push((k, format!("{}:{}", pcf_string(k), pcf_string(&n))));
2192            continue;
2193        }
2194
2195        // Strip off quotes surrounding "size" type, if they exist ([INTEGERS] rule).
2196        if k == "size" || k == "precision" || k == "scale" {
2197            let i = match v.as_str() {
2198                Some(s) => s.parse::<i64>().expect("Only valid schemas are accepted!"),
2199                None => v.as_i64().unwrap(),
2200            };
2201            fields.push((k, format!("{}:{}", pcf_string(k), i)));
2202            continue;
2203        }
2204
2205        // For anything else, recursively process the result.
2206        fields.push((
2207            k,
2208            format!("{}:{}", pcf_string(k), parsing_canonical_form(v)),
2209        ));
2210    }
2211
2212    // Sort the fields by their canonical ordering ([ORDER] rule).
2213    fields.sort_unstable_by_key(|(k, _)| field_ordering_position(k).unwrap());
2214    let inter = fields
2215        .into_iter()
2216        .map(|(_, v)| v)
2217        .collect::<Vec<_>>()
2218        .join(",");
2219    format!("{{{inter}}}")
2220}
2221
2222fn is_named_type(typ: Option<&str>) -> bool {
2223    matches!(
2224        typ,
2225        Some("record") | Some("enum") | Some("fixed") | Some("ref")
2226    )
2227}
2228
2229fn pcf_array(arr: &[Value]) -> String {
2230    let inter = arr
2231        .iter()
2232        .map(parsing_canonical_form)
2233        .collect::<Vec<String>>()
2234        .join(",");
2235    format!("[{inter}]")
2236}
2237
2238fn pcf_string(s: &str) -> String {
2239    format!("\"{s}\"")
2240}
2241
2242const RESERVED_FIELDS: &[&str] = &[
2243    "name",
2244    "type",
2245    "fields",
2246    "symbols",
2247    "items",
2248    "values",
2249    "size",
2250    "logicalType",
2251    "order",
2252    "doc",
2253    "aliases",
2254    "default",
2255    "precision",
2256    "scale",
2257];
2258
2259// Used to define the ordering and inclusion of fields.
2260fn field_ordering_position(field: &str) -> Option<usize> {
2261    RESERVED_FIELDS
2262        .iter()
2263        .position(|&f| f == field)
2264        .map(|pos| pos + 1)
2265}
2266
2267/// Trait for types that serve as an Avro data model. Derive implementation available
2268/// through `derive` feature. Do not implement directly!
2269/// Implement `apache_avro::schema::derive::AvroSchemaComponent` to get this trait
2270/// through a blanket implementation.
2271pub trait AvroSchema {
2272    fn get_schema() -> Schema;
2273}
2274
2275#[cfg(feature = "derive")]
2276pub mod derive {
2277    use super::*;
2278
2279    /// Trait for types that serve as fully defined components inside an Avro data model. Derive
2280    /// implementation available through `derive` feature. This is what is implemented by
2281    /// the `derive(AvroSchema)` macro.
2282    ///
2283    /// # Implementation guide
2284    ///
2285    ///### Simple implementation
2286    /// To construct a non named simple schema, it is possible to ignore the input argument making the
2287    /// general form implementation look like
2288    /// ```ignore
2289    /// impl AvroSchemaComponent for AType {
2290    ///     fn get_schema_in_ctxt(_: &mut Names, _: &Namespace) -> Schema {
2291    ///        Schema::?
2292    ///    }
2293    ///}
2294    /// ```
2295    /// ### Passthrough implementation
2296    /// To construct a schema for a Type that acts as in "inner" type, such as for smart pointers, simply
2297    /// pass through the arguments to the inner type
2298    /// ```ignore
2299    /// impl AvroSchemaComponent for PassthroughType {
2300    ///     fn get_schema_in_ctxt(named_schemas: &mut Names, enclosing_namespace: &Namespace) -> Schema {
2301    ///        InnerType::get_schema_in_ctxt(names, enclosing_namespace)
2302    ///    }
2303    ///}
2304    /// ```
2305    ///### Complex implementation
2306    /// To implement this for Named schema there is a general form needed to avoid creating invalid
2307    /// schemas or infinite loops.
2308    /// ```ignore
2309    /// impl AvroSchemaComponent for ComplexType {
2310    ///     fn get_schema_in_ctxt(named_schemas: &mut Names, enclosing_namespace: &Namespace) -> Schema {
2311    ///         // Create the fully qualified name for your type given the enclosing namespace
2312    ///         let name =  apache_avro::schema::Name::new("MyName")
2313    ///             .expect("Unable to parse schema name")
2314    ///             .fully_qualified_name(enclosing_namespace);
2315    ///         let enclosing_namespace = &name.namespace;
2316    ///         // Check, if your name is already defined, and if so, return a ref to that name
2317    ///         if named_schemas.contains_key(&name) {
2318    ///             apache_avro::schema::Schema::Ref{name: name.clone()}
2319    ///         } else {
2320    ///             named_schemas.insert(name.clone(), apache_avro::schema::Schema::Ref{name: name.clone()});
2321    ///             // YOUR SCHEMA DEFINITION HERE with the name equivalent to "MyName".
2322    ///             // For non-simple sub types delegate to their implementation of AvroSchemaComponent
2323    ///         }
2324    ///    }
2325    ///}
2326    /// ```
2327    pub trait AvroSchemaComponent {
2328        fn get_schema_in_ctxt(named_schemas: &mut Names, enclosing_namespace: &Namespace)
2329            -> Schema;
2330    }
2331
2332    impl<T> AvroSchema for T
2333    where
2334        T: AvroSchemaComponent,
2335    {
2336        fn get_schema() -> Schema {
2337            T::get_schema_in_ctxt(&mut HashMap::default(), &None)
2338        }
2339    }
2340
2341    macro_rules! impl_schema (
2342        ($type:ty, $variant_constructor:expr) => (
2343            impl AvroSchemaComponent for $type {
2344                fn get_schema_in_ctxt(_: &mut Names, _: &Namespace) -> Schema {
2345                    $variant_constructor
2346                }
2347            }
2348        );
2349    );
2350
2351    impl_schema!(bool, Schema::Boolean);
2352    impl_schema!(i8, Schema::Int);
2353    impl_schema!(i16, Schema::Int);
2354    impl_schema!(i32, Schema::Int);
2355    impl_schema!(i64, Schema::Long);
2356    impl_schema!(u8, Schema::Int);
2357    impl_schema!(u16, Schema::Int);
2358    impl_schema!(u32, Schema::Long);
2359    impl_schema!(f32, Schema::Float);
2360    impl_schema!(f64, Schema::Double);
2361    impl_schema!(String, Schema::String);
2362    impl_schema!(uuid::Uuid, Schema::Uuid);
2363    impl_schema!(core::time::Duration, Schema::Duration);
2364
2365    impl<T> AvroSchemaComponent for Vec<T>
2366    where
2367        T: AvroSchemaComponent,
2368    {
2369        fn get_schema_in_ctxt(
2370            named_schemas: &mut Names,
2371            enclosing_namespace: &Namespace,
2372        ) -> Schema {
2373            Schema::array(T::get_schema_in_ctxt(named_schemas, enclosing_namespace))
2374        }
2375    }
2376
2377    impl<T> AvroSchemaComponent for Option<T>
2378    where
2379        T: AvroSchemaComponent,
2380    {
2381        fn get_schema_in_ctxt(
2382            named_schemas: &mut Names,
2383            enclosing_namespace: &Namespace,
2384        ) -> Schema {
2385            let inner_schema = T::get_schema_in_ctxt(named_schemas, enclosing_namespace);
2386            Schema::Union(UnionSchema {
2387                schemas: vec![Schema::Null, inner_schema.clone()],
2388                variant_index: vec![Schema::Null, inner_schema]
2389                    .iter()
2390                    .enumerate()
2391                    .map(|(idx, s)| (SchemaKind::from(s), idx))
2392                    .collect(),
2393            })
2394        }
2395    }
2396
2397    impl<T> AvroSchemaComponent for Map<String, T>
2398    where
2399        T: AvroSchemaComponent,
2400    {
2401        fn get_schema_in_ctxt(
2402            named_schemas: &mut Names,
2403            enclosing_namespace: &Namespace,
2404        ) -> Schema {
2405            Schema::map(T::get_schema_in_ctxt(named_schemas, enclosing_namespace))
2406        }
2407    }
2408
2409    impl<T> AvroSchemaComponent for HashMap<String, T>
2410    where
2411        T: AvroSchemaComponent,
2412    {
2413        fn get_schema_in_ctxt(
2414            named_schemas: &mut Names,
2415            enclosing_namespace: &Namespace,
2416        ) -> Schema {
2417            Schema::map(T::get_schema_in_ctxt(named_schemas, enclosing_namespace))
2418        }
2419    }
2420
2421    impl<T> AvroSchemaComponent for Box<T>
2422    where
2423        T: AvroSchemaComponent,
2424    {
2425        fn get_schema_in_ctxt(
2426            named_schemas: &mut Names,
2427            enclosing_namespace: &Namespace,
2428        ) -> Schema {
2429            T::get_schema_in_ctxt(named_schemas, enclosing_namespace)
2430        }
2431    }
2432
2433    impl<T> AvroSchemaComponent for std::sync::Mutex<T>
2434    where
2435        T: AvroSchemaComponent,
2436    {
2437        fn get_schema_in_ctxt(
2438            named_schemas: &mut Names,
2439            enclosing_namespace: &Namespace,
2440        ) -> Schema {
2441            T::get_schema_in_ctxt(named_schemas, enclosing_namespace)
2442        }
2443    }
2444
2445    impl<T> AvroSchemaComponent for Cow<'_, T>
2446    where
2447        T: AvroSchemaComponent + Clone,
2448    {
2449        fn get_schema_in_ctxt(
2450            named_schemas: &mut Names,
2451            enclosing_namespace: &Namespace,
2452        ) -> Schema {
2453            T::get_schema_in_ctxt(named_schemas, enclosing_namespace)
2454        }
2455    }
2456}
2457
2458#[cfg(test)]
2459mod tests {
2460    use super::*;
2461    use crate::rabin::Rabin;
2462    use apache_avro_test_helper::{
2463        logger::{assert_logged, assert_not_logged},
2464        TestResult,
2465    };
2466    use serde_json::json;
2467
2468    #[test]
2469    fn test_invalid_schema() {
2470        assert!(Schema::parse_str("invalid").is_err());
2471    }
2472
2473    #[test]
2474    fn test_primitive_schema() -> TestResult {
2475        assert_eq!(Schema::Null, Schema::parse_str("\"null\"")?);
2476        assert_eq!(Schema::Int, Schema::parse_str("\"int\"")?);
2477        assert_eq!(Schema::Double, Schema::parse_str("\"double\"")?);
2478        Ok(())
2479    }
2480
2481    #[test]
2482    fn test_array_schema() -> TestResult {
2483        let schema = Schema::parse_str(r#"{"type": "array", "items": "string"}"#)?;
2484        assert_eq!(Schema::array(Schema::String), schema);
2485        Ok(())
2486    }
2487
2488    #[test]
2489    fn test_map_schema() -> TestResult {
2490        let schema = Schema::parse_str(r#"{"type": "map", "values": "double"}"#)?;
2491        assert_eq!(Schema::map(Schema::Double), schema);
2492        Ok(())
2493    }
2494
2495    #[test]
2496    fn test_union_schema() -> TestResult {
2497        let schema = Schema::parse_str(r#"["null", "int"]"#)?;
2498        assert_eq!(
2499            Schema::Union(UnionSchema::new(vec![Schema::Null, Schema::Int])?),
2500            schema
2501        );
2502        Ok(())
2503    }
2504
2505    #[test]
2506    fn test_union_unsupported_schema() {
2507        let schema = Schema::parse_str(r#"["null", ["null", "int"], "string"]"#);
2508        assert!(schema.is_err());
2509    }
2510
2511    #[test]
2512    fn test_multi_union_schema() -> TestResult {
2513        let schema = Schema::parse_str(r#"["null", "int", "float", "string", "bytes"]"#);
2514        assert!(schema.is_ok());
2515        let schema = schema?;
2516        assert_eq!(SchemaKind::from(&schema), SchemaKind::Union);
2517        let union_schema = match schema {
2518            Schema::Union(u) => u,
2519            _ => unreachable!(),
2520        };
2521        assert_eq!(union_schema.variants().len(), 5);
2522        let mut variants = union_schema.variants().iter();
2523        assert_eq!(SchemaKind::from(variants.next().unwrap()), SchemaKind::Null);
2524        assert_eq!(SchemaKind::from(variants.next().unwrap()), SchemaKind::Int);
2525        assert_eq!(
2526            SchemaKind::from(variants.next().unwrap()),
2527            SchemaKind::Float
2528        );
2529        assert_eq!(
2530            SchemaKind::from(variants.next().unwrap()),
2531            SchemaKind::String
2532        );
2533        assert_eq!(
2534            SchemaKind::from(variants.next().unwrap()),
2535            SchemaKind::Bytes
2536        );
2537        assert_eq!(variants.next(), None);
2538
2539        Ok(())
2540    }
2541
2542    #[test]
2543    fn test_avro_3621_nullable_record_field() -> TestResult {
2544        let nullable_record_field = RecordField {
2545            name: "next".to_string(),
2546            doc: None,
2547            default: None,
2548            aliases: None,
2549            schema: Schema::Union(UnionSchema::new(vec![
2550                Schema::Null,
2551                Schema::Ref {
2552                    name: Name {
2553                        name: "LongList".to_owned(),
2554                        namespace: None,
2555                    },
2556                },
2557            ])?),
2558            order: RecordFieldOrder::Ascending,
2559            position: 1,
2560            custom_attributes: Default::default(),
2561        };
2562
2563        assert!(nullable_record_field.is_nullable());
2564
2565        let non_nullable_record_field = RecordField {
2566            name: "next".to_string(),
2567            doc: None,
2568            default: Some(json!(2)),
2569            aliases: None,
2570            schema: Schema::Long,
2571            order: RecordFieldOrder::Ascending,
2572            position: 1,
2573            custom_attributes: Default::default(),
2574        };
2575
2576        assert!(!non_nullable_record_field.is_nullable());
2577        Ok(())
2578    }
2579
2580    // AVRO-3248
2581    #[test]
2582    fn test_union_of_records() -> TestResult {
2583        use std::iter::FromIterator;
2584
2585        // A and B are the same except the name.
2586        let schema_str_a = r#"{
2587            "name": "A",
2588            "type": "record",
2589            "fields": [
2590                {"name": "field_one", "type": "float"}
2591            ]
2592        }"#;
2593
2594        let schema_str_b = r#"{
2595            "name": "B",
2596            "type": "record",
2597            "fields": [
2598                {"name": "field_one", "type": "float"}
2599            ]
2600        }"#;
2601
2602        // we get Error::GetNameField if we put ["A", "B"] directly here.
2603        let schema_str_c = r#"{
2604            "name": "C",
2605            "type": "record",
2606            "fields": [
2607                {"name": "field_one",  "type": ["A", "B"]}
2608            ]
2609        }"#;
2610
2611        let schema_c = Schema::parse_list(&[schema_str_a, schema_str_b, schema_str_c])?
2612            .last()
2613            .unwrap()
2614            .clone();
2615
2616        let schema_c_expected = Schema::Record(RecordSchema {
2617            name: Name::new("C")?,
2618            aliases: None,
2619            doc: None,
2620            fields: vec![RecordField {
2621                name: "field_one".to_string(),
2622                doc: None,
2623                default: None,
2624                aliases: None,
2625                schema: Schema::Union(UnionSchema::new(vec![
2626                    Schema::Ref {
2627                        name: Name::new("A")?,
2628                    },
2629                    Schema::Ref {
2630                        name: Name::new("B")?,
2631                    },
2632                ])?),
2633                order: RecordFieldOrder::Ignore,
2634                position: 0,
2635                custom_attributes: Default::default(),
2636            }],
2637            lookup: BTreeMap::from_iter(vec![("field_one".to_string(), 0)]),
2638            attributes: Default::default(),
2639        });
2640
2641        assert_eq!(schema_c, schema_c_expected);
2642        Ok(())
2643    }
2644
2645    #[test]
2646    fn avro_3584_test_recursion_records() -> TestResult {
2647        // A and B are the same except the name.
2648        let schema_str_a = r#"{
2649            "name": "A",
2650            "type": "record",
2651            "fields": [ {"name": "field_one", "type": "B"} ]
2652        }"#;
2653
2654        let schema_str_b = r#"{
2655            "name": "B",
2656            "type": "record",
2657            "fields": [ {"name": "field_one", "type": "A"} ]
2658        }"#;
2659
2660        let list = Schema::parse_list(&[schema_str_a, schema_str_b])?;
2661
2662        let schema_a = list.first().unwrap().clone();
2663
2664        match schema_a {
2665            Schema::Record(RecordSchema { fields, .. }) => {
2666                let f1 = fields.first();
2667
2668                let ref_schema = Schema::Ref {
2669                    name: Name::new("B")?,
2670                };
2671                assert_eq!(ref_schema, f1.unwrap().schema);
2672            }
2673            _ => panic!("Expected a record schema!"),
2674        }
2675
2676        Ok(())
2677    }
2678
2679    #[test]
2680    fn test_avro_3248_nullable_record() -> TestResult {
2681        use std::iter::FromIterator;
2682
2683        let schema_str_a = r#"{
2684            "name": "A",
2685            "type": "record",
2686            "fields": [
2687                {"name": "field_one", "type": "float"}
2688            ]
2689        }"#;
2690
2691        // we get Error::GetNameField if we put ["null", "B"] directly here.
2692        let schema_str_option_a = r#"{
2693            "name": "OptionA",
2694            "type": "record",
2695            "fields": [
2696                {"name": "field_one",  "type": ["null", "A"], "default": null}
2697            ]
2698        }"#;
2699
2700        let schema_option_a = Schema::parse_list(&[schema_str_a, schema_str_option_a])?
2701            .last()
2702            .unwrap()
2703            .clone();
2704
2705        let schema_option_a_expected = Schema::Record(RecordSchema {
2706            name: Name::new("OptionA")?,
2707            aliases: None,
2708            doc: None,
2709            fields: vec![RecordField {
2710                name: "field_one".to_string(),
2711                doc: None,
2712                default: Some(Value::Null),
2713                aliases: None,
2714                schema: Schema::Union(UnionSchema::new(vec![
2715                    Schema::Null,
2716                    Schema::Ref {
2717                        name: Name::new("A")?,
2718                    },
2719                ])?),
2720                order: RecordFieldOrder::Ignore,
2721                position: 0,
2722                custom_attributes: Default::default(),
2723            }],
2724            lookup: BTreeMap::from_iter(vec![("field_one".to_string(), 0)]),
2725            attributes: Default::default(),
2726        });
2727
2728        assert_eq!(schema_option_a, schema_option_a_expected);
2729
2730        Ok(())
2731    }
2732
2733    #[test]
2734    fn test_record_schema() -> TestResult {
2735        let parsed = Schema::parse_str(
2736            r#"
2737            {
2738                "type": "record",
2739                "name": "test",
2740                "fields": [
2741                    {"name": "a", "type": "long", "default": 42},
2742                    {"name": "b", "type": "string"}
2743                ]
2744            }
2745        "#,
2746        )?;
2747
2748        let mut lookup = BTreeMap::new();
2749        lookup.insert("a".to_owned(), 0);
2750        lookup.insert("b".to_owned(), 1);
2751
2752        let expected = Schema::Record(RecordSchema {
2753            name: Name::new("test")?,
2754            aliases: None,
2755            doc: None,
2756            fields: vec![
2757                RecordField {
2758                    name: "a".to_string(),
2759                    doc: None,
2760                    default: Some(Value::Number(42i64.into())),
2761                    aliases: None,
2762                    schema: Schema::Long,
2763                    order: RecordFieldOrder::Ascending,
2764                    position: 0,
2765                    custom_attributes: Default::default(),
2766                },
2767                RecordField {
2768                    name: "b".to_string(),
2769                    doc: None,
2770                    default: None,
2771                    aliases: None,
2772                    schema: Schema::String,
2773                    order: RecordFieldOrder::Ascending,
2774                    position: 1,
2775                    custom_attributes: Default::default(),
2776                },
2777            ],
2778            lookup,
2779            attributes: Default::default(),
2780        });
2781
2782        assert_eq!(parsed, expected);
2783
2784        Ok(())
2785    }
2786
2787    #[test]
2788    fn test_avro_3302_record_schema_with_currently_parsing_schema() -> TestResult {
2789        let schema = Schema::parse_str(
2790            r#"
2791            {
2792                "type": "record",
2793                "name": "test",
2794                "fields": [{
2795                    "name": "recordField",
2796                    "type": {
2797                        "type": "record",
2798                        "name": "Node",
2799                        "fields": [
2800                            {"name": "label", "type": "string"},
2801                            {"name": "children", "type": {"type": "array", "items": "Node"}}
2802                        ]
2803                    }
2804                }]
2805            }
2806        "#,
2807        )?;
2808
2809        let mut lookup = BTreeMap::new();
2810        lookup.insert("recordField".to_owned(), 0);
2811
2812        let mut node_lookup = BTreeMap::new();
2813        node_lookup.insert("children".to_owned(), 1);
2814        node_lookup.insert("label".to_owned(), 0);
2815
2816        let expected = Schema::Record(RecordSchema {
2817            name: Name::new("test")?,
2818            aliases: None,
2819            doc: None,
2820            fields: vec![RecordField {
2821                name: "recordField".to_string(),
2822                doc: None,
2823                default: None,
2824                aliases: None,
2825                schema: Schema::Record(RecordSchema {
2826                    name: Name::new("Node")?,
2827                    aliases: None,
2828                    doc: None,
2829                    fields: vec![
2830                        RecordField {
2831                            name: "label".to_string(),
2832                            doc: None,
2833                            default: None,
2834                            aliases: None,
2835                            schema: Schema::String,
2836                            order: RecordFieldOrder::Ascending,
2837                            position: 0,
2838                            custom_attributes: Default::default(),
2839                        },
2840                        RecordField {
2841                            name: "children".to_string(),
2842                            doc: None,
2843                            default: None,
2844                            aliases: None,
2845                            schema: Schema::array(Schema::Ref {
2846                                name: Name::new("Node")?,
2847                            }),
2848                            order: RecordFieldOrder::Ascending,
2849                            position: 1,
2850                            custom_attributes: Default::default(),
2851                        },
2852                    ],
2853                    lookup: node_lookup,
2854                    attributes: Default::default(),
2855                }),
2856                order: RecordFieldOrder::Ascending,
2857                position: 0,
2858                custom_attributes: Default::default(),
2859            }],
2860            lookup,
2861            attributes: Default::default(),
2862        });
2863        assert_eq!(schema, expected);
2864
2865        let canonical_form = &schema.canonical_form();
2866        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"}}]}}]}"#;
2867        assert_eq!(canonical_form, &expected);
2868
2869        Ok(())
2870    }
2871
2872    // https://github.com/flavray/avro-rs/pull/99#issuecomment-1016948451
2873    #[test]
2874    fn test_parsing_of_recursive_type_enum() -> TestResult {
2875        let schema = r#"
2876    {
2877        "type": "record",
2878        "name": "User",
2879        "namespace": "office",
2880        "fields": [
2881            {
2882              "name": "details",
2883              "type": [
2884                {
2885                  "type": "record",
2886                  "name": "Employee",
2887                  "fields": [
2888                    {
2889                      "name": "gender",
2890                      "type": {
2891                        "type": "enum",
2892                        "name": "Gender",
2893                        "symbols": [
2894                          "male",
2895                          "female"
2896                        ]
2897                      },
2898                      "default": "female"
2899                    }
2900                  ]
2901                },
2902                {
2903                  "type": "record",
2904                  "name": "Manager",
2905                  "fields": [
2906                    {
2907                      "name": "gender",
2908                      "type": "Gender"
2909                    }
2910                  ]
2911                }
2912              ]
2913            }
2914          ]
2915        }
2916        "#;
2917
2918        let schema = Schema::parse_str(schema)?;
2919        let schema_str = schema.canonical_form();
2920        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"}]}]}]}"#;
2921        assert_eq!(schema_str, expected);
2922
2923        Ok(())
2924    }
2925
2926    #[test]
2927    fn test_parsing_of_recursive_type_fixed() -> TestResult {
2928        let schema = r#"
2929    {
2930        "type": "record",
2931        "name": "User",
2932        "namespace": "office",
2933        "fields": [
2934            {
2935              "name": "details",
2936              "type": [
2937                {
2938                  "type": "record",
2939                  "name": "Employee",
2940                  "fields": [
2941                    {
2942                      "name": "id",
2943                      "type": {
2944                        "type": "fixed",
2945                        "name": "EmployeeId",
2946                        "size": 16
2947                      },
2948                      "default": "female"
2949                    }
2950                  ]
2951                },
2952                {
2953                  "type": "record",
2954                  "name": "Manager",
2955                  "fields": [
2956                    {
2957                      "name": "id",
2958                      "type": "EmployeeId"
2959                    }
2960                  ]
2961                }
2962              ]
2963            }
2964          ]
2965        }
2966        "#;
2967
2968        let schema = Schema::parse_str(schema)?;
2969        let schema_str = schema.canonical_form();
2970        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"}]}]}]}"#;
2971        assert_eq!(schema_str, expected);
2972
2973        Ok(())
2974    }
2975
2976    #[test]
2977    fn test_avro_3302_record_schema_with_currently_parsing_schema_aliases() -> TestResult {
2978        let schema = Schema::parse_str(
2979            r#"
2980            {
2981              "type": "record",
2982              "name": "LongList",
2983              "aliases": ["LinkedLongs"],
2984              "fields" : [
2985                {"name": "value", "type": "long"},
2986                {"name": "next", "type": ["null", "LinkedLongs"]}
2987              ]
2988            }
2989        "#,
2990        )?;
2991
2992        let mut lookup = BTreeMap::new();
2993        lookup.insert("value".to_owned(), 0);
2994        lookup.insert("next".to_owned(), 1);
2995
2996        let expected = Schema::Record(RecordSchema {
2997            name: Name {
2998                name: "LongList".to_owned(),
2999                namespace: None,
3000            },
3001            aliases: Some(vec![Alias::new("LinkedLongs").unwrap()]),
3002            doc: None,
3003            fields: vec![
3004                RecordField {
3005                    name: "value".to_string(),
3006                    doc: None,
3007                    default: None,
3008                    aliases: None,
3009                    schema: Schema::Long,
3010                    order: RecordFieldOrder::Ascending,
3011                    position: 0,
3012                    custom_attributes: Default::default(),
3013                },
3014                RecordField {
3015                    name: "next".to_string(),
3016                    doc: None,
3017                    default: None,
3018                    aliases: None,
3019                    schema: Schema::Union(UnionSchema::new(vec![
3020                        Schema::Null,
3021                        Schema::Ref {
3022                            name: Name {
3023                                name: "LongList".to_owned(),
3024                                namespace: None,
3025                            },
3026                        },
3027                    ])?),
3028                    order: RecordFieldOrder::Ascending,
3029                    position: 1,
3030                    custom_attributes: Default::default(),
3031                },
3032            ],
3033            lookup,
3034            attributes: Default::default(),
3035        });
3036        assert_eq!(schema, expected);
3037
3038        let canonical_form = &schema.canonical_form();
3039        let expected = r#"{"name":"LongList","type":"record","fields":[{"name":"value","type":"long"},{"name":"next","type":["null","LongList"]}]}"#;
3040        assert_eq!(canonical_form, &expected);
3041
3042        Ok(())
3043    }
3044
3045    #[test]
3046    fn test_avro_3370_record_schema_with_currently_parsing_schema_named_record() -> TestResult {
3047        let schema = Schema::parse_str(
3048            r#"
3049            {
3050              "type" : "record",
3051              "name" : "record",
3052              "fields" : [
3053                 { "name" : "value", "type" : "long" },
3054                 { "name" : "next", "type" : "record" }
3055             ]
3056            }
3057        "#,
3058        )?;
3059
3060        let mut lookup = BTreeMap::new();
3061        lookup.insert("value".to_owned(), 0);
3062        lookup.insert("next".to_owned(), 1);
3063
3064        let expected = Schema::Record(RecordSchema {
3065            name: Name {
3066                name: "record".to_owned(),
3067                namespace: None,
3068            },
3069            aliases: None,
3070            doc: None,
3071            fields: vec![
3072                RecordField {
3073                    name: "value".to_string(),
3074                    doc: None,
3075                    default: None,
3076                    aliases: None,
3077                    schema: Schema::Long,
3078                    order: RecordFieldOrder::Ascending,
3079                    position: 0,
3080                    custom_attributes: Default::default(),
3081                },
3082                RecordField {
3083                    name: "next".to_string(),
3084                    doc: None,
3085                    default: None,
3086                    aliases: None,
3087                    schema: Schema::Ref {
3088                        name: Name {
3089                            name: "record".to_owned(),
3090                            namespace: None,
3091                        },
3092                    },
3093                    order: RecordFieldOrder::Ascending,
3094                    position: 1,
3095                    custom_attributes: Default::default(),
3096                },
3097            ],
3098            lookup,
3099            attributes: Default::default(),
3100        });
3101        assert_eq!(schema, expected);
3102
3103        let canonical_form = &schema.canonical_form();
3104        let expected = r#"{"name":"record","type":"record","fields":[{"name":"value","type":"long"},{"name":"next","type":"record"}]}"#;
3105        assert_eq!(canonical_form, &expected);
3106
3107        Ok(())
3108    }
3109
3110    #[test]
3111    fn test_avro_3370_record_schema_with_currently_parsing_schema_named_enum() -> TestResult {
3112        let schema = Schema::parse_str(
3113            r#"
3114            {
3115              "type" : "record",
3116              "name" : "record",
3117              "fields" : [
3118                 {
3119                    "type" : "enum",
3120                    "name" : "enum",
3121                    "symbols": ["one", "two", "three"]
3122                 },
3123                 { "name" : "next", "type" : "enum" }
3124             ]
3125            }
3126        "#,
3127        )?;
3128
3129        let mut lookup = BTreeMap::new();
3130        lookup.insert("enum".to_owned(), 0);
3131        lookup.insert("next".to_owned(), 1);
3132
3133        let expected = Schema::Record(RecordSchema {
3134            name: Name {
3135                name: "record".to_owned(),
3136                namespace: None,
3137            },
3138            aliases: None,
3139            doc: None,
3140            fields: vec![
3141                RecordField {
3142                    name: "enum".to_string(),
3143                    doc: None,
3144                    default: None,
3145                    aliases: None,
3146                    schema: Schema::Enum(EnumSchema {
3147                        name: Name {
3148                            name: "enum".to_owned(),
3149                            namespace: None,
3150                        },
3151                        aliases: None,
3152                        doc: None,
3153                        symbols: vec!["one".to_string(), "two".to_string(), "three".to_string()],
3154                        default: None,
3155                        attributes: Default::default(),
3156                    }),
3157                    order: RecordFieldOrder::Ascending,
3158                    position: 0,
3159                    custom_attributes: Default::default(),
3160                },
3161                RecordField {
3162                    name: "next".to_string(),
3163                    doc: None,
3164                    default: None,
3165                    aliases: None,
3166                    schema: Schema::Enum(EnumSchema {
3167                        name: Name {
3168                            name: "enum".to_owned(),
3169                            namespace: None,
3170                        },
3171                        aliases: None,
3172                        doc: None,
3173                        symbols: vec!["one".to_string(), "two".to_string(), "three".to_string()],
3174                        default: None,
3175                        attributes: Default::default(),
3176                    }),
3177                    order: RecordFieldOrder::Ascending,
3178                    position: 1,
3179                    custom_attributes: Default::default(),
3180                },
3181            ],
3182            lookup,
3183            attributes: Default::default(),
3184        });
3185        assert_eq!(schema, expected);
3186
3187        let canonical_form = &schema.canonical_form();
3188        let expected = r#"{"name":"record","type":"record","fields":[{"name":"enum","type":{"name":"enum","type":"enum","symbols":["one","two","three"]}},{"name":"next","type":{"name":"enum","type":"enum","symbols":["one","two","three"]}}]}"#;
3189        assert_eq!(canonical_form, &expected);
3190
3191        Ok(())
3192    }
3193
3194    #[test]
3195    fn test_avro_3370_record_schema_with_currently_parsing_schema_named_fixed() -> TestResult {
3196        let schema = Schema::parse_str(
3197            r#"
3198            {
3199              "type" : "record",
3200              "name" : "record",
3201              "fields" : [
3202                 {
3203                    "type" : "fixed",
3204                    "name" : "fixed",
3205                    "size": 456
3206                 },
3207                 { "name" : "next", "type" : "fixed" }
3208             ]
3209            }
3210        "#,
3211        )?;
3212
3213        let mut lookup = BTreeMap::new();
3214        lookup.insert("fixed".to_owned(), 0);
3215        lookup.insert("next".to_owned(), 1);
3216
3217        let expected = Schema::Record(RecordSchema {
3218            name: Name {
3219                name: "record".to_owned(),
3220                namespace: None,
3221            },
3222            aliases: None,
3223            doc: None,
3224            fields: vec![
3225                RecordField {
3226                    name: "fixed".to_string(),
3227                    doc: None,
3228                    default: None,
3229                    aliases: None,
3230                    schema: Schema::Fixed(FixedSchema {
3231                        name: Name {
3232                            name: "fixed".to_owned(),
3233                            namespace: None,
3234                        },
3235                        aliases: None,
3236                        doc: None,
3237                        size: 456,
3238                        default: None,
3239                        attributes: Default::default(),
3240                    }),
3241                    order: RecordFieldOrder::Ascending,
3242                    position: 0,
3243                    custom_attributes: Default::default(),
3244                },
3245                RecordField {
3246                    name: "next".to_string(),
3247                    doc: None,
3248                    default: None,
3249                    aliases: None,
3250                    schema: Schema::Fixed(FixedSchema {
3251                        name: Name {
3252                            name: "fixed".to_owned(),
3253                            namespace: None,
3254                        },
3255                        aliases: None,
3256                        doc: None,
3257                        size: 456,
3258                        default: None,
3259                        attributes: Default::default(),
3260                    }),
3261                    order: RecordFieldOrder::Ascending,
3262                    position: 1,
3263                    custom_attributes: Default::default(),
3264                },
3265            ],
3266            lookup,
3267            attributes: Default::default(),
3268        });
3269        assert_eq!(schema, expected);
3270
3271        let canonical_form = &schema.canonical_form();
3272        let expected = r#"{"name":"record","type":"record","fields":[{"name":"fixed","type":{"name":"fixed","type":"fixed","size":456}},{"name":"next","type":{"name":"fixed","type":"fixed","size":456}}]}"#;
3273        assert_eq!(canonical_form, &expected);
3274
3275        Ok(())
3276    }
3277
3278    #[test]
3279    fn test_enum_schema() -> TestResult {
3280        let schema = Schema::parse_str(
3281            r#"{"type": "enum", "name": "Suit", "symbols": ["diamonds", "spades", "clubs", "hearts"]}"#,
3282        )?;
3283
3284        let expected = Schema::Enum(EnumSchema {
3285            name: Name::new("Suit")?,
3286            aliases: None,
3287            doc: None,
3288            symbols: vec![
3289                "diamonds".to_owned(),
3290                "spades".to_owned(),
3291                "clubs".to_owned(),
3292                "hearts".to_owned(),
3293            ],
3294            default: None,
3295            attributes: Default::default(),
3296        });
3297
3298        assert_eq!(expected, schema);
3299
3300        Ok(())
3301    }
3302
3303    #[test]
3304    fn test_enum_schema_duplicate() -> TestResult {
3305        // Duplicate "diamonds"
3306        let schema = Schema::parse_str(
3307            r#"{"type": "enum", "name": "Suit", "symbols": ["diamonds", "spades", "clubs", "diamonds"]}"#,
3308        );
3309        assert!(schema.is_err());
3310
3311        Ok(())
3312    }
3313
3314    #[test]
3315    fn test_enum_schema_name() -> TestResult {
3316        // Invalid name "0000" does not match [A-Za-z_][A-Za-z0-9_]*
3317        let schema = Schema::parse_str(
3318            r#"{"type": "enum", "name": "Enum", "symbols": ["0000", "variant"]}"#,
3319        );
3320        assert!(schema.is_err());
3321
3322        Ok(())
3323    }
3324
3325    #[test]
3326    fn test_fixed_schema() -> TestResult {
3327        let schema = Schema::parse_str(r#"{"type": "fixed", "name": "test", "size": 16}"#)?;
3328
3329        let expected = Schema::Fixed(FixedSchema {
3330            name: Name::new("test")?,
3331            aliases: None,
3332            doc: None,
3333            size: 16_usize,
3334            default: None,
3335            attributes: Default::default(),
3336        });
3337
3338        assert_eq!(expected, schema);
3339
3340        Ok(())
3341    }
3342
3343    #[test]
3344    fn test_fixed_schema_with_documentation() -> TestResult {
3345        let schema = Schema::parse_str(
3346            r#"{"type": "fixed", "name": "test", "size": 16, "doc": "FixedSchema documentation"}"#,
3347        )?;
3348
3349        let expected = Schema::Fixed(FixedSchema {
3350            name: Name::new("test")?,
3351            aliases: None,
3352            doc: Some(String::from("FixedSchema documentation")),
3353            size: 16_usize,
3354            default: None,
3355            attributes: Default::default(),
3356        });
3357
3358        assert_eq!(expected, schema);
3359
3360        Ok(())
3361    }
3362
3363    #[test]
3364    fn test_no_documentation() -> TestResult {
3365        let schema = Schema::parse_str(
3366            r#"{"type": "enum", "name": "Coin", "symbols": ["heads", "tails"]}"#,
3367        )?;
3368
3369        let doc = match schema {
3370            Schema::Enum(EnumSchema { doc, .. }) => doc,
3371            _ => unreachable!(),
3372        };
3373
3374        assert!(doc.is_none());
3375
3376        Ok(())
3377    }
3378
3379    #[test]
3380    fn test_documentation() -> TestResult {
3381        let schema = Schema::parse_str(
3382            r#"{"type": "enum", "name": "Coin", "doc": "Some documentation", "symbols": ["heads", "tails"]}"#,
3383        )?;
3384
3385        let doc = match schema {
3386            Schema::Enum(EnumSchema { doc, .. }) => doc,
3387            _ => None,
3388        };
3389
3390        assert_eq!("Some documentation".to_owned(), doc.unwrap());
3391
3392        Ok(())
3393    }
3394
3395    // Tests to ensure Schema is Send + Sync. These tests don't need to _do_ anything, if they can
3396    // compile, they pass.
3397    #[test]
3398    fn test_schema_is_send() {
3399        fn send<S: Send>(_s: S) {}
3400
3401        let schema = Schema::Null;
3402        send(schema);
3403    }
3404
3405    #[test]
3406    fn test_schema_is_sync() {
3407        fn sync<S: Sync>(_s: S) {}
3408
3409        let schema = Schema::Null;
3410        sync(&schema);
3411        sync(schema);
3412    }
3413
3414    #[test]
3415    fn test_schema_fingerprint() -> TestResult {
3416        use crate::rabin::Rabin;
3417        use md5::Md5;
3418        use sha2::Sha256;
3419
3420        let raw_schema = r#"
3421    {
3422        "type": "record",
3423        "name": "test",
3424        "fields": [
3425            {"name": "a", "type": "long", "default": 42},
3426            {"name": "b", "type": "string"},
3427            {"name": "c", "type": "long", "logicalType": "timestamp-micros"}
3428        ]
3429    }
3430"#;
3431
3432        let schema = Schema::parse_str(raw_schema)?;
3433        assert_eq!(
3434            "7eb3b28d73dfc99bdd9af1848298b40804a2f8ad5d2642be2ecc2ad34842b987",
3435            format!("{}", schema.fingerprint::<Sha256>())
3436        );
3437
3438        assert_eq!(
3439            "cb11615e412ee5d872620d8df78ff6ae",
3440            format!("{}", schema.fingerprint::<Md5>())
3441        );
3442        assert_eq!(
3443            "92f2ccef718c6754",
3444            format!("{}", schema.fingerprint::<Rabin>())
3445        );
3446
3447        Ok(())
3448    }
3449
3450    #[test]
3451    fn test_logical_types() -> TestResult {
3452        let schema = Schema::parse_str(r#"{"type": "int", "logicalType": "date"}"#)?;
3453        assert_eq!(schema, Schema::Date);
3454
3455        let schema = Schema::parse_str(r#"{"type": "long", "logicalType": "timestamp-micros"}"#)?;
3456        assert_eq!(schema, Schema::TimestampMicros);
3457
3458        Ok(())
3459    }
3460
3461    #[test]
3462    fn test_nullable_logical_type() -> TestResult {
3463        let schema = Schema::parse_str(
3464            r#"{"type": ["null", {"type": "long", "logicalType": "timestamp-micros"}]}"#,
3465        )?;
3466        assert_eq!(
3467            schema,
3468            Schema::Union(UnionSchema::new(vec![
3469                Schema::Null,
3470                Schema::TimestampMicros,
3471            ])?)
3472        );
3473
3474        Ok(())
3475    }
3476
3477    #[test]
3478    fn record_field_order_from_str() -> TestResult {
3479        use std::str::FromStr;
3480
3481        assert_eq!(
3482            RecordFieldOrder::from_str("ascending").unwrap(),
3483            RecordFieldOrder::Ascending
3484        );
3485        assert_eq!(
3486            RecordFieldOrder::from_str("descending").unwrap(),
3487            RecordFieldOrder::Descending
3488        );
3489        assert_eq!(
3490            RecordFieldOrder::from_str("ignore").unwrap(),
3491            RecordFieldOrder::Ignore
3492        );
3493        assert!(RecordFieldOrder::from_str("not an ordering").is_err());
3494
3495        Ok(())
3496    }
3497
3498    #[test]
3499    fn test_avro_3374_preserve_namespace_for_primitive() -> TestResult {
3500        let schema = Schema::parse_str(
3501            r#"
3502            {
3503              "type" : "record",
3504              "name" : "ns.int",
3505              "fields" : [
3506                {"name" : "value", "type" : "int"},
3507                {"name" : "next", "type" : [ "null", "ns.int" ]}
3508              ]
3509            }
3510            "#,
3511        )?;
3512
3513        let json = schema.canonical_form();
3514        assert_eq!(
3515            json,
3516            r#"{"name":"ns.int","type":"record","fields":[{"name":"value","type":"int"},{"name":"next","type":["null","ns.int"]}]}"#
3517        );
3518
3519        Ok(())
3520    }
3521
3522    #[test]
3523    fn test_avro_3433_preserve_schema_refs_in_json() -> TestResult {
3524        let schema = r#"
3525    {
3526      "name": "test.test",
3527      "type": "record",
3528      "fields": [
3529        {
3530          "name": "bar",
3531          "type": { "name": "test.foo", "type": "record", "fields": [{ "name": "id", "type": "long" }] }
3532        },
3533        { "name": "baz", "type": "test.foo" }
3534      ]
3535    }
3536    "#;
3537
3538        let schema = Schema::parse_str(schema)?;
3539
3540        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"}]}"#;
3541        assert_eq!(schema.canonical_form(), expected);
3542
3543        Ok(())
3544    }
3545
3546    #[test]
3547    fn test_read_namespace_from_name() -> TestResult {
3548        let schema = r#"
3549    {
3550      "name": "space.name",
3551      "type": "record",
3552      "fields": [
3553        {
3554          "name": "num",
3555          "type": "int"
3556        }
3557      ]
3558    }
3559    "#;
3560
3561        let schema = Schema::parse_str(schema)?;
3562        if let Schema::Record(RecordSchema { name, .. }) = schema {
3563            assert_eq!(name.name, "name");
3564            assert_eq!(name.namespace, Some("space".to_string()));
3565        } else {
3566            panic!("Expected a record schema!");
3567        }
3568
3569        Ok(())
3570    }
3571
3572    #[test]
3573    fn test_namespace_from_name_has_priority_over_from_field() -> TestResult {
3574        let schema = r#"
3575    {
3576      "name": "space1.name",
3577      "namespace": "space2",
3578      "type": "record",
3579      "fields": [
3580        {
3581          "name": "num",
3582          "type": "int"
3583        }
3584      ]
3585    }
3586    "#;
3587
3588        let schema = Schema::parse_str(schema)?;
3589        if let Schema::Record(RecordSchema { name, .. }) = schema {
3590            assert_eq!(name.namespace, Some("space1".to_string()));
3591        } else {
3592            panic!("Expected a record schema!");
3593        }
3594
3595        Ok(())
3596    }
3597
3598    #[test]
3599    fn test_namespace_from_field() -> TestResult {
3600        let schema = r#"
3601    {
3602      "name": "name",
3603      "namespace": "space2",
3604      "type": "record",
3605      "fields": [
3606        {
3607          "name": "num",
3608          "type": "int"
3609        }
3610      ]
3611    }
3612    "#;
3613
3614        let schema = Schema::parse_str(schema)?;
3615        if let Schema::Record(RecordSchema { name, .. }) = schema {
3616            assert_eq!(name.namespace, Some("space2".to_string()));
3617        } else {
3618            panic!("Expected a record schema!");
3619        }
3620
3621        Ok(())
3622    }
3623
3624    #[test]
3625    /// Zero-length namespace is considered as no-namespace.
3626    fn test_namespace_from_name_with_empty_value() -> TestResult {
3627        let name = Name::new(".name")?;
3628        assert_eq!(name.name, "name");
3629        assert_eq!(name.namespace, None);
3630
3631        Ok(())
3632    }
3633
3634    #[test]
3635    /// Whitespace is not allowed in the name.
3636    fn test_name_with_whitespace_value() {
3637        match Name::new(" ") {
3638            Err(Error::InvalidSchemaName(_, _)) => {}
3639            _ => panic!("Expected an Error::InvalidSchemaName!"),
3640        }
3641    }
3642
3643    #[test]
3644    /// The name must be non-empty.
3645    fn test_name_with_no_name_part() {
3646        match Name::new("space.") {
3647            Err(Error::InvalidSchemaName(_, _)) => {}
3648            _ => panic!("Expected an Error::InvalidSchemaName!"),
3649        }
3650    }
3651
3652    #[test]
3653    fn avro_3448_test_proper_resolution_inner_record_inherited_namespace() -> TestResult {
3654        let schema = r#"
3655        {
3656          "name": "record_name",
3657          "namespace": "space",
3658          "type": "record",
3659          "fields": [
3660            {
3661              "name": "outer_field_1",
3662              "type": [
3663                        "null",
3664                        {
3665                            "type":"record",
3666                            "name":"inner_record_name",
3667                            "fields":[
3668                                {
3669                                    "name":"inner_field_1",
3670                                    "type":"double"
3671                                }
3672                            ]
3673                        }
3674                    ]
3675            },
3676            {
3677                "name": "outer_field_2",
3678                "type" : "inner_record_name"
3679            }
3680          ]
3681        }
3682        "#;
3683        let schema = Schema::parse_str(schema)?;
3684        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
3685        assert_eq!(rs.get_names().len(), 2);
3686        for s in &["space.record_name", "space.inner_record_name"] {
3687            assert!(rs.get_names().contains_key(&Name::new(s)?));
3688        }
3689
3690        Ok(())
3691    }
3692
3693    #[test]
3694    fn avro_3448_test_proper_resolution_inner_record_qualified_namespace() -> TestResult {
3695        let schema = r#"
3696        {
3697          "name": "record_name",
3698          "namespace": "space",
3699          "type": "record",
3700          "fields": [
3701            {
3702              "name": "outer_field_1",
3703              "type": [
3704                        "null",
3705                        {
3706                            "type":"record",
3707                            "name":"inner_record_name",
3708                            "fields":[
3709                                {
3710                                    "name":"inner_field_1",
3711                                    "type":"double"
3712                                }
3713                            ]
3714                        }
3715                    ]
3716            },
3717            {
3718                "name": "outer_field_2",
3719                "type" : "space.inner_record_name"
3720            }
3721          ]
3722        }
3723        "#;
3724        let schema = Schema::parse_str(schema)?;
3725        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
3726        assert_eq!(rs.get_names().len(), 2);
3727        for s in &["space.record_name", "space.inner_record_name"] {
3728            assert!(rs.get_names().contains_key(&Name::new(s)?));
3729        }
3730
3731        Ok(())
3732    }
3733
3734    #[test]
3735    fn avro_3448_test_proper_resolution_inner_enum_inherited_namespace() -> TestResult {
3736        let schema = r#"
3737        {
3738          "name": "record_name",
3739          "namespace": "space",
3740          "type": "record",
3741          "fields": [
3742            {
3743              "name": "outer_field_1",
3744              "type": [
3745                        "null",
3746                        {
3747                            "type":"enum",
3748                            "name":"inner_enum_name",
3749                            "symbols":["Extensive","Testing"]
3750                        }
3751                    ]
3752            },
3753            {
3754                "name": "outer_field_2",
3755                "type" : "inner_enum_name"
3756            }
3757          ]
3758        }
3759        "#;
3760        let schema = Schema::parse_str(schema)?;
3761        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
3762        assert_eq!(rs.get_names().len(), 2);
3763        for s in &["space.record_name", "space.inner_enum_name"] {
3764            assert!(rs.get_names().contains_key(&Name::new(s)?));
3765        }
3766
3767        Ok(())
3768    }
3769
3770    #[test]
3771    fn avro_3448_test_proper_resolution_inner_enum_qualified_namespace() -> TestResult {
3772        let schema = r#"
3773        {
3774          "name": "record_name",
3775          "namespace": "space",
3776          "type": "record",
3777          "fields": [
3778            {
3779              "name": "outer_field_1",
3780              "type": [
3781                        "null",
3782                        {
3783                            "type":"enum",
3784                            "name":"inner_enum_name",
3785                            "symbols":["Extensive","Testing"]
3786                        }
3787                    ]
3788            },
3789            {
3790                "name": "outer_field_2",
3791                "type" : "space.inner_enum_name"
3792            }
3793          ]
3794        }
3795        "#;
3796        let schema = Schema::parse_str(schema)?;
3797        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
3798        assert_eq!(rs.get_names().len(), 2);
3799        for s in &["space.record_name", "space.inner_enum_name"] {
3800            assert!(rs.get_names().contains_key(&Name::new(s)?));
3801        }
3802
3803        Ok(())
3804    }
3805
3806    #[test]
3807    fn avro_3448_test_proper_resolution_inner_fixed_inherited_namespace() -> TestResult {
3808        let schema = r#"
3809        {
3810          "name": "record_name",
3811          "namespace": "space",
3812          "type": "record",
3813          "fields": [
3814            {
3815              "name": "outer_field_1",
3816              "type": [
3817                        "null",
3818                        {
3819                            "type":"fixed",
3820                            "name":"inner_fixed_name",
3821                            "size": 16
3822                        }
3823                    ]
3824            },
3825            {
3826                "name": "outer_field_2",
3827                "type" : "inner_fixed_name"
3828            }
3829          ]
3830        }
3831        "#;
3832        let schema = Schema::parse_str(schema)?;
3833        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
3834        assert_eq!(rs.get_names().len(), 2);
3835        for s in &["space.record_name", "space.inner_fixed_name"] {
3836            assert!(rs.get_names().contains_key(&Name::new(s)?));
3837        }
3838
3839        Ok(())
3840    }
3841
3842    #[test]
3843    fn avro_3448_test_proper_resolution_inner_fixed_qualified_namespace() -> TestResult {
3844        let schema = r#"
3845        {
3846          "name": "record_name",
3847          "namespace": "space",
3848          "type": "record",
3849          "fields": [
3850            {
3851              "name": "outer_field_1",
3852              "type": [
3853                        "null",
3854                        {
3855                            "type":"fixed",
3856                            "name":"inner_fixed_name",
3857                            "size": 16
3858                        }
3859                    ]
3860            },
3861            {
3862                "name": "outer_field_2",
3863                "type" : "space.inner_fixed_name"
3864            }
3865          ]
3866        }
3867        "#;
3868        let schema = Schema::parse_str(schema)?;
3869        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
3870        assert_eq!(rs.get_names().len(), 2);
3871        for s in &["space.record_name", "space.inner_fixed_name"] {
3872            assert!(rs.get_names().contains_key(&Name::new(s)?));
3873        }
3874
3875        Ok(())
3876    }
3877
3878    #[test]
3879    fn avro_3448_test_proper_resolution_inner_record_inner_namespace() -> TestResult {
3880        let schema = r#"
3881        {
3882          "name": "record_name",
3883          "namespace": "space",
3884          "type": "record",
3885          "fields": [
3886            {
3887              "name": "outer_field_1",
3888              "type": [
3889                        "null",
3890                        {
3891                            "type":"record",
3892                            "name":"inner_record_name",
3893                            "namespace":"inner_space",
3894                            "fields":[
3895                                {
3896                                    "name":"inner_field_1",
3897                                    "type":"double"
3898                                }
3899                            ]
3900                        }
3901                    ]
3902            },
3903            {
3904                "name": "outer_field_2",
3905                "type" : "inner_space.inner_record_name"
3906            }
3907          ]
3908        }
3909        "#;
3910        let schema = Schema::parse_str(schema)?;
3911        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
3912        assert_eq!(rs.get_names().len(), 2);
3913        for s in &["space.record_name", "inner_space.inner_record_name"] {
3914            assert!(rs.get_names().contains_key(&Name::new(s)?));
3915        }
3916
3917        Ok(())
3918    }
3919
3920    #[test]
3921    fn avro_3448_test_proper_resolution_inner_enum_inner_namespace() -> TestResult {
3922        let schema = r#"
3923        {
3924          "name": "record_name",
3925          "namespace": "space",
3926          "type": "record",
3927          "fields": [
3928            {
3929              "name": "outer_field_1",
3930              "type": [
3931                        "null",
3932                        {
3933                            "type":"enum",
3934                            "name":"inner_enum_name",
3935                            "namespace": "inner_space",
3936                            "symbols":["Extensive","Testing"]
3937                        }
3938                    ]
3939            },
3940            {
3941                "name": "outer_field_2",
3942                "type" : "inner_space.inner_enum_name"
3943            }
3944          ]
3945        }
3946        "#;
3947        let schema = Schema::parse_str(schema)?;
3948        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
3949        assert_eq!(rs.get_names().len(), 2);
3950        for s in &["space.record_name", "inner_space.inner_enum_name"] {
3951            assert!(rs.get_names().contains_key(&Name::new(s)?));
3952        }
3953
3954        Ok(())
3955    }
3956
3957    #[test]
3958    fn avro_3448_test_proper_resolution_inner_fixed_inner_namespace() -> TestResult {
3959        let schema = r#"
3960        {
3961          "name": "record_name",
3962          "namespace": "space",
3963          "type": "record",
3964          "fields": [
3965            {
3966              "name": "outer_field_1",
3967              "type": [
3968                        "null",
3969                        {
3970                            "type":"fixed",
3971                            "name":"inner_fixed_name",
3972                            "namespace": "inner_space",
3973                            "size": 16
3974                        }
3975                    ]
3976            },
3977            {
3978                "name": "outer_field_2",
3979                "type" : "inner_space.inner_fixed_name"
3980            }
3981          ]
3982        }
3983        "#;
3984        let schema = Schema::parse_str(schema)?;
3985        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
3986        assert_eq!(rs.get_names().len(), 2);
3987        for s in &["space.record_name", "inner_space.inner_fixed_name"] {
3988            assert!(rs.get_names().contains_key(&Name::new(s)?));
3989        }
3990
3991        Ok(())
3992    }
3993
3994    #[test]
3995    fn avro_3448_test_proper_multi_level_resolution_inner_record_outer_namespace() -> TestResult {
3996        let schema = r#"
3997        {
3998          "name": "record_name",
3999          "namespace": "space",
4000          "type": "record",
4001          "fields": [
4002            {
4003              "name": "outer_field_1",
4004              "type": [
4005                        "null",
4006                        {
4007                            "type":"record",
4008                            "name":"middle_record_name",
4009                            "fields":[
4010                                {
4011                                    "name":"middle_field_1",
4012                                    "type":[
4013                                        "null",
4014                                        {
4015                                            "type":"record",
4016                                            "name":"inner_record_name",
4017                                            "fields":[
4018                                                {
4019                                                    "name":"inner_field_1",
4020                                                    "type":"double"
4021                                                }
4022                                            ]
4023                                        }
4024                                    ]
4025                                }
4026                            ]
4027                        }
4028                    ]
4029            },
4030            {
4031                "name": "outer_field_2",
4032                "type" : "space.inner_record_name"
4033            }
4034          ]
4035        }
4036        "#;
4037        let schema = Schema::parse_str(schema)?;
4038        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
4039        assert_eq!(rs.get_names().len(), 3);
4040        for s in &[
4041            "space.record_name",
4042            "space.middle_record_name",
4043            "space.inner_record_name",
4044        ] {
4045            assert!(rs.get_names().contains_key(&Name::new(s)?));
4046        }
4047
4048        Ok(())
4049    }
4050
4051    #[test]
4052    fn avro_3448_test_proper_multi_level_resolution_inner_record_middle_namespace() -> TestResult {
4053        let schema = r#"
4054        {
4055          "name": "record_name",
4056          "namespace": "space",
4057          "type": "record",
4058          "fields": [
4059            {
4060              "name": "outer_field_1",
4061              "type": [
4062                        "null",
4063                        {
4064                            "type":"record",
4065                            "name":"middle_record_name",
4066                            "namespace":"middle_namespace",
4067                            "fields":[
4068                                {
4069                                    "name":"middle_field_1",
4070                                    "type":[
4071                                        "null",
4072                                        {
4073                                            "type":"record",
4074                                            "name":"inner_record_name",
4075                                            "fields":[
4076                                                {
4077                                                    "name":"inner_field_1",
4078                                                    "type":"double"
4079                                                }
4080                                            ]
4081                                        }
4082                                    ]
4083                                }
4084                            ]
4085                        }
4086                    ]
4087            },
4088            {
4089                "name": "outer_field_2",
4090                "type" : "middle_namespace.inner_record_name"
4091            }
4092          ]
4093        }
4094        "#;
4095        let schema = Schema::parse_str(schema)?;
4096        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
4097        assert_eq!(rs.get_names().len(), 3);
4098        for s in &[
4099            "space.record_name",
4100            "middle_namespace.middle_record_name",
4101            "middle_namespace.inner_record_name",
4102        ] {
4103            assert!(rs.get_names().contains_key(&Name::new(s)?));
4104        }
4105
4106        Ok(())
4107    }
4108
4109    #[test]
4110    fn avro_3448_test_proper_multi_level_resolution_inner_record_inner_namespace() -> TestResult {
4111        let schema = r#"
4112        {
4113          "name": "record_name",
4114          "namespace": "space",
4115          "type": "record",
4116          "fields": [
4117            {
4118              "name": "outer_field_1",
4119              "type": [
4120                        "null",
4121                        {
4122                            "type":"record",
4123                            "name":"middle_record_name",
4124                            "namespace":"middle_namespace",
4125                            "fields":[
4126                                {
4127                                    "name":"middle_field_1",
4128                                    "type":[
4129                                        "null",
4130                                        {
4131                                            "type":"record",
4132                                            "name":"inner_record_name",
4133                                            "namespace":"inner_namespace",
4134                                            "fields":[
4135                                                {
4136                                                    "name":"inner_field_1",
4137                                                    "type":"double"
4138                                                }
4139                                            ]
4140                                        }
4141                                    ]
4142                                }
4143                            ]
4144                        }
4145                    ]
4146            },
4147            {
4148                "name": "outer_field_2",
4149                "type" : "inner_namespace.inner_record_name"
4150            }
4151          ]
4152        }
4153        "#;
4154        let schema = Schema::parse_str(schema)?;
4155        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
4156        assert_eq!(rs.get_names().len(), 3);
4157        for s in &[
4158            "space.record_name",
4159            "middle_namespace.middle_record_name",
4160            "inner_namespace.inner_record_name",
4161        ] {
4162            assert!(rs.get_names().contains_key(&Name::new(s)?));
4163        }
4164
4165        Ok(())
4166    }
4167
4168    #[test]
4169    fn avro_3448_test_proper_in_array_resolution_inherited_namespace() -> TestResult {
4170        let schema = r#"
4171        {
4172          "name": "record_name",
4173          "namespace": "space",
4174          "type": "record",
4175          "fields": [
4176            {
4177              "name": "outer_field_1",
4178              "type": {
4179                  "type":"array",
4180                  "items":{
4181                      "type":"record",
4182                      "name":"in_array_record",
4183                      "fields": [
4184                          {
4185                              "name":"array_record_field",
4186                              "type":"string"
4187                          }
4188                      ]
4189                  }
4190              }
4191            },
4192            {
4193                "name":"outer_field_2",
4194                "type":"in_array_record"
4195            }
4196          ]
4197        }
4198        "#;
4199        let schema = Schema::parse_str(schema)?;
4200        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
4201        assert_eq!(rs.get_names().len(), 2);
4202        for s in &["space.record_name", "space.in_array_record"] {
4203            assert!(rs.get_names().contains_key(&Name::new(s)?));
4204        }
4205
4206        Ok(())
4207    }
4208
4209    #[test]
4210    fn avro_3448_test_proper_in_map_resolution_inherited_namespace() -> TestResult {
4211        let schema = r#"
4212        {
4213          "name": "record_name",
4214          "namespace": "space",
4215          "type": "record",
4216          "fields": [
4217            {
4218              "name": "outer_field_1",
4219              "type": {
4220                  "type":"map",
4221                  "values":{
4222                      "type":"record",
4223                      "name":"in_map_record",
4224                      "fields": [
4225                          {
4226                              "name":"map_record_field",
4227                              "type":"string"
4228                          }
4229                      ]
4230                  }
4231              }
4232            },
4233            {
4234                "name":"outer_field_2",
4235                "type":"in_map_record"
4236            }
4237          ]
4238        }
4239        "#;
4240        let schema = Schema::parse_str(schema)?;
4241        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
4242        assert_eq!(rs.get_names().len(), 2);
4243        for s in &["space.record_name", "space.in_map_record"] {
4244            assert!(rs.get_names().contains_key(&Name::new(s)?));
4245        }
4246
4247        Ok(())
4248    }
4249
4250    #[test]
4251    fn avro_3466_test_to_json_inner_enum_inner_namespace() -> TestResult {
4252        let schema = r#"
4253        {
4254        "name": "record_name",
4255        "namespace": "space",
4256        "type": "record",
4257        "fields": [
4258            {
4259            "name": "outer_field_1",
4260            "type": [
4261                        "null",
4262                        {
4263                            "type":"enum",
4264                            "name":"inner_enum_name",
4265                            "namespace": "inner_space",
4266                            "symbols":["Extensive","Testing"]
4267                        }
4268                    ]
4269            },
4270            {
4271                "name": "outer_field_2",
4272                "type" : "inner_space.inner_enum_name"
4273            }
4274        ]
4275        }
4276        "#;
4277        let schema = Schema::parse_str(schema)?;
4278        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
4279
4280        // confirm we have expected 2 full-names
4281        assert_eq!(rs.get_names().len(), 2);
4282        for s in &["space.record_name", "inner_space.inner_enum_name"] {
4283            assert!(rs.get_names().contains_key(&Name::new(s)?));
4284        }
4285
4286        // convert Schema back to JSON string
4287        let schema_str = serde_json::to_string(&schema).expect("test failed");
4288        let _schema = Schema::parse_str(&schema_str).expect("test failed");
4289        assert_eq!(schema, _schema);
4290
4291        Ok(())
4292    }
4293
4294    #[test]
4295    fn avro_3466_test_to_json_inner_fixed_inner_namespace() -> TestResult {
4296        let schema = r#"
4297        {
4298        "name": "record_name",
4299        "namespace": "space",
4300        "type": "record",
4301        "fields": [
4302            {
4303            "name": "outer_field_1",
4304            "type": [
4305                        "null",
4306                        {
4307                            "type":"fixed",
4308                            "name":"inner_fixed_name",
4309                            "namespace": "inner_space",
4310                            "size":54
4311                        }
4312                    ]
4313            },
4314            {
4315                "name": "outer_field_2",
4316                "type" : "inner_space.inner_fixed_name"
4317            }
4318        ]
4319        }
4320        "#;
4321        let schema = Schema::parse_str(schema)?;
4322        let rs = ResolvedSchema::try_from(&schema).expect("Schema didn't successfully parse");
4323
4324        // confirm we have expected 2 full-names
4325        assert_eq!(rs.get_names().len(), 2);
4326        for s in &["space.record_name", "inner_space.inner_fixed_name"] {
4327            assert!(rs.get_names().contains_key(&Name::new(s)?));
4328        }
4329
4330        // convert Schema back to JSON string
4331        let schema_str = serde_json::to_string(&schema).expect("test failed");
4332        let _schema = Schema::parse_str(&schema_str).expect("test failed");
4333        assert_eq!(schema, _schema);
4334
4335        Ok(())
4336    }
4337
4338    fn assert_avro_3512_aliases(aliases: &Aliases) {
4339        match aliases {
4340            Some(aliases) => {
4341                assert_eq!(aliases.len(), 3);
4342                assert_eq!(aliases[0], Alias::new("space.b").unwrap());
4343                assert_eq!(aliases[1], Alias::new("x.y").unwrap());
4344                assert_eq!(aliases[2], Alias::new(".c").unwrap());
4345            }
4346            None => {
4347                panic!("'aliases' must be Some");
4348            }
4349        }
4350    }
4351
4352    #[test]
4353    fn avro_3512_alias_with_null_namespace_record() -> TestResult {
4354        let schema = Schema::parse_str(
4355            r#"
4356            {
4357              "type": "record",
4358              "name": "a",
4359              "namespace": "space",
4360              "aliases": ["b", "x.y", ".c"],
4361              "fields" : [
4362                {"name": "time", "type": "long"}
4363              ]
4364            }
4365        "#,
4366        )?;
4367
4368        if let Schema::Record(RecordSchema { ref aliases, .. }) = schema {
4369            assert_avro_3512_aliases(aliases);
4370        } else {
4371            panic!("The Schema should be a record: {schema:?}");
4372        }
4373
4374        Ok(())
4375    }
4376
4377    #[test]
4378    fn avro_3512_alias_with_null_namespace_enum() -> TestResult {
4379        let schema = Schema::parse_str(
4380            r#"
4381            {
4382              "type": "enum",
4383              "name": "a",
4384              "namespace": "space",
4385              "aliases": ["b", "x.y", ".c"],
4386              "symbols" : [
4387                "symbol1", "symbol2"
4388              ]
4389            }
4390        "#,
4391        )?;
4392
4393        if let Schema::Enum(EnumSchema { ref aliases, .. }) = schema {
4394            assert_avro_3512_aliases(aliases);
4395        } else {
4396            panic!("The Schema should be an enum: {schema:?}");
4397        }
4398
4399        Ok(())
4400    }
4401
4402    #[test]
4403    fn avro_3512_alias_with_null_namespace_fixed() -> TestResult {
4404        let schema = Schema::parse_str(
4405            r#"
4406            {
4407              "type": "fixed",
4408              "name": "a",
4409              "namespace": "space",
4410              "aliases": ["b", "x.y", ".c"],
4411              "size" : 12
4412            }
4413        "#,
4414        )?;
4415
4416        if let Schema::Fixed(FixedSchema { ref aliases, .. }) = schema {
4417            assert_avro_3512_aliases(aliases);
4418        } else {
4419            panic!("The Schema should be a fixed: {schema:?}");
4420        }
4421
4422        Ok(())
4423    }
4424
4425    #[test]
4426    fn avro_3518_serialize_aliases_record() -> TestResult {
4427        let schema = Schema::parse_str(
4428            r#"
4429            {
4430              "type": "record",
4431              "name": "a",
4432              "namespace": "space",
4433              "aliases": ["b", "x.y", ".c"],
4434              "fields" : [
4435                {
4436                    "name": "time",
4437                    "type": "long",
4438                    "doc": "The documentation is not serialized",
4439                    "default": 123,
4440                    "aliases": ["time1", "ns.time2"]
4441                }
4442              ]
4443            }
4444        "#,
4445        )?;
4446
4447        let value = serde_json::to_value(&schema)?;
4448        let serialized = serde_json::to_string(&value)?;
4449        assert_eq!(
4450            r#"{"aliases":["space.b","x.y","c"],"fields":[{"aliases":["time1","ns.time2"],"default":123,"name":"time","type":"long"}],"name":"a","namespace":"space","type":"record"}"#,
4451            &serialized
4452        );
4453        assert_eq!(schema, Schema::parse_str(&serialized)?);
4454
4455        Ok(())
4456    }
4457
4458    #[test]
4459    fn avro_3518_serialize_aliases_enum() -> TestResult {
4460        let schema = Schema::parse_str(
4461            r#"
4462            {
4463              "type": "enum",
4464              "name": "a",
4465              "namespace": "space",
4466              "aliases": ["b", "x.y", ".c"],
4467              "symbols" : [
4468                "symbol1", "symbol2"
4469              ]
4470            }
4471        "#,
4472        )?;
4473
4474        let value = serde_json::to_value(&schema)?;
4475        let serialized = serde_json::to_string(&value)?;
4476        assert_eq!(
4477            r#"{"aliases":["space.b","x.y","c"],"name":"a","namespace":"space","symbols":["symbol1","symbol2"],"type":"enum"}"#,
4478            &serialized
4479        );
4480        assert_eq!(schema, Schema::parse_str(&serialized)?);
4481
4482        Ok(())
4483    }
4484
4485    #[test]
4486    fn avro_3518_serialize_aliases_fixed() -> TestResult {
4487        let schema = Schema::parse_str(
4488            r#"
4489            {
4490              "type": "fixed",
4491              "name": "a",
4492              "namespace": "space",
4493              "aliases": ["b", "x.y", ".c"],
4494              "size" : 12
4495            }
4496        "#,
4497        )?;
4498
4499        let value = serde_json::to_value(&schema)?;
4500        let serialized = serde_json::to_string(&value)?;
4501        assert_eq!(
4502            r#"{"aliases":["space.b","x.y","c"],"name":"a","namespace":"space","size":12,"type":"fixed"}"#,
4503            &serialized
4504        );
4505        assert_eq!(schema, Schema::parse_str(&serialized)?);
4506
4507        Ok(())
4508    }
4509
4510    #[test]
4511    fn avro_3130_parse_anonymous_union_type() -> TestResult {
4512        let schema_str = r#"
4513        {
4514            "type": "record",
4515            "name": "AccountEvent",
4516            "fields": [
4517                {"type":
4518                  ["null",
4519                   { "name": "accountList",
4520                      "type": {
4521                        "type": "array",
4522                        "items": "long"
4523                      }
4524                  }
4525                  ],
4526                 "name":"NullableLongArray"
4527               }
4528            ]
4529        }
4530        "#;
4531        let schema = Schema::parse_str(schema_str)?;
4532
4533        if let Schema::Record(RecordSchema { name, fields, .. }) = schema {
4534            assert_eq!(name, Name::new("AccountEvent")?);
4535
4536            let field = &fields[0];
4537            assert_eq!(&field.name, "NullableLongArray");
4538
4539            if let Schema::Union(ref union) = field.schema {
4540                assert_eq!(union.schemas[0], Schema::Null);
4541
4542                if let Schema::Array(ref array_schema) = union.schemas[1] {
4543                    if let Schema::Long = *array_schema.items {
4544                        // OK
4545                    } else {
4546                        panic!("Expected a Schema::Array of type Long");
4547                    }
4548                } else {
4549                    panic!("Expected Schema::Array");
4550                }
4551            } else {
4552                panic!("Expected Schema::Union");
4553            }
4554        } else {
4555            panic!("Expected Schema::Record");
4556        }
4557
4558        Ok(())
4559    }
4560
4561    #[test]
4562    fn avro_custom_attributes_schema_without_attributes() -> TestResult {
4563        let schemata_str = [
4564            r#"
4565            {
4566                "type": "record",
4567                "name": "Rec",
4568                "doc": "A Record schema without custom attributes",
4569                "fields": []
4570            }
4571            "#,
4572            r#"
4573            {
4574                "type": "enum",
4575                "name": "Enum",
4576                "doc": "An Enum schema without custom attributes",
4577                "symbols": []
4578            }
4579            "#,
4580            r#"
4581            {
4582                "type": "fixed",
4583                "name": "Fixed",
4584                "doc": "A Fixed schema without custom attributes",
4585                "size": 0
4586            }
4587            "#,
4588        ];
4589        for schema_str in schemata_str.iter() {
4590            let schema = Schema::parse_str(schema_str)?;
4591            assert_eq!(schema.custom_attributes(), Some(&Default::default()));
4592        }
4593
4594        Ok(())
4595    }
4596
4597    const CUSTOM_ATTRS_SUFFIX: &str = r#"
4598            "string_key": "value",
4599            "number_key": 1.23,
4600            "null_key": null,
4601            "array_key": [1, 2, 3],
4602            "object_key": {
4603                "key": "value"
4604            }
4605        "#;
4606
4607    #[test]
4608    fn avro_3609_custom_attributes_schema_with_attributes() -> TestResult {
4609        let schemata_str = [
4610            r#"
4611            {
4612                "type": "record",
4613                "name": "Rec",
4614                "namespace": "ns",
4615                "doc": "A Record schema with custom attributes",
4616                "fields": [],
4617                {{{}}}
4618            }
4619            "#,
4620            r#"
4621            {
4622                "type": "enum",
4623                "name": "Enum",
4624                "namespace": "ns",
4625                "doc": "An Enum schema with custom attributes",
4626                "symbols": [],
4627                {{{}}}
4628            }
4629            "#,
4630            r#"
4631            {
4632                "type": "fixed",
4633                "name": "Fixed",
4634                "namespace": "ns",
4635                "doc": "A Fixed schema with custom attributes",
4636                "size": 2,
4637                {{{}}}
4638            }
4639            "#,
4640        ];
4641
4642        for schema_str in schemata_str.iter() {
4643            let schema = Schema::parse_str(
4644                schema_str
4645                    .to_owned()
4646                    .replace("{{{}}}", CUSTOM_ATTRS_SUFFIX)
4647                    .as_str(),
4648            )?;
4649
4650            assert_eq!(
4651                schema.custom_attributes(),
4652                Some(&expected_custom_attributes())
4653            );
4654        }
4655
4656        Ok(())
4657    }
4658
4659    fn expected_custom_attributes() -> BTreeMap<String, Value> {
4660        let mut expected_attributes: BTreeMap<String, Value> = Default::default();
4661        expected_attributes.insert("string_key".to_string(), Value::String("value".to_string()));
4662        expected_attributes.insert("number_key".to_string(), json!(1.23));
4663        expected_attributes.insert("null_key".to_string(), Value::Null);
4664        expected_attributes.insert(
4665            "array_key".to_string(),
4666            Value::Array(vec![json!(1), json!(2), json!(3)]),
4667        );
4668        let mut object_value: HashMap<String, Value> = HashMap::new();
4669        object_value.insert("key".to_string(), Value::String("value".to_string()));
4670        expected_attributes.insert("object_key".to_string(), json!(object_value));
4671        expected_attributes
4672    }
4673
4674    #[test]
4675    fn avro_3609_custom_attributes_record_field_without_attributes() -> TestResult {
4676        let schema_str = String::from(
4677            r#"
4678            {
4679                "type": "record",
4680                "name": "Rec",
4681                "doc": "A Record schema without custom attributes",
4682                "fields": [
4683                    {
4684                        "name": "field_one",
4685                        "type": "float",
4686                        {{{}}}
4687                    }
4688                ]
4689            }
4690        "#,
4691        );
4692
4693        let schema = Schema::parse_str(schema_str.replace("{{{}}}", CUSTOM_ATTRS_SUFFIX).as_str())?;
4694
4695        match schema {
4696            Schema::Record(RecordSchema { name, fields, .. }) => {
4697                assert_eq!(name, Name::new("Rec")?);
4698                assert_eq!(fields.len(), 1);
4699                let field = &fields[0];
4700                assert_eq!(&field.name, "field_one");
4701                assert_eq!(field.custom_attributes, expected_custom_attributes());
4702            }
4703            _ => panic!("Expected Schema::Record"),
4704        }
4705
4706        Ok(())
4707    }
4708
4709    #[test]
4710    fn avro_3625_null_is_first() -> TestResult {
4711        let schema_str = String::from(
4712            r#"
4713            {
4714                "type": "record",
4715                "name": "union_schema_test",
4716                "fields": [
4717                    {"name": "a", "type": ["null", "long"], "default": null}
4718                ]
4719            }
4720        "#,
4721        );
4722
4723        let schema = Schema::parse_str(&schema_str)?;
4724
4725        match schema {
4726            Schema::Record(RecordSchema { name, fields, .. }) => {
4727                assert_eq!(name, Name::new("union_schema_test")?);
4728                assert_eq!(fields.len(), 1);
4729                let field = &fields[0];
4730                assert_eq!(&field.name, "a");
4731                assert_eq!(&field.default, &Some(Value::Null));
4732                match &field.schema {
4733                    Schema::Union(union) => {
4734                        assert_eq!(union.variants().len(), 2);
4735                        assert!(union.is_nullable());
4736                        assert_eq!(union.variants()[0], Schema::Null);
4737                        assert_eq!(union.variants()[1], Schema::Long);
4738                    }
4739                    _ => panic!("Expected Schema::Union"),
4740                }
4741            }
4742            _ => panic!("Expected Schema::Record"),
4743        }
4744
4745        Ok(())
4746    }
4747
4748    #[test]
4749    fn avro_3625_null_is_last() -> TestResult {
4750        let schema_str = String::from(
4751            r#"
4752            {
4753                "type": "record",
4754                "name": "union_schema_test",
4755                "fields": [
4756                    {"name": "a", "type": ["long","null"], "default": 123}
4757                ]
4758            }
4759        "#,
4760        );
4761
4762        let schema = Schema::parse_str(&schema_str)?;
4763
4764        match schema {
4765            Schema::Record(RecordSchema { name, fields, .. }) => {
4766                assert_eq!(name, Name::new("union_schema_test")?);
4767                assert_eq!(fields.len(), 1);
4768                let field = &fields[0];
4769                assert_eq!(&field.name, "a");
4770                assert_eq!(&field.default, &Some(json!(123)));
4771                match &field.schema {
4772                    Schema::Union(union) => {
4773                        assert_eq!(union.variants().len(), 2);
4774                        assert_eq!(union.variants()[0], Schema::Long);
4775                        assert_eq!(union.variants()[1], Schema::Null);
4776                    }
4777                    _ => panic!("Expected Schema::Union"),
4778                }
4779            }
4780            _ => panic!("Expected Schema::Record"),
4781        }
4782
4783        Ok(())
4784    }
4785
4786    #[test]
4787    fn avro_3625_null_is_the_middle() -> TestResult {
4788        let schema_str = String::from(
4789            r#"
4790            {
4791                "type": "record",
4792                "name": "union_schema_test",
4793                "fields": [
4794                    {"name": "a", "type": ["long","null","int"], "default": 123}
4795                ]
4796            }
4797        "#,
4798        );
4799
4800        let schema = Schema::parse_str(&schema_str)?;
4801
4802        match schema {
4803            Schema::Record(RecordSchema { name, fields, .. }) => {
4804                assert_eq!(name, Name::new("union_schema_test")?);
4805                assert_eq!(fields.len(), 1);
4806                let field = &fields[0];
4807                assert_eq!(&field.name, "a");
4808                assert_eq!(&field.default, &Some(json!(123)));
4809                match &field.schema {
4810                    Schema::Union(union) => {
4811                        assert_eq!(union.variants().len(), 3);
4812                        assert_eq!(union.variants()[0], Schema::Long);
4813                        assert_eq!(union.variants()[1], Schema::Null);
4814                        assert_eq!(union.variants()[2], Schema::Int);
4815                    }
4816                    _ => panic!("Expected Schema::Union"),
4817                }
4818            }
4819            _ => panic!("Expected Schema::Record"),
4820        }
4821
4822        Ok(())
4823    }
4824
4825    #[test]
4826    fn avro_3649_default_notintfirst() -> TestResult {
4827        let schema_str = String::from(
4828            r#"
4829            {
4830                "type": "record",
4831                "name": "union_schema_test",
4832                "fields": [
4833                    {"name": "a", "type": ["string", "int"], "default": 123}
4834                ]
4835            }
4836        "#,
4837        );
4838
4839        let schema = Schema::parse_str(&schema_str)?;
4840
4841        match schema {
4842            Schema::Record(RecordSchema { name, fields, .. }) => {
4843                assert_eq!(name, Name::new("union_schema_test")?);
4844                assert_eq!(fields.len(), 1);
4845                let field = &fields[0];
4846                assert_eq!(&field.name, "a");
4847                assert_eq!(&field.default, &Some(json!(123)));
4848                match &field.schema {
4849                    Schema::Union(union) => {
4850                        assert_eq!(union.variants().len(), 2);
4851                        assert_eq!(union.variants()[0], Schema::String);
4852                        assert_eq!(union.variants()[1], Schema::Int);
4853                    }
4854                    _ => panic!("Expected Schema::Union"),
4855                }
4856            }
4857            _ => panic!("Expected Schema::Record"),
4858        }
4859
4860        Ok(())
4861    }
4862
4863    #[test]
4864    fn avro_3709_parsing_of_record_field_aliases() -> TestResult {
4865        let schema = r#"
4866        {
4867          "name": "rec",
4868          "type": "record",
4869          "fields": [
4870            {
4871              "name": "num",
4872              "type": "int",
4873              "aliases": ["num1", "num2"]
4874            }
4875          ]
4876        }
4877        "#;
4878
4879        let schema = Schema::parse_str(schema)?;
4880        if let Schema::Record(RecordSchema { fields, .. }) = schema {
4881            let num_field = &fields[0];
4882            assert_eq!(num_field.name, "num");
4883            assert_eq!(num_field.aliases, Some(vec!("num1".into(), "num2".into())));
4884        } else {
4885            panic!("Expected a record schema!");
4886        }
4887
4888        Ok(())
4889    }
4890
4891    #[test]
4892    fn avro_3735_parse_enum_namespace() -> TestResult {
4893        let schema = r#"
4894        {
4895            "type": "record",
4896            "name": "Foo",
4897            "namespace": "name.space",
4898            "fields":
4899            [
4900                {
4901                    "name": "barInit",
4902                    "type":
4903                    {
4904                        "type": "enum",
4905                        "name": "Bar",
4906                        "symbols":
4907                        [
4908                            "bar0",
4909                            "bar1"
4910                        ]
4911                    }
4912                },
4913                {
4914                    "name": "barUse",
4915                    "type": "Bar"
4916                }
4917            ]
4918        } 
4919        "#;
4920
4921        #[derive(
4922            Debug, PartialEq, Eq, Hash, PartialOrd, Ord, Clone, serde::Deserialize, serde::Serialize,
4923        )]
4924        pub enum Bar {
4925            #[serde(rename = "bar0")]
4926            Bar0,
4927            #[serde(rename = "bar1")]
4928            Bar1,
4929        }
4930
4931        #[derive(Debug, PartialEq, Eq, Clone, serde::Deserialize, serde::Serialize)]
4932        pub struct Foo {
4933            #[serde(rename = "barInit")]
4934            pub bar_init: Bar,
4935            #[serde(rename = "barUse")]
4936            pub bar_use: Bar,
4937        }
4938
4939        let schema = Schema::parse_str(schema)?;
4940
4941        let foo = Foo {
4942            bar_init: Bar::Bar0,
4943            bar_use: Bar::Bar1,
4944        };
4945
4946        let avro_value = crate::to_value(foo)?;
4947        assert!(avro_value.validate(&schema));
4948
4949        let mut writer = crate::Writer::new(&schema, Vec::new());
4950
4951        // schema validation happens here
4952        writer.append(avro_value)?;
4953
4954        Ok(())
4955    }
4956
4957    #[test]
4958    fn avro_3755_deserialize() -> TestResult {
4959        #[derive(
4960            Debug, PartialEq, Eq, Hash, PartialOrd, Ord, Clone, serde::Deserialize, serde::Serialize,
4961        )]
4962        pub enum Bar {
4963            #[serde(rename = "bar0")]
4964            Bar0,
4965            #[serde(rename = "bar1")]
4966            Bar1,
4967            #[serde(rename = "bar2")]
4968            Bar2,
4969        }
4970
4971        #[derive(Debug, PartialEq, Eq, Clone, serde::Deserialize, serde::Serialize)]
4972        pub struct Foo {
4973            #[serde(rename = "barInit")]
4974            pub bar_init: Bar,
4975            #[serde(rename = "barUse")]
4976            pub bar_use: Bar,
4977        }
4978
4979        let writer_schema = r#"{
4980            "type": "record",
4981            "name": "Foo",
4982            "fields":
4983            [
4984                {
4985                    "name": "barInit",
4986                    "type":
4987                    {
4988                        "type": "enum",
4989                        "name": "Bar",
4990                        "symbols":
4991                        [
4992                            "bar0",
4993                            "bar1"
4994                        ]
4995                    }
4996                },
4997                {
4998                    "name": "barUse",
4999                    "type": "Bar"
5000                }
5001            ]
5002            }"#;
5003
5004        let reader_schema = r#"{
5005            "type": "record",
5006            "name": "Foo",
5007            "namespace": "name.space",
5008            "fields":
5009            [
5010                {
5011                    "name": "barInit",
5012                    "type":
5013                    {
5014                        "type": "enum",
5015                        "name": "Bar",
5016                        "symbols":
5017                        [
5018                            "bar0",
5019                            "bar1",
5020                            "bar2"
5021                        ]
5022                    }
5023                },
5024                {
5025                    "name": "barUse",
5026                    "type": "Bar"
5027                }
5028            ]
5029            }"#;
5030
5031        let writer_schema = Schema::parse_str(writer_schema)?;
5032        let foo = Foo {
5033            bar_init: Bar::Bar0,
5034            bar_use: Bar::Bar1,
5035        };
5036        let avro_value = crate::to_value(foo)?;
5037        assert!(
5038            avro_value.validate(&writer_schema),
5039            "value is valid for schema",
5040        );
5041        let datum = crate::to_avro_datum(&writer_schema, avro_value)?;
5042        let mut x = &datum[..];
5043        let reader_schema = Schema::parse_str(reader_schema)?;
5044        let deser_value = crate::from_avro_datum(&writer_schema, &mut x, Some(&reader_schema))?;
5045        match deser_value {
5046            types::Value::Record(fields) => {
5047                assert_eq!(fields.len(), 2);
5048                assert_eq!(fields[0].0, "barInit");
5049                assert_eq!(fields[0].1, types::Value::Enum(0, "bar0".to_string()));
5050                assert_eq!(fields[1].0, "barUse");
5051                assert_eq!(fields[1].1, types::Value::Enum(1, "bar1".to_string()));
5052            }
5053            _ => panic!("Expected Value::Record"),
5054        }
5055
5056        Ok(())
5057    }
5058
5059    #[test]
5060    fn test_avro_3780_decimal_schema_type_with_fixed() -> TestResult {
5061        let schema = json!(
5062        {
5063          "type": "record",
5064          "name": "recordWithDecimal",
5065          "fields": [
5066            {
5067                "name": "decimal",
5068                "type": "fixed",
5069                "name": "nestedFixed",
5070                "size": 8,
5071                "logicalType": "decimal",
5072                "precision": 4
5073            }
5074          ]
5075        });
5076
5077        let parse_result = Schema::parse(&schema);
5078        assert!(
5079            parse_result.is_ok(),
5080            "parse result must be ok, got: {:?}",
5081            parse_result
5082        );
5083
5084        Ok(())
5085    }
5086
5087    #[test]
5088    fn test_avro_3772_enum_default_wrong_type() -> TestResult {
5089        let schema = r#"
5090        {
5091          "type": "record",
5092          "name": "test",
5093          "fields": [
5094            {"name": "a", "type": "long", "default": 42},
5095            {"name": "b", "type": "string"},
5096            {
5097              "name": "c",
5098              "type": {
5099                "type": "enum",
5100                "name": "suit",
5101                "symbols": ["diamonds", "spades", "clubs", "hearts"],
5102                "default": 123
5103              }
5104            }
5105          ]
5106        }
5107        "#;
5108
5109        match Schema::parse_str(schema) {
5110            Err(err) => {
5111                assert_eq!(
5112                    err.to_string(),
5113                    "Default value for enum must be a string! Got: 123"
5114                );
5115            }
5116            _ => panic!("Expected an error"),
5117        }
5118        Ok(())
5119    }
5120
5121    #[test]
5122    fn test_avro_3812_handle_null_namespace_properly() -> TestResult {
5123        let schema_str = r#"
5124        {
5125          "namespace": "",
5126          "type": "record",
5127          "name": "my_schema",
5128          "fields": [
5129            {
5130              "name": "a",
5131              "type": {
5132                "type": "enum",
5133                "name": "my_enum",
5134                "namespace": "",
5135                "symbols": ["a", "b"]
5136              }
5137            },  {
5138              "name": "b",
5139              "type": {
5140                "type": "fixed",
5141                "name": "my_fixed",
5142                "namespace": "",
5143                "size": 10
5144              }
5145            }
5146          ]
5147         }
5148         "#;
5149
5150        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}}]}"#;
5151        let schema = Schema::parse_str(schema_str)?;
5152        let canonical_form = schema.canonical_form();
5153        assert_eq!(canonical_form, expected);
5154
5155        let name = Name::new("my_name")?;
5156        let fullname = name.fullname(Some("".to_string()));
5157        assert_eq!(fullname, "my_name");
5158        let qname = name.fully_qualified_name(&Some("".to_string())).to_string();
5159        assert_eq!(qname, "my_name");
5160
5161        Ok(())
5162    }
5163
5164    #[test]
5165    fn test_avro_3818_inherit_enclosing_namespace() -> TestResult {
5166        // Enclosing namespace is specified but inner namespaces are not.
5167        let schema_str = r#"
5168        {
5169          "namespace": "my_ns",
5170          "type": "record",
5171          "name": "my_schema",
5172          "fields": [
5173            {
5174              "name": "f1",
5175              "type": {
5176                "name": "enum1",
5177                "type": "enum",
5178                "symbols": ["a"]
5179              }
5180            },  {
5181              "name": "f2",
5182              "type": {
5183                "name": "fixed1",
5184                "type": "fixed",
5185                "size": 1
5186              }
5187            }
5188          ]
5189        }
5190        "#;
5191
5192        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}}]}"#;
5193        let schema = Schema::parse_str(schema_str)?;
5194        let canonical_form = schema.canonical_form();
5195        assert_eq!(canonical_form, expected);
5196
5197        // Enclosing namespace and inner namespaces are specified
5198        // but inner namespaces are ""
5199        let schema_str = r#"
5200        {
5201          "namespace": "my_ns",
5202          "type": "record",
5203          "name": "my_schema",
5204          "fields": [
5205            {
5206              "name": "f1",
5207              "type": {
5208                "name": "enum1",
5209                "type": "enum",
5210                "namespace": "",
5211                "symbols": ["a"]
5212              }
5213            },  {
5214              "name": "f2",
5215              "type": {
5216                "name": "fixed1",
5217                "type": "fixed",
5218                "namespace": "",
5219                "size": 1
5220              }
5221            }
5222          ]
5223        }
5224        "#;
5225
5226        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}}]}"#;
5227        let schema = Schema::parse_str(schema_str)?;
5228        let canonical_form = schema.canonical_form();
5229        assert_eq!(canonical_form, expected);
5230
5231        // Enclosing namespace is "" and inner non-empty namespaces are specified.
5232        let schema_str = r#"
5233        {
5234          "namespace": "",
5235          "type": "record",
5236          "name": "my_schema",
5237          "fields": [
5238            {
5239              "name": "f1",
5240              "type": {
5241                "name": "enum1",
5242                "type": "enum",
5243                "namespace": "f1.ns",
5244                "symbols": ["a"]
5245              }
5246            },  {
5247              "name": "f2",
5248              "type": {
5249                "name": "f2.ns.fixed1",
5250                "type": "fixed",
5251                "size": 1
5252              }
5253            }
5254          ]
5255        }
5256        "#;
5257
5258        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}}]}"#;
5259        let schema = Schema::parse_str(schema_str)?;
5260        let canonical_form = schema.canonical_form();
5261        assert_eq!(canonical_form, expected);
5262
5263        // Nested complex types with non-empty enclosing namespace.
5264        let schema_str = r#"
5265        {
5266          "type": "record",
5267          "name": "my_ns.my_schema",
5268          "fields": [
5269            {
5270              "name": "f1",
5271              "type": {
5272                "name": "inner_record1",
5273                "type": "record",
5274                "fields": [
5275                  {
5276                    "name": "f1_1",
5277                    "type": {
5278                      "name": "enum1",
5279                      "type": "enum",
5280                      "symbols": ["a"]
5281                    }
5282                  }
5283                ]
5284              }
5285            },  {
5286              "name": "f2",
5287                "type": {
5288                "name": "inner_record2",
5289                "type": "record",
5290                "namespace": "inner_ns",
5291                "fields": [
5292                  {
5293                    "name": "f2_1",
5294                    "type": {
5295                      "name": "enum2",
5296                      "type": "enum",
5297                      "symbols": ["a"]
5298                    }
5299                  }
5300                ]
5301              }
5302            }
5303          ]
5304        }
5305        "#;
5306
5307        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"]}}]}}]}"#;
5308        let schema = Schema::parse_str(schema_str)?;
5309        let canonical_form = schema.canonical_form();
5310        assert_eq!(canonical_form, expected);
5311
5312        Ok(())
5313    }
5314
5315    #[test]
5316    fn test_avro_3779_bigdecimal_schema() -> TestResult {
5317        let schema = json!(
5318            {
5319                "name": "decimal",
5320                "type": "bytes",
5321                "logicalType": "big-decimal"
5322            }
5323        );
5324
5325        let parse_result = Schema::parse(&schema);
5326        assert!(
5327            parse_result.is_ok(),
5328            "parse result must be ok, got: {:?}",
5329            parse_result
5330        );
5331        match parse_result? {
5332            Schema::BigDecimal => (),
5333            other => panic!("Expected Schema::BigDecimal but got: {other:?}"),
5334        }
5335
5336        Ok(())
5337    }
5338
5339    #[test]
5340    fn test_avro_3820_deny_invalid_field_names() -> TestResult {
5341        let schema_str = r#"
5342        {
5343          "name": "my_record",
5344          "type": "record",
5345          "fields": [
5346            {
5347              "name": "f1.x",
5348              "type": {
5349                "name": "my_enum",
5350                "type": "enum",
5351                "symbols": ["a"]
5352              }
5353            },  {
5354              "name": "f2",
5355              "type": {
5356                "name": "my_fixed",
5357                "type": "fixed",
5358                "size": 1
5359              }
5360            }
5361          ]
5362        }
5363        "#;
5364
5365        match Schema::parse_str(schema_str) {
5366            Err(Error::FieldName(x)) if x == "f1.x" => Ok(()),
5367            other => Err(format!("Expected Error::FieldName, got {other:?}").into()),
5368        }
5369    }
5370
5371    #[test]
5372    fn test_avro_3827_disallow_duplicate_field_names() -> TestResult {
5373        let schema_str = r#"
5374        {
5375          "name": "my_schema",
5376          "type": "record",
5377          "fields": [
5378            {
5379              "name": "f1",
5380              "type": {
5381                "name": "a",
5382                "type": "record",
5383                "fields": []
5384              }
5385            },  {
5386              "name": "f1",
5387              "type": {
5388                "name": "b",
5389                "type": "record",
5390                "fields": []
5391              }
5392            }
5393          ]
5394        }
5395        "#;
5396
5397        match Schema::parse_str(schema_str) {
5398            Err(Error::FieldNameDuplicate(_)) => (),
5399            other => {
5400                return Err(format!("Expected Error::FieldNameDuplicate, got {other:?}").into());
5401            }
5402        };
5403
5404        let schema_str = r#"
5405        {
5406          "name": "my_schema",
5407          "type": "record",
5408          "fields": [
5409            {
5410              "name": "f1",
5411              "type": {
5412                "name": "a",
5413                "type": "record",
5414                "fields": [
5415                  {
5416                    "name": "f1",
5417                    "type": {
5418                      "name": "b",
5419                      "type": "record",
5420                      "fields": []
5421                    }
5422                  }
5423                ]
5424              }
5425            }
5426          ]
5427        }
5428        "#;
5429
5430        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":[]}}]}}]}"#;
5431        let schema = Schema::parse_str(schema_str)?;
5432        let canonical_form = schema.canonical_form();
5433        assert_eq!(canonical_form, expected);
5434
5435        Ok(())
5436    }
5437
5438    #[test]
5439    fn test_avro_3830_null_namespace_in_fully_qualified_names() -> TestResult {
5440        // Check whether all the named types don't refer to the namespace field
5441        // if their name starts with a dot.
5442        let schema_str = r#"
5443        {
5444          "name": ".record1",
5445          "namespace": "ns1",
5446          "type": "record",
5447          "fields": [
5448            {
5449              "name": "f1",
5450              "type": {
5451                "name": ".enum1",
5452                "namespace": "ns2",
5453                "type": "enum",
5454                "symbols": ["a"]
5455              }
5456            },  {
5457              "name": "f2",
5458              "type": {
5459                "name": ".fxed1",
5460                "namespace": "ns3",
5461                "type": "fixed",
5462                "size": 1
5463              }
5464            }
5465          ]
5466        }
5467        "#;
5468
5469        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}}]}"#;
5470        let schema = Schema::parse_str(schema_str)?;
5471        let canonical_form = schema.canonical_form();
5472        assert_eq!(canonical_form, expected);
5473
5474        // Check whether inner types don't inherit ns1.
5475        let schema_str = r#"
5476        {
5477          "name": ".record1",
5478          "namespace": "ns1",
5479          "type": "record",
5480          "fields": [
5481            {
5482              "name": "f1",
5483              "type": {
5484                "name": "enum1",
5485                "type": "enum",
5486                "symbols": ["a"]
5487              }
5488            },  {
5489              "name": "f2",
5490              "type": {
5491                "name": "fxed1",
5492                "type": "fixed",
5493                "size": 1
5494              }
5495            }
5496          ]
5497        }
5498        "#;
5499
5500        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}}]}"#;
5501        let schema = Schema::parse_str(schema_str)?;
5502        let canonical_form = schema.canonical_form();
5503        assert_eq!(canonical_form, expected);
5504
5505        let name = Name::new(".my_name")?;
5506        let fullname = name.fullname(None);
5507        assert_eq!(fullname, "my_name");
5508        let qname = name.fully_qualified_name(&None).to_string();
5509        assert_eq!(qname, "my_name");
5510
5511        Ok(())
5512    }
5513
5514    #[test]
5515    fn test_avro_3814_schema_resolution_failure() -> TestResult {
5516        // Define a reader schema: a nested record with an optional field.
5517        let reader_schema = json!(
5518            {
5519                "type": "record",
5520                "name": "MyOuterRecord",
5521                "fields": [
5522                    {
5523                        "name": "inner_record",
5524                        "type": [
5525                            "null",
5526                            {
5527                                "type": "record",
5528                                "name": "MyRecord",
5529                                "fields": [
5530                                    {"name": "a", "type": "string"}
5531                                ]
5532                            }
5533                        ],
5534                        "default": null
5535                    }
5536                ]
5537            }
5538        );
5539
5540        // Define a writer schema: a nested record with an optional field, which
5541        // may optionally contain an enum.
5542        let writer_schema = json!(
5543            {
5544                "type": "record",
5545                "name": "MyOuterRecord",
5546                "fields": [
5547                    {
5548                        "name": "inner_record",
5549                        "type": [
5550                            "null",
5551                            {
5552                                "type": "record",
5553                                "name": "MyRecord",
5554                                "fields": [
5555                                    {"name": "a", "type": "string"},
5556                                    {
5557                                        "name": "b",
5558                                        "type": [
5559                                            "null",
5560                                            {
5561                                                "type": "enum",
5562                                                "name": "MyEnum",
5563                                                "symbols": ["A", "B", "C"],
5564                                                "default": "C"
5565                                            }
5566                                        ],
5567                                        "default": null
5568                                    },
5569                                ]
5570                            }
5571                        ]
5572                    }
5573                ],
5574                "default": null
5575            }
5576        );
5577
5578        // Use different structs to represent the "Reader" and the "Writer"
5579        // to mimic two different versions of a producer & consumer application.
5580        #[derive(Serialize, Deserialize, Debug)]
5581        struct MyInnerRecordReader {
5582            a: String,
5583        }
5584
5585        #[derive(Serialize, Deserialize, Debug)]
5586        struct MyRecordReader {
5587            inner_record: Option<MyInnerRecordReader>,
5588        }
5589
5590        #[derive(Serialize, Deserialize, Debug)]
5591        enum MyEnum {
5592            A,
5593            B,
5594            C,
5595        }
5596
5597        #[derive(Serialize, Deserialize, Debug)]
5598        struct MyInnerRecordWriter {
5599            a: String,
5600            b: Option<MyEnum>,
5601        }
5602
5603        #[derive(Serialize, Deserialize, Debug)]
5604        struct MyRecordWriter {
5605            inner_record: Option<MyInnerRecordWriter>,
5606        }
5607
5608        let s = MyRecordWriter {
5609            inner_record: Some(MyInnerRecordWriter {
5610                a: "foo".to_string(),
5611                b: None,
5612            }),
5613        };
5614
5615        // Serialize using the writer schema.
5616        let writer_schema = Schema::parse(&writer_schema)?;
5617        let avro_value = crate::to_value(s)?;
5618        assert!(
5619            avro_value.validate(&writer_schema),
5620            "value is valid for schema",
5621        );
5622        let datum = crate::to_avro_datum(&writer_schema, avro_value)?;
5623
5624        // Now, attempt to deserialize using the reader schema.
5625        let reader_schema = Schema::parse(&reader_schema)?;
5626        let mut x = &datum[..];
5627
5628        // Deserialization should succeed and we should be able to resolve the schema.
5629        let deser_value = crate::from_avro_datum(&writer_schema, &mut x, Some(&reader_schema))?;
5630        assert!(deser_value.validate(&reader_schema));
5631
5632        // Verify that we can read a field from the record.
5633        let d: MyRecordReader = crate::from_value(&deser_value)?;
5634        assert_eq!(d.inner_record.unwrap().a, "foo".to_string());
5635        Ok(())
5636    }
5637
5638    #[test]
5639    fn test_avro_3837_disallow_invalid_namespace() -> TestResult {
5640        // Valid namespace #1 (Single name portion)
5641        let schema_str = r#"
5642        {
5643          "name": "record1",
5644          "namespace": "ns1",
5645          "type": "record",
5646          "fields": []
5647        }
5648        "#;
5649
5650        let expected = r#"{"name":"ns1.record1","type":"record","fields":[]}"#;
5651        let schema = Schema::parse_str(schema_str)?;
5652        let canonical_form = schema.canonical_form();
5653        assert_eq!(canonical_form, expected);
5654
5655        // Valid namespace #2 (multiple name portions).
5656        let schema_str = r#"
5657        {
5658          "name": "enum1",
5659          "namespace": "ns1.foo.bar",
5660          "type": "enum",
5661          "symbols": ["a"]
5662        }
5663        "#;
5664
5665        let expected = r#"{"name":"ns1.foo.bar.enum1","type":"enum","symbols":["a"]}"#;
5666        let schema = Schema::parse_str(schema_str)?;
5667        let canonical_form = schema.canonical_form();
5668        assert_eq!(canonical_form, expected);
5669
5670        // Invalid namespace #1 (a name portion starts with dot)
5671        let schema_str = r#"
5672        {
5673          "name": "fixed1",
5674          "namespace": ".ns1.a.b",
5675          "type": "fixed",
5676          "size": 1
5677        }
5678        "#;
5679
5680        match Schema::parse_str(schema_str) {
5681            Err(Error::InvalidNamespace(_, _)) => (),
5682            other => return Err(format!("Expected Error::InvalidNamespace, got {other:?}").into()),
5683        };
5684
5685        // Invalid namespace #2 (invalid character in a name portion)
5686        let schema_str = r#"
5687        {
5688          "name": "record1",
5689          "namespace": "ns1.a*b.c",
5690          "type": "record",
5691          "fields": []
5692        }
5693        "#;
5694
5695        match Schema::parse_str(schema_str) {
5696            Err(Error::InvalidNamespace(_, _)) => (),
5697            other => return Err(format!("Expected Error::InvalidNamespace, got {other:?}").into()),
5698        };
5699
5700        // Invalid namespace #3 (a name portion starts with a digit)
5701        let schema_str = r#"
5702        {
5703          "name": "fixed1",
5704          "namespace": "ns1.1a.b",
5705          "type": "fixed",
5706          "size": 1
5707        }
5708        "#;
5709
5710        match Schema::parse_str(schema_str) {
5711            Err(Error::InvalidNamespace(_, _)) => (),
5712            other => return Err(format!("Expected Error::InvalidNamespace, got {other:?}").into()),
5713        };
5714
5715        // Invalid namespace #4 (a name portion is missing - two dots in a row)
5716        let schema_str = r#"
5717        {
5718          "name": "fixed1",
5719          "namespace": "ns1..a",
5720          "type": "fixed",
5721          "size": 1
5722        }
5723        "#;
5724
5725        match Schema::parse_str(schema_str) {
5726            Err(Error::InvalidNamespace(_, _)) => (),
5727            other => return Err(format!("Expected Error::InvalidNamespace, got {other:?}").into()),
5728        };
5729
5730        // Invalid namespace #5 (a name portion is missing - ends with a dot)
5731        let schema_str = r#"
5732        {
5733          "name": "fixed1",
5734          "namespace": "ns1.a.",
5735          "type": "fixed",
5736          "size": 1
5737        }
5738        "#;
5739
5740        match Schema::parse_str(schema_str) {
5741            Err(Error::InvalidNamespace(_, _)) => (),
5742            other => return Err(format!("Expected Error::InvalidNamespace, got {other:?}").into()),
5743        };
5744
5745        Ok(())
5746    }
5747
5748    #[test]
5749    fn test_avro_3851_validate_default_value_of_simple_record_field() -> TestResult {
5750        let schema_str = r#"
5751        {
5752            "name": "record1",
5753            "namespace": "ns",
5754            "type": "record",
5755            "fields": [
5756                {
5757                    "name": "f1",
5758                    "type": "int",
5759                    "default": "invalid"
5760                }
5761            ]
5762        }
5763        "#;
5764        let expected = Error::GetDefaultRecordField(
5765            "f1".to_string(),
5766            "ns.record1".to_string(),
5767            r#""int""#.to_string(),
5768        )
5769        .to_string();
5770        let result = Schema::parse_str(schema_str);
5771        assert!(result.is_err());
5772        let err = result
5773            .map_err(|e| e.to_string())
5774            .err()
5775            .unwrap_or_else(|| "unexpected".to_string());
5776        assert_eq!(expected, err);
5777
5778        Ok(())
5779    }
5780
5781    #[test]
5782    fn test_avro_3851_validate_default_value_of_nested_record_field() -> TestResult {
5783        let schema_str = r#"
5784        {
5785            "name": "record1",
5786            "namespace": "ns",
5787            "type": "record",
5788            "fields": [
5789                {
5790                    "name": "f1",
5791                    "type": {
5792                        "name": "record2",
5793                        "type": "record",
5794                        "fields": [
5795                            {
5796                                "name": "f1_1",
5797                                "type": "int"
5798                            }
5799                        ]
5800                    },
5801                    "default": "invalid"
5802                }
5803            ]
5804        }
5805        "#;
5806        let expected = Error::GetDefaultRecordField(
5807            "f1".to_string(),
5808            "ns.record1".to_string(),
5809            r#"{"name":"ns.record2","type":"record","fields":[{"name":"f1_1","type":"int"}]}"#
5810                .to_string(),
5811        )
5812        .to_string();
5813        let result = Schema::parse_str(schema_str);
5814        assert!(result.is_err());
5815        let err = result
5816            .map_err(|e| e.to_string())
5817            .err()
5818            .unwrap_or_else(|| "unexpected".to_string());
5819        assert_eq!(expected, err);
5820
5821        Ok(())
5822    }
5823
5824    #[test]
5825    fn test_avro_3851_validate_default_value_of_enum_record_field() -> TestResult {
5826        let schema_str = r#"
5827        {
5828            "name": "record1",
5829            "namespace": "ns",
5830            "type": "record",
5831            "fields": [
5832                {
5833                    "name": "f1",
5834                    "type": {
5835                        "name": "enum1",
5836                        "type": "enum",
5837                        "symbols": ["a", "b", "c"]
5838                    },
5839                    "default": "invalid"
5840                }
5841            ]
5842        }
5843        "#;
5844        let expected = Error::GetDefaultRecordField(
5845            "f1".to_string(),
5846            "ns.record1".to_string(),
5847            r#"{"name":"ns.enum1","type":"enum","symbols":["a","b","c"]}"#.to_string(),
5848        )
5849        .to_string();
5850        let result = Schema::parse_str(schema_str);
5851        assert!(result.is_err());
5852        let err = result
5853            .map_err(|e| e.to_string())
5854            .err()
5855            .unwrap_or_else(|| "unexpected".to_string());
5856        assert_eq!(expected, err);
5857
5858        Ok(())
5859    }
5860
5861    #[test]
5862    fn test_avro_3851_validate_default_value_of_fixed_record_field() -> TestResult {
5863        let schema_str = r#"
5864        {
5865            "name": "record1",
5866            "namespace": "ns",
5867            "type": "record",
5868            "fields": [
5869                {
5870                    "name": "f1",
5871                    "type": {
5872                        "name": "fixed1",
5873                        "type": "fixed",
5874                        "size": 3
5875                    },
5876                    "default": 100
5877                }
5878            ]
5879        }
5880        "#;
5881        let expected = Error::GetDefaultRecordField(
5882            "f1".to_string(),
5883            "ns.record1".to_string(),
5884            r#"{"name":"ns.fixed1","type":"fixed","size":3}"#.to_string(),
5885        )
5886        .to_string();
5887        let result = Schema::parse_str(schema_str);
5888        assert!(result.is_err());
5889        let err = result
5890            .map_err(|e| e.to_string())
5891            .err()
5892            .unwrap_or_else(|| "unexpected".to_string());
5893        assert_eq!(expected, err);
5894
5895        Ok(())
5896    }
5897
5898    #[test]
5899    fn test_avro_3851_validate_default_value_of_array_record_field() -> TestResult {
5900        let schema_str = r#"
5901        {
5902            "name": "record1",
5903            "namespace": "ns",
5904            "type": "record",
5905            "fields": [
5906                {
5907                    "name": "f1",
5908                    "type": "array",
5909                    "items": "int",
5910                    "default": "invalid"
5911                }
5912            ]
5913        }
5914        "#;
5915        let expected = Error::GetDefaultRecordField(
5916            "f1".to_string(),
5917            "ns.record1".to_string(),
5918            r#"{"type":"array","items":"int"}"#.to_string(),
5919        )
5920        .to_string();
5921        let result = Schema::parse_str(schema_str);
5922        assert!(result.is_err());
5923        let err = result
5924            .map_err(|e| e.to_string())
5925            .err()
5926            .unwrap_or_else(|| "unexpected".to_string());
5927        assert_eq!(expected, err);
5928
5929        Ok(())
5930    }
5931
5932    #[test]
5933    fn test_avro_3851_validate_default_value_of_map_record_field() -> TestResult {
5934        let schema_str = r#"
5935        {
5936            "name": "record1",
5937            "namespace": "ns",
5938            "type": "record",
5939            "fields": [
5940                {
5941                    "name": "f1",
5942                    "type": "map",
5943                    "values": "string",
5944                    "default": "invalid"
5945                }
5946            ]
5947        }
5948        "#;
5949        let expected = Error::GetDefaultRecordField(
5950            "f1".to_string(),
5951            "ns.record1".to_string(),
5952            r#"{"type":"map","values":"string"}"#.to_string(),
5953        )
5954        .to_string();
5955        let result = Schema::parse_str(schema_str);
5956        assert!(result.is_err());
5957        let err = result
5958            .map_err(|e| e.to_string())
5959            .err()
5960            .unwrap_or_else(|| "unexpected".to_string());
5961        assert_eq!(expected, err);
5962
5963        Ok(())
5964    }
5965
5966    #[test]
5967    fn test_avro_3851_validate_default_value_of_ref_record_field() -> TestResult {
5968        let schema_str = r#"
5969        {
5970            "name": "record1",
5971            "namespace": "ns",
5972            "type": "record",
5973            "fields": [
5974                {
5975                    "name": "f1",
5976                    "type": {
5977                        "name": "record2",
5978                        "type": "record",
5979                        "fields": [
5980                            {
5981                                "name": "f1_1",
5982                                "type": "int"
5983                            }
5984                        ]
5985                    }
5986                },  {
5987                    "name": "f2",
5988                    "type": "ns.record2",
5989                    "default": { "f1_1": true }
5990                }
5991            ]
5992        }
5993        "#;
5994        let expected = Error::GetDefaultRecordField(
5995            "f2".to_string(),
5996            "ns.record1".to_string(),
5997            r#""ns.record2""#.to_string(),
5998        )
5999        .to_string();
6000        let result = Schema::parse_str(schema_str);
6001        assert!(result.is_err());
6002        let err = result
6003            .map_err(|e| e.to_string())
6004            .err()
6005            .unwrap_or_else(|| "unexpected".to_string());
6006        assert_eq!(expected, err);
6007
6008        Ok(())
6009    }
6010
6011    #[test]
6012    fn test_avro_3851_validate_default_value_of_enum() -> TestResult {
6013        let schema_str = r#"
6014        {
6015            "name": "enum1",
6016            "namespace": "ns",
6017            "type": "enum",
6018            "symbols": ["a", "b", "c"],
6019            "default": 100
6020        }
6021        "#;
6022        let expected = Error::EnumDefaultWrongType(100.into()).to_string();
6023        let result = Schema::parse_str(schema_str);
6024        assert!(result.is_err());
6025        let err = result
6026            .map_err(|e| e.to_string())
6027            .err()
6028            .unwrap_or_else(|| "unexpected".to_string());
6029        assert_eq!(expected, err);
6030
6031        let schema_str = r#"
6032        {
6033            "name": "enum1",
6034            "namespace": "ns",
6035            "type": "enum",
6036            "symbols": ["a", "b", "c"],
6037            "default": "d"
6038        }
6039        "#;
6040        let expected = Error::GetEnumDefault {
6041            symbol: "d".to_string(),
6042            symbols: vec!["a".to_string(), "b".to_string(), "c".to_string()],
6043        }
6044        .to_string();
6045        let result = Schema::parse_str(schema_str);
6046        assert!(result.is_err());
6047        let err = result
6048            .map_err(|e| e.to_string())
6049            .err()
6050            .unwrap_or_else(|| "unexpected".to_string());
6051        assert_eq!(expected, err);
6052
6053        Ok(())
6054    }
6055
6056    #[test]
6057    fn test_avro_3862_get_aliases() -> TestResult {
6058        // Test for Record
6059        let schema_str = r#"
6060        {
6061            "name": "record1",
6062            "namespace": "ns1",
6063            "type": "record",
6064            "aliases": ["r1", "ns2.r2"],
6065            "fields": [
6066                { "name": "f1", "type": "int" },
6067                { "name": "f2", "type": "string" }
6068            ]
6069        }
6070        "#;
6071        let schema = Schema::parse_str(schema_str)?;
6072        let expected = vec![Alias::new("ns1.r1")?, Alias::new("ns2.r2")?];
6073        match schema.aliases() {
6074            Some(aliases) => assert_eq!(aliases, &expected),
6075            None => panic!("Expected Some({:?}), got None", expected),
6076        }
6077
6078        let schema_str = r#"
6079        {
6080            "name": "record1",
6081            "namespace": "ns1",
6082            "type": "record",
6083            "fields": [
6084                { "name": "f1", "type": "int" },
6085                { "name": "f2", "type": "string" }
6086            ]
6087        }
6088        "#;
6089        let schema = Schema::parse_str(schema_str)?;
6090        match schema.aliases() {
6091            None => (),
6092            some => panic!("Expected None, got {some:?}"),
6093        }
6094
6095        // Test for Enum
6096        let schema_str = r#"
6097        {
6098            "name": "enum1",
6099            "namespace": "ns1",
6100            "type": "enum",
6101            "aliases": ["en1", "ns2.en2"],
6102            "symbols": ["a", "b", "c"]
6103        }
6104        "#;
6105        let schema = Schema::parse_str(schema_str)?;
6106        let expected = vec![Alias::new("ns1.en1")?, Alias::new("ns2.en2")?];
6107        match schema.aliases() {
6108            Some(aliases) => assert_eq!(aliases, &expected),
6109            None => panic!("Expected Some({:?}), got None", expected),
6110        }
6111
6112        let schema_str = r#"
6113        {
6114            "name": "enum1",
6115            "namespace": "ns1",
6116            "type": "enum",
6117            "symbols": ["a", "b", "c"]
6118        }
6119        "#;
6120        let schema = Schema::parse_str(schema_str)?;
6121        match schema.aliases() {
6122            None => (),
6123            some => panic!("Expected None, got {some:?}"),
6124        }
6125
6126        // Test for Fixed
6127        let schema_str = r#"
6128        {
6129            "name": "fixed1",
6130            "namespace": "ns1",
6131            "type": "fixed",
6132            "aliases": ["fx1", "ns2.fx2"],
6133            "size": 10
6134        }
6135        "#;
6136        let schema = Schema::parse_str(schema_str)?;
6137        let expected = vec![Alias::new("ns1.fx1")?, Alias::new("ns2.fx2")?];
6138        match schema.aliases() {
6139            Some(aliases) => assert_eq!(aliases, &expected),
6140            None => panic!("Expected Some({:?}), got None", expected),
6141        }
6142
6143        let schema_str = r#"
6144        {
6145            "name": "fixed1",
6146            "namespace": "ns1",
6147            "type": "fixed",
6148            "size": 10
6149        }
6150        "#;
6151        let schema = Schema::parse_str(schema_str)?;
6152        match schema.aliases() {
6153            None => (),
6154            some => panic!("Expected None, got {some:?}"),
6155        }
6156
6157        // Test for non-named type
6158        let schema = Schema::Int;
6159        match schema.aliases() {
6160            None => (),
6161            some => panic!("Expected None, got {some:?}"),
6162        }
6163
6164        Ok(())
6165    }
6166
6167    #[test]
6168    fn test_avro_3862_get_doc() -> TestResult {
6169        // Test for Record
6170        let schema_str = r#"
6171        {
6172            "name": "record1",
6173            "type": "record",
6174            "doc": "Record Document",
6175            "fields": [
6176                { "name": "f1", "type": "int" },
6177                { "name": "f2", "type": "string" }
6178            ]
6179        }
6180        "#;
6181        let schema = Schema::parse_str(schema_str)?;
6182        let expected = "Record Document";
6183        match schema.doc() {
6184            Some(doc) => assert_eq!(doc, expected),
6185            None => panic!("Expected Some({:?}), got None", expected),
6186        }
6187
6188        let schema_str = r#"
6189        {
6190            "name": "record1",
6191            "type": "record",
6192            "fields": [
6193                { "name": "f1", "type": "int" },
6194                { "name": "f2", "type": "string" }
6195            ]
6196        }
6197        "#;
6198        let schema = Schema::parse_str(schema_str)?;
6199        match schema.doc() {
6200            None => (),
6201            some => panic!("Expected None, got {some:?}"),
6202        }
6203
6204        // Test for Enum
6205        let schema_str = r#"
6206        {
6207            "name": "enum1",
6208            "type": "enum",
6209            "doc": "Enum Document",
6210            "symbols": ["a", "b", "c"]
6211        }
6212        "#;
6213        let schema = Schema::parse_str(schema_str)?;
6214        let expected = "Enum Document";
6215        match schema.doc() {
6216            Some(doc) => assert_eq!(doc, expected),
6217            None => panic!("Expected Some({:?}), got None", expected),
6218        }
6219
6220        let schema_str = r#"
6221        {
6222            "name": "enum1",
6223            "type": "enum",
6224            "symbols": ["a", "b", "c"]
6225        }
6226        "#;
6227        let schema = Schema::parse_str(schema_str)?;
6228        match schema.doc() {
6229            None => (),
6230            some => panic!("Expected None, got {some:?}"),
6231        }
6232
6233        // Test for Fixed
6234        let schema_str = r#"
6235        {
6236            "name": "fixed1",
6237            "type": "fixed",
6238            "doc": "Fixed Document",
6239            "size": 10
6240        }
6241        "#;
6242        let schema = Schema::parse_str(schema_str)?;
6243        let expected = "Fixed Document";
6244        match schema.doc() {
6245            Some(doc) => assert_eq!(doc, expected),
6246            None => panic!("Expected Some({:?}), got None", expected),
6247        }
6248
6249        let schema_str = r#"
6250        {
6251            "name": "fixed1",
6252            "type": "fixed",
6253            "size": 10
6254        }
6255        "#;
6256        let schema = Schema::parse_str(schema_str)?;
6257        match schema.doc() {
6258            None => (),
6259            some => panic!("Expected None, got {some:?}"),
6260        }
6261
6262        // Test for non-named type
6263        let schema = Schema::Int;
6264        match schema.doc() {
6265            None => (),
6266            some => panic!("Expected None, got {some:?}"),
6267        }
6268
6269        Ok(())
6270    }
6271
6272    #[test]
6273    fn avro_3886_serialize_attributes() -> TestResult {
6274        let attributes = BTreeMap::from([
6275            ("string_key".into(), "value".into()),
6276            ("number_key".into(), 1.23.into()),
6277            ("null_key".into(), Value::Null),
6278            (
6279                "array_key".into(),
6280                Value::Array(vec![1.into(), 2.into(), 3.into()]),
6281            ),
6282            ("object_key".into(), Value::Object(Map::default())),
6283        ]);
6284
6285        // Test serialize enum attributes
6286        let schema = Schema::Enum(EnumSchema {
6287            name: Name::new("a")?,
6288            aliases: None,
6289            doc: None,
6290            symbols: vec![],
6291            default: None,
6292            attributes: attributes.clone(),
6293        });
6294        let serialized = serde_json::to_string(&schema)?;
6295        assert_eq!(
6296            r#"{"type":"enum","name":"a","symbols":[],"array_key":[1,2,3],"null_key":null,"number_key":1.23,"object_key":{},"string_key":"value"}"#,
6297            &serialized
6298        );
6299
6300        // Test serialize fixed custom_attributes
6301        let schema = Schema::Fixed(FixedSchema {
6302            name: Name::new("a")?,
6303            aliases: None,
6304            doc: None,
6305            size: 1,
6306            default: None,
6307            attributes: attributes.clone(),
6308        });
6309        let serialized = serde_json::to_string(&schema)?;
6310        assert_eq!(
6311            r#"{"type":"fixed","name":"a","size":1,"array_key":[1,2,3],"null_key":null,"number_key":1.23,"object_key":{},"string_key":"value"}"#,
6312            &serialized
6313        );
6314
6315        // Test serialize record custom_attributes
6316        let schema = Schema::Record(RecordSchema {
6317            name: Name::new("a")?,
6318            aliases: None,
6319            doc: None,
6320            fields: vec![],
6321            lookup: BTreeMap::new(),
6322            attributes,
6323        });
6324        let serialized = serde_json::to_string(&schema)?;
6325        assert_eq!(
6326            r#"{"type":"record","name":"a","fields":[],"array_key":[1,2,3],"null_key":null,"number_key":1.23,"object_key":{},"string_key":"value"}"#,
6327            &serialized
6328        );
6329
6330        Ok(())
6331    }
6332
6333    /// A test cases showing that names and namespaces can be constructed
6334    /// entirely by underscores.
6335    #[test]
6336    fn test_avro_3897_funny_valid_names_and_namespaces() -> TestResult {
6337        for funny_name in ["_", "_._", "__._", "_.__", "_._._"] {
6338            let name = Name::new(funny_name);
6339            assert!(name.is_ok());
6340        }
6341        Ok(())
6342    }
6343
6344    #[test]
6345    fn test_avro_3896_decimal_schema() -> TestResult {
6346        // bytes decimal, represented as native logical type.
6347        let schema = json!(
6348        {
6349          "type": "bytes",
6350          "name": "BytesDecimal",
6351          "logicalType": "decimal",
6352          "size": 38,
6353          "precision": 9,
6354          "scale": 2
6355        });
6356        let parse_result = Schema::parse(&schema)?;
6357        assert!(matches!(
6358            parse_result,
6359            Schema::Decimal(DecimalSchema {
6360                precision: 9,
6361                scale: 2,
6362                ..
6363            })
6364        ));
6365
6366        // long decimal, represents as native complex type.
6367        let schema = json!(
6368        {
6369          "type": "long",
6370          "name": "LongDecimal",
6371          "logicalType": "decimal"
6372        });
6373        let parse_result = Schema::parse(&schema)?;
6374        // assert!(matches!(parse_result, Schema::Long));
6375        assert_eq!(parse_result, Schema::Long);
6376
6377        Ok(())
6378    }
6379
6380    #[test]
6381    fn avro_3896_uuid_schema_for_string() -> TestResult {
6382        // string uuid, represents as native logical type.
6383        let schema = json!(
6384        {
6385          "type": "string",
6386          "name": "StringUUID",
6387          "logicalType": "uuid"
6388        });
6389        let parse_result = Schema::parse(&schema)?;
6390        assert_eq!(parse_result, Schema::Uuid);
6391
6392        Ok(())
6393    }
6394
6395    #[test]
6396    fn avro_3926_uuid_schema_for_fixed_with_size_16() -> TestResult {
6397        let schema = json!(
6398        {
6399            "type": "fixed",
6400            "name": "FixedUUID",
6401            "size": 16,
6402            "logicalType": "uuid"
6403        });
6404        let parse_result = Schema::parse(&schema)?;
6405        assert_eq!(parse_result, Schema::Uuid);
6406        assert_not_logged(
6407            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")} })"#,
6408        );
6409
6410        Ok(())
6411    }
6412
6413    #[test]
6414    fn avro_3926_uuid_schema_for_fixed_with_size_different_than_16() -> TestResult {
6415        let schema = json!(
6416        {
6417            "type": "fixed",
6418            "name": "FixedUUID",
6419            "size": 6,
6420            "logicalType": "uuid"
6421        });
6422        let parse_result = Schema::parse(&schema)?;
6423
6424        assert_eq!(
6425            parse_result,
6426            Schema::Fixed(FixedSchema {
6427                name: Name::new("FixedUUID")?,
6428                aliases: None,
6429                doc: None,
6430                size: 6,
6431                default: None,
6432                attributes: BTreeMap::from([("logicalType".to_string(), "uuid".into())]),
6433            })
6434        );
6435        assert_logged(
6436            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, default: None, attributes: {"logicalType": String("uuid")} })"#,
6437        );
6438
6439        Ok(())
6440    }
6441
6442    #[test]
6443    fn test_avro_3896_timestamp_millis_schema() -> TestResult {
6444        // long timestamp-millis, represents as native logical type.
6445        let schema = json!(
6446        {
6447          "type": "long",
6448          "name": "LongTimestampMillis",
6449          "logicalType": "timestamp-millis"
6450        });
6451        let parse_result = Schema::parse(&schema)?;
6452        assert_eq!(parse_result, Schema::TimestampMillis);
6453
6454        // int timestamp-millis, represents as native complex type.
6455        let schema = json!(
6456        {
6457            "type": "int",
6458            "name": "IntTimestampMillis",
6459            "logicalType": "timestamp-millis"
6460        });
6461        let parse_result = Schema::parse(&schema)?;
6462        assert_eq!(parse_result, Schema::Int);
6463
6464        Ok(())
6465    }
6466
6467    #[test]
6468    fn test_avro_3896_custom_bytes_schema() -> TestResult {
6469        // log type, represents as complex type.
6470        let schema = json!(
6471        {
6472            "type": "bytes",
6473            "name": "BytesLog",
6474            "logicalType": "custom"
6475        });
6476        let parse_result = Schema::parse(&schema)?;
6477        assert_eq!(parse_result, Schema::Bytes);
6478        assert_eq!(parse_result.custom_attributes(), None);
6479
6480        Ok(())
6481    }
6482
6483    #[test]
6484    fn test_avro_3899_parse_decimal_type() -> TestResult {
6485        let schema = Schema::parse_str(
6486            r#"{
6487             "name": "InvalidDecimal",
6488             "type": "fixed",
6489             "size": 16,
6490             "logicalType": "decimal",
6491             "precision": 2,
6492             "scale": 3
6493         }"#,
6494        )?;
6495        match schema {
6496            Schema::Fixed(fixed_schema) => {
6497                let attrs = fixed_schema.attributes;
6498                let precision = attrs
6499                    .get("precision")
6500                    .expect("The 'precision' attribute is missing");
6501                let scale = attrs
6502                    .get("scale")
6503                    .expect("The 'scale' attribute is missing");
6504                assert_logged(&format!("Ignoring invalid decimal logical type: The decimal precision ({}) must be bigger or equal to the scale ({})", precision, scale));
6505            }
6506            _ => unreachable!("Expected Schema::Fixed, got {:?}", schema),
6507        }
6508
6509        let schema = Schema::parse_str(
6510            r#"{
6511            "name": "ValidDecimal",
6512             "type": "bytes",
6513             "logicalType": "decimal",
6514             "precision": 3,
6515             "scale": 2
6516         }"#,
6517        )?;
6518        match schema {
6519            Schema::Decimal(_) => {
6520                assert_not_logged("Ignoring invalid decimal logical type: The decimal precision (2) must be bigger or equal to the scale (3)");
6521            }
6522            _ => unreachable!("Expected Schema::Decimal, got {:?}", schema),
6523        }
6524
6525        Ok(())
6526    }
6527
6528    #[test]
6529    fn avro_3920_serialize_record_with_custom_attributes() -> TestResult {
6530        let expected = {
6531            let mut lookup = BTreeMap::new();
6532            lookup.insert("value".to_owned(), 0);
6533            Schema::Record(RecordSchema {
6534                name: Name {
6535                    name: "LongList".to_owned(),
6536                    namespace: None,
6537                },
6538                aliases: Some(vec![Alias::new("LinkedLongs").unwrap()]),
6539                doc: None,
6540                fields: vec![RecordField {
6541                    name: "value".to_string(),
6542                    doc: None,
6543                    default: None,
6544                    aliases: None,
6545                    schema: Schema::Long,
6546                    order: RecordFieldOrder::Ascending,
6547                    position: 0,
6548                    custom_attributes: BTreeMap::from([("field-id".to_string(), 1.into())]),
6549                }],
6550                lookup,
6551                attributes: BTreeMap::from([("custom-attribute".to_string(), "value".into())]),
6552            })
6553        };
6554
6555        let value = serde_json::to_value(&expected)?;
6556        let serialized = serde_json::to_string(&value)?;
6557        assert_eq!(
6558            r#"{"aliases":["LinkedLongs"],"custom-attribute":"value","fields":[{"field-id":1,"name":"value","type":"long"}],"name":"LongList","type":"record"}"#,
6559            &serialized
6560        );
6561        assert_eq!(expected, Schema::parse_str(&serialized)?);
6562
6563        Ok(())
6564    }
6565
6566    #[test]
6567    fn test_avro_3925_serialize_decimal_inner_fixed() -> TestResult {
6568        let schema = Schema::Decimal(DecimalSchema {
6569            precision: 36,
6570            scale: 10,
6571            inner: Box::new(Schema::Fixed(FixedSchema {
6572                name: Name::new("decimal_36_10").unwrap(),
6573                aliases: None,
6574                doc: None,
6575                size: 16,
6576                default: None,
6577                attributes: Default::default(),
6578            })),
6579        });
6580
6581        let serialized_json = serde_json::to_string_pretty(&schema)?;
6582
6583        let expected_json = r#"{
6584  "type": "fixed",
6585  "name": "decimal_36_10",
6586  "size": 16,
6587  "logicalType": "decimal",
6588  "scale": 10,
6589  "precision": 36
6590}"#;
6591
6592        assert_eq!(serialized_json, expected_json);
6593
6594        Ok(())
6595    }
6596
6597    #[test]
6598    fn test_avro_3925_serialize_decimal_inner_bytes() -> TestResult {
6599        let schema = Schema::Decimal(DecimalSchema {
6600            precision: 36,
6601            scale: 10,
6602            inner: Box::new(Schema::Bytes),
6603        });
6604
6605        let serialized_json = serde_json::to_string_pretty(&schema)?;
6606
6607        let expected_json = r#"{
6608  "type": "bytes",
6609  "logicalType": "decimal",
6610  "scale": 10,
6611  "precision": 36
6612}"#;
6613
6614        assert_eq!(serialized_json, expected_json);
6615
6616        Ok(())
6617    }
6618
6619    #[test]
6620    fn test_avro_3925_serialize_decimal_inner_invalid() -> TestResult {
6621        let schema = Schema::Decimal(DecimalSchema {
6622            precision: 36,
6623            scale: 10,
6624            inner: Box::new(Schema::String),
6625        });
6626
6627        let serialized_json = serde_json::to_string_pretty(&schema);
6628
6629        assert!(serialized_json.is_err());
6630
6631        Ok(())
6632    }
6633
6634    #[test]
6635    fn test_avro_3927_serialize_array_with_custom_attributes() -> TestResult {
6636        let expected = Schema::array_with_attributes(
6637            Schema::Long,
6638            BTreeMap::from([("field-id".to_string(), "1".into())]),
6639        );
6640
6641        let value = serde_json::to_value(&expected)?;
6642        let serialized = serde_json::to_string(&value)?;
6643        assert_eq!(
6644            r#"{"field-id":"1","items":"long","type":"array"}"#,
6645            &serialized
6646        );
6647        let actual_schema = Schema::parse_str(&serialized)?;
6648        assert_eq!(expected, actual_schema);
6649        assert_eq!(
6650            expected.custom_attributes(),
6651            actual_schema.custom_attributes()
6652        );
6653
6654        Ok(())
6655    }
6656
6657    #[test]
6658    fn test_avro_3927_serialize_map_with_custom_attributes() -> TestResult {
6659        let expected = Schema::map_with_attributes(
6660            Schema::Long,
6661            BTreeMap::from([("field-id".to_string(), "1".into())]),
6662        );
6663
6664        let value = serde_json::to_value(&expected)?;
6665        let serialized = serde_json::to_string(&value)?;
6666        assert_eq!(
6667            r#"{"field-id":"1","type":"map","values":"long"}"#,
6668            &serialized
6669        );
6670        let actual_schema = Schema::parse_str(&serialized)?;
6671        assert_eq!(expected, actual_schema);
6672        assert_eq!(
6673            expected.custom_attributes(),
6674            actual_schema.custom_attributes()
6675        );
6676
6677        Ok(())
6678    }
6679
6680    #[test]
6681    fn avro_3928_parse_int_based_schema_with_default() -> TestResult {
6682        let schema = r#"
6683        {
6684          "type": "record",
6685          "name": "DateLogicalType",
6686          "fields": [ {
6687            "name": "birthday",
6688            "type": {"type": "int", "logicalType": "date"},
6689            "default": 1681601653
6690          } ]
6691        }"#;
6692
6693        match Schema::parse_str(schema)? {
6694            Schema::Record(record_schema) => {
6695                assert_eq!(record_schema.fields.len(), 1);
6696                let field = record_schema.fields.first().unwrap();
6697                assert_eq!(field.name, "birthday");
6698                assert_eq!(field.schema, Schema::Date);
6699                assert_eq!(
6700                    types::Value::from(field.default.clone().unwrap()),
6701                    types::Value::Int(1681601653)
6702                );
6703            }
6704            _ => unreachable!("Expected Schema::Record"),
6705        }
6706
6707        Ok(())
6708    }
6709
6710    #[test]
6711    fn avro_3946_union_with_single_type() -> TestResult {
6712        let schema = r#"
6713        {
6714          "type": "record",
6715          "name": "Issue",
6716          "namespace": "invalid.example",
6717          "fields": [
6718            {
6719              "name": "myField",
6720              "type": ["long"]
6721            }
6722          ]
6723        }"#;
6724
6725        let _ = Schema::parse_str(schema)?;
6726
6727        assert_logged(
6728            "Union schema with just one member! Consider dropping the union! \
6729                    Please enable debug logging to find out which Record schema \
6730                    declares the union with 'RUST_LOG=apache_avro::schema=debug'.",
6731        );
6732
6733        Ok(())
6734    }
6735
6736    #[test]
6737    fn avro_3946_union_without_any_types() -> TestResult {
6738        let schema = r#"
6739        {
6740          "type": "record",
6741          "name": "Issue",
6742          "namespace": "invalid.example",
6743          "fields": [
6744            {
6745              "name": "myField",
6746              "type": []
6747            }
6748          ]
6749        }"#;
6750
6751        let _ = Schema::parse_str(schema)?;
6752
6753        assert_logged(
6754            "Union schemas should have at least two members! \
6755                    Please enable debug logging to find out which Record schema \
6756                    declares the union with 'RUST_LOG=apache_avro::schema=debug'.",
6757        );
6758
6759        Ok(())
6760    }
6761
6762    #[test]
6763    fn avro_3965_fixed_schema_with_default_bigger_than_size() -> TestResult {
6764        match Schema::parse_str(
6765            r#"{
6766                "type": "fixed",
6767                "name": "test",
6768                "size": 1,
6769                "default": "123456789"
6770               }"#,
6771        ) {
6772            Ok(_schema) => panic!("Must fail!"),
6773            Err(err) => {
6774                assert_eq!(
6775                    err.to_string(),
6776                    "Fixed schema's default value length (9) does not match its size (1)"
6777                );
6778            }
6779        }
6780
6781        Ok(())
6782    }
6783
6784    #[test]
6785    fn avro_4004_canonical_form_strip_logical_types() -> TestResult {
6786        let schema_str = r#"
6787      {
6788        "type": "record",
6789        "name": "test",
6790        "fields": [
6791            {"name": "a", "type": "long", "default": 42, "doc": "The field a"},
6792            {"name": "b", "type": "string", "namespace": "test.a"},
6793            {"name": "c", "type": "long", "logicalType": "timestamp-micros"}
6794        ]
6795    }"#;
6796
6797        let schema = Schema::parse_str(schema_str)?;
6798        let canonical_form = schema.canonical_form();
6799        let fp_rabin = schema.fingerprint::<Rabin>();
6800        assert_eq!(
6801            r#"{"name":"test","type":"record","fields":[{"name":"a","type":"long"},{"name":"b","type":"string"},{"name":"c","type":{"type":"long"}}]}"#,
6802            canonical_form
6803        );
6804        assert_eq!("92f2ccef718c6754", fp_rabin.to_string());
6805        Ok(())
6806    }
6807}