You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/12/07 17:43:00 UTC

[jira] [Commented] (AVRO-1702) Add LogicalType support to c++ library

    [ https://issues.apache.org/jira/browse/AVRO-1702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16713125#comment-16713125 ] 

ASF GitHub Bot commented on AVRO-1702:
--------------------------------------

dkulp closed pull request #302: AVRO-1702: Added support for logical types in the C++ client.
URL: https://github.com/apache/avro/pull/302
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/lang/c++/CMakeLists.txt b/lang/c++/CMakeLists.txt
index be3921521..d0b4c98f2 100644
--- a/lang/c++/CMakeLists.txt
+++ b/lang/c++/CMakeLists.txt
@@ -78,7 +78,7 @@ add_definitions (${Boost_LIB_DIAGNOSTIC_DEFINITIONS})
 include_directories (api ${CMAKE_CURRENT_BINARY_DIR} ${Boost_INCLUDE_DIRS})
 
 set (AVRO_SOURCE_FILES
-        impl/Compiler.cc impl/Node.cc
+        impl/Compiler.cc impl/Node.cc impl/LogicalType.cc
         impl/NodeImpl.cc impl/ResolverSchema.cc impl/Schema.cc
         impl/Types.cc impl/ValidSchema.cc impl/Zigzag.cc
         impl/BinaryEncoder.cc impl/BinaryDecoder.cc
diff --git a/lang/c++/api/GenericDatum.hh b/lang/c++/api/GenericDatum.hh
index edc4fbc43..0273443da 100644
--- a/lang/c++/api/GenericDatum.hh
+++ b/lang/c++/api/GenericDatum.hh
@@ -26,6 +26,7 @@
 
 #include <boost/any.hpp>
 
+#include "LogicalType.hh"
 #include "Node.hh"
 #include "ValidSchema.hh"
 
@@ -54,12 +55,18 @@ namespace avro {
  */
 class AVRO_DECL GenericDatum {
     Type type_;
+    LogicalType logicalType_;
     boost::any value_;
 
-    GenericDatum(Type t) : type_(t) { }
+    GenericDatum(Type t)
+        : type_(t), logicalType_(LogicalType::NONE) { }
+
+    GenericDatum(Type t, LogicalType logicalType)
+        : type_(t), logicalType_(logicalType) { }
 
     template <typename T>
-    GenericDatum(Type t, const T& v) : type_(t), value_(v) { }
+    GenericDatum(Type t, LogicalType logicalType, const T& v)
+        : type_(t), logicalType_(logicalType), value_(v) { }
 
     void init(const NodePtr& schema);
 public:
@@ -68,6 +75,11 @@ public:
      */
     Type type() const;
 
+    /**
+     * The avro logical type that augments the main data type this datum holds.
+     */
+    LogicalType logicalType() const;
+
     /**
      * Returns the value held by this datum.
      * T The type for the value. This must correspond to the
@@ -104,30 +116,36 @@ public:
     void selectBranch(size_t branch);
 
     /// Makes a new AVRO_NULL datum.
-    GenericDatum() : type_(AVRO_NULL) { }
+    GenericDatum() : type_(AVRO_NULL), logicalType_(LogicalType::NONE) { }
 
     /// Makes a new AVRO_BOOL datum whose value is of type bool.
-    GenericDatum(bool v) : type_(AVRO_BOOL), value_(v) { }
+    GenericDatum(bool v)
+        : type_(AVRO_BOOL), logicalType_(LogicalType::NONE), value_(v) { }
 
     /// Makes a new AVRO_INT datum whose value is of type int32_t.
-    GenericDatum(int32_t v) : type_(AVRO_INT), value_(v) { }
+    GenericDatum(int32_t v)
+        : type_(AVRO_INT), logicalType_(LogicalType::NONE), value_(v) { }
 
     /// Makes a new AVRO_LONG datum whose value is of type int64_t.
-    GenericDatum(int64_t v) : type_(AVRO_LONG), value_(v) { }
+    GenericDatum(int64_t v)
+        : type_(AVRO_LONG), logicalType_(LogicalType::NONE), value_(v) { }
 
     /// Makes a new AVRO_FLOAT datum whose value is of type float.
-    GenericDatum(float v) : type_(AVRO_FLOAT), value_(v) { }
+    GenericDatum(float v)
+        : type_(AVRO_FLOAT), logicalType_(LogicalType::NONE), value_(v) { }
 
     /// Makes a new AVRO_DOUBLE datum whose value is of type double.
-    GenericDatum(double v) : type_(AVRO_DOUBLE), value_(v) { }
+    GenericDatum(double v)
+        : type_(AVRO_DOUBLE), logicalType_(LogicalType::NONE), value_(v) { }
 
     /// Makes a new AVRO_STRING datum whose value is of type std::string.
-    GenericDatum(const std::string& v) : type_(AVRO_STRING), value_(v) { }
+    GenericDatum(const std::string& v)
+        : type_(AVRO_STRING), logicalType_(LogicalType::NONE), value_(v) { }
 
     /// Makes a new AVRO_BYTES datum whose value is of type
     /// std::vector<uint8_t>.
     GenericDatum(const std::vector<uint8_t>& v) :
-        type_(AVRO_BYTES), value_(v) { }
+        type_(AVRO_BYTES), logicalType_(LogicalType::NONE), value_(v) { }
 
     /**
      * Constructs a datum corresponding to the given avro type.
@@ -145,7 +163,7 @@ public:
      */
     template<typename T>
     GenericDatum(const NodePtr& schema, const T& v) :
-        type_(schema->type()) {
+        type_(schema->type()), logicalType_(schema->logicalType()) {
         init(schema);
         *boost::any_cast<T>(&value_) = v;
     }
@@ -494,6 +512,10 @@ inline Type GenericDatum::type() const {
         type_;
 }
 
+inline LogicalType GenericDatum::logicalType() const {
+    return logicalType_;
+}
+
 template<typename T> T& GenericDatum::value() {
     return (type_ == AVRO_UNION) ?
         boost::any_cast<GenericUnion>(&value_)->datum().value<T>() :
diff --git a/lang/c++/api/LogicalType.hh b/lang/c++/api/LogicalType.hh
new file mode 100644
index 000000000..46c0e7645
--- /dev/null
+++ b/lang/c++/api/LogicalType.hh
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef avro_LogicalType_hh__
+#define avro_LogicalType_hh__
+
+#include <iostream>
+
+#include "Config.hh"
+
+namespace avro {
+
+class AVRO_DECL LogicalType {
+  public:
+    enum Type {
+        NONE,
+        DECIMAL,
+        DATE,
+        TIME_MILLIS,
+        TIME_MICROS,
+        TIMESTAMP_MILLIS,
+        TIMESTAMP_MICROS,
+        DURATION
+    };
+
+    explicit LogicalType(Type type);
+
+    Type type() const;
+
+    // Precision and scale can only be set for the DECIMAL logical type.
+    // Precision must be positive and scale must be either positive or zero. The
+    // setters will throw an exception if they are called on any type other
+    // than DECIMAL.
+    void setPrecision(int precision);
+    int precision() const { return precision_; }
+    void setScale(int scale);
+    int scale() const { return scale_; }
+
+    void printJson(std::ostream& os) const;
+
+  private:
+    Type type_;
+    int precision_;
+    int scale_;
+};
+
+}  // namespace avro
+
+#endif
diff --git a/lang/c++/api/Node.hh b/lang/c++/api/Node.hh
index ff227b9e4..fe42a26b5 100644
--- a/lang/c++/api/Node.hh
+++ b/lang/c++/api/Node.hh
@@ -26,6 +26,7 @@
 #include <boost/shared_ptr.hpp>
 
 #include "Exception.hh"
+#include "LogicalType.hh"
 #include "Types.hh"
 #include "SchemaResolution.hh"
 
@@ -91,6 +92,7 @@ class AVRO_DECL Node : private boost::noncopyable
 
     Node(Type type) :
         type_(type),
+        logicalType_(LogicalType::NONE),
         locked_(false)
     {}
 
@@ -100,6 +102,12 @@ class AVRO_DECL Node : private boost::noncopyable
         return type_;
     }
 
+    LogicalType logicalType() const {
+        return logicalType_;
+    }
+
+    void setLogicalType(LogicalType logicalType);
+
     void lock() {
         locked_ = true;
     }
@@ -172,6 +180,7 @@ class AVRO_DECL Node : private boost::noncopyable
   private:
 
     const Type type_;
+    LogicalType logicalType_;
     bool locked_;
 };
 
diff --git a/lang/c++/impl/Compiler.cc b/lang/c++/impl/Compiler.cc
index be5fe3f86..ae17dc0cd 100644
--- a/lang/c++/impl/Compiler.cc
+++ b/lang/c++/impl/Compiler.cc
@@ -100,6 +100,13 @@ static NodePtr makeNode(const std::string& t, SymbolTable& st, const string& ns)
     throw Exception(boost::format("Unknown type: %1%") % n.fullname());
 }
 
+/** Returns "true" if the field is in the container */
+// e.g.: can be false for non-mandatory fields
+bool containsField(const Entity& e, const Object& m, const string& fieldName) {
+    Object::const_iterator it = m.find(fieldName);
+    return (it != m.end());
+}
+
 const json::Object::const_iterator findField(const Entity& e,
     const Object& m, const string& fieldName)
 {
@@ -327,6 +334,44 @@ static NodePtr makeRecordNode(const Entity& e,
         fieldValues, fieldNames, defaultValues));
 }
 
+static LogicalType makeLogicalType(const Entity& e, const Object& m) {
+    if (!containsField(e, m, "logicalType")) {
+        return LogicalType(LogicalType::NONE);
+    }
+
+    const std::string& typeField = getStringField(e, m, "logicalType");
+
+    if (typeField == "decimal") {
+        LogicalType decimalType(LogicalType::DECIMAL);
+        try {
+            decimalType.setPrecision(getLongField(e, m, "precision"));
+            if (containsField(e, m, "scale")) {
+                decimalType.setScale(getLongField(e, m, "scale"));
+            }
+        } catch (Exception& ex) {
+            // If any part of the logical type is malformed, per the standard we
+            // must ignore the whole attribute.
+            return LogicalType(LogicalType::NONE);
+        }
+        return decimalType;
+    }
+
+    LogicalType::Type t = LogicalType::NONE;
+    if (typeField == "date")
+        t = LogicalType::DATE;
+    else if (typeField == "time-millis")
+        t = LogicalType::TIME_MILLIS;
+    else if (typeField == "time-micros")
+        t = LogicalType::TIME_MICROS;
+    else if (typeField == "timestamp-millis")
+        t = LogicalType::TIMESTAMP_MILLIS;
+    else if (typeField == "timestamp-micros")
+        t = LogicalType::TIMESTAMP_MICROS;
+    else if (typeField == "duration")
+        t = LogicalType::DURATION;
+    return LogicalType(t);
+}
+
 static NodePtr makeEnumNode(const Entity& e,
     const Name& name, const Object& m)
 {
@@ -397,12 +442,10 @@ static NodePtr makeNode(const Entity& e, const Object& m,
     SymbolTable& st, const string& ns)
 {
     const string& type = getStringField(e, m, "type");
-    if (NodePtr result = makePrimitive(type)) {
-        return result;
-    } else if (type == "record" || type == "error" ||
+    NodePtr result;
+    if (type == "record" || type == "error" ||
         type == "enum" || type == "fixed") {
         Name nm = getName(e, m, ns);
-        NodePtr result;
         if (type == "record" || type == "error") {
             result = NodePtr(new NodeRecord());
             st[nm] = result;
@@ -414,12 +457,24 @@ static NodePtr makeNode(const Entity& e, const Object& m,
                 makeFixedNode(e, nm, m);
             st[nm] = result;
         }
-        return result;
     } else if (type == "array") {
-        return makeArrayNode(e, m, st, ns);
+        result = makeArrayNode(e, m, st, ns);
     } else if (type == "map") {
-        return makeMapNode(e, m, st, ns);
+        result = makeMapNode(e, m, st, ns);
+    } else {
+        result = makePrimitive(type);
+    }
+
+    if (result) {
+        try {
+            result->setLogicalType(makeLogicalType(e, m));
+        } catch (Exception& ex) {
+            // Per the standard we must ignore the logical type attribute if it
+            // is malformed.
+        }
+        return result;
     }
+
     throw Exception(boost::format("Unknown type definition: %1%")
         % e.toString());
 }
diff --git a/lang/c++/impl/GenericDatum.cc b/lang/c++/impl/GenericDatum.cc
index b5998a84a..4dc50477d 100644
--- a/lang/c++/impl/GenericDatum.cc
+++ b/lang/c++/impl/GenericDatum.cc
@@ -25,12 +25,15 @@ using std::vector;
 namespace avro {
 
 GenericDatum::GenericDatum(const ValidSchema& schema) :
-    type_(schema.root()->type())
+    type_(schema.root()->type()),
+    logicalType_(schema.root()->logicalType())
 {
     init(schema.root());
 }
 
-GenericDatum::GenericDatum(const NodePtr& schema) : type_(schema->type())
+GenericDatum::GenericDatum(const NodePtr& schema) :
+    type_(schema->type()),
+    logicalType_(schema->logicalType())
 {
     init(schema);
 }
@@ -41,6 +44,7 @@ void GenericDatum::init(const NodePtr& schema)
     if (type_ == AVRO_SYMBOLIC) {
         sc = resolveSymbol(schema);
         type_ = sc->type();
+        logicalType_ = sc->logicalType();
     }
     switch (type_) {
     case AVRO_NULL:
diff --git a/lang/c++/impl/LogicalType.cc b/lang/c++/impl/LogicalType.cc
new file mode 100644
index 000000000..1e84dd200
--- /dev/null
+++ b/lang/c++/impl/LogicalType.cc
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "Exception.hh"
+#include "LogicalType.hh"
+
+namespace avro {
+
+LogicalType::LogicalType(Type type)
+    : type_(type), precision_(0), scale_(0) {}
+
+LogicalType::Type LogicalType::type() const {
+    return type_;
+}
+
+void LogicalType::setPrecision(int precision) {
+    if (type_ != DECIMAL) {
+        throw Exception("Only logical type DECIMAL can have precision");
+    }
+    if (precision <= 0) {
+        throw Exception(boost::format("Precision cannot be: %1%") % precision);
+    }
+    precision_ = precision;
+}
+
+void LogicalType::setScale(int scale) {
+    if (type_ != DECIMAL) {
+        throw Exception("Only logical type DECIMAL can have scale");
+    }
+    if (scale < 0) {
+        throw Exception(boost::format("Scale cannot be: %1%") % scale);
+    }
+    scale_ = scale;
+}
+
+void LogicalType::printJson(std::ostream& os) const {
+    switch (type_) {
+    case LogicalType::NONE:
+        break;
+    case LogicalType::DECIMAL:
+        os << "\"logicalType\": \"decimal\"";
+        os << ", \"precision\": " << precision_;
+        os << ", \"scale\": " << scale_;
+        break;
+    case DATE:
+        os << "\"logicalType\": \"date\"";
+        break;
+    case TIME_MILLIS:
+        os << "\"logicalType\": \"time-millis\"";
+        break;
+    case TIME_MICROS:
+        os << "\"logicalType\": \"time-micros\"";
+        break;
+    case TIMESTAMP_MILLIS:
+        os << "\"logicalType\": \"timestamp-millis\"";
+        break;
+    case TIMESTAMP_MICROS:
+        os << "\"logicalType\": \"timestamp-micros\"";
+        break;
+    case DURATION:
+        os << "\"logicalType\": \"duration\"";
+        break;
+    }
+}
+
+}  // namespace avro
diff --git a/lang/c++/impl/Node.cc b/lang/c++/impl/Node.cc
index 5fa965f69..1beb1df0b 100644
--- a/lang/c++/impl/Node.cc
+++ b/lang/c++/impl/Node.cc
@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 
+#include <cmath>
+
 #include "Node.hh"
 
 namespace avro {
@@ -80,4 +82,75 @@ bool Name::operator == (const Name& n) const
     return ns_ == n.ns_ && simpleName_ == n.simpleName_;
 }
 
+void Node::setLogicalType(LogicalType logicalType) {
+    checkLock();
+
+    // Check that the logical type is applicable to the node type.
+    switch (logicalType.type()) {
+    case LogicalType::NONE:
+        break;
+    case LogicalType::DECIMAL: {
+        if (type_ != AVRO_BYTES && type_ != AVRO_FIXED) {
+            throw Exception("DECIMAL logical type can annotate "
+                            "only BYTES or FIXED type");
+        }
+        if (type_ == AVRO_FIXED) {
+            // Max precision that can be supported by the current size of
+            // the FIXED type.
+            long maxPrecision =
+                floor(log10(pow(2.0, 8.0 * fixedSize() - 1) - 1));
+            if (logicalType.precision() > maxPrecision) {
+                throw Exception(
+                    boost::format(
+                        "DECIMAL precision %1% is too large for the "
+                        "FIXED type of size %2%, precision cannot be "
+                        "larget than %3%") % logicalType.precision() %
+                        fixedSize() % maxPrecision);
+            }
+        }
+        if (logicalType.scale() > logicalType.precision()) {
+            throw Exception("DECIMAL scale cannot exceed precision");
+        }
+        break;
+    }
+    case LogicalType::DATE:
+        if (type_ != AVRO_INT) {
+            throw Exception("DATE logical type can only annotate INT type");
+        }
+        break;
+    case LogicalType::TIME_MILLIS:
+        if (type_ != AVRO_INT) {
+            throw Exception("TIME-MILLIS logical type can only annotate "
+                            "INT type");
+        }
+        break;
+    case LogicalType::TIME_MICROS:
+        if (type_ != AVRO_LONG) {
+            throw Exception("TIME-MICROS logical type can only annotate "
+                            "LONG type");
+        }
+        break;
+    case LogicalType::TIMESTAMP_MILLIS:
+        if (type_ != AVRO_LONG) {
+            throw Exception("TIMESTAMP-MILLIS logical type can only annotate "
+                            "LONG type");
+        }
+        break;
+    case LogicalType::TIMESTAMP_MICROS:
+        if (type_ != AVRO_LONG) {
+            throw Exception("TIMESTAMP-MICROS logical type can only annotate "
+                            "LONG type");
+        }
+        break;
+    case LogicalType::DURATION:
+        if (type_ != AVRO_FIXED || fixedSize() != 12) {
+            throw Exception("DURATION logical type can only annotate "
+                            "FIXED type of size 12");
+        }
+        break;
+    }
+
+    logicalType_ = logicalType;
+}
+
 } // namespace avro
diff --git a/lang/c++/impl/NodeImpl.cc b/lang/c++/impl/NodeImpl.cc
index 606cd2093..da1949d77 100644
--- a/lang/c++/impl/NodeImpl.cc
+++ b/lang/c++/impl/NodeImpl.cc
@@ -166,7 +166,19 @@ std::ostream& operator <<(std::ostream &os, indent x)
 void 
 NodePrimitive::printJson(std::ostream &os, int depth) const
 {
+    bool hasLogicalType = logicalType().type() != LogicalType::NONE;
+
+    if (hasLogicalType) {
+        os << "{\n" << indent(depth) << "\"type\": ";
+    }
+
     os << '\"' << type() << '\"';
+
+    if (hasLogicalType) {
+        os << ",\n" << indent(depth);
+        logicalType().printJson(os);
+        os << "\n}";
+    }
 }
 
 void 
@@ -274,8 +286,14 @@ NodeFixed::printJson(std::ostream &os, int depth) const
     os << "{\n";
     os << indent(++depth) << "\"type\": \"fixed\",\n";
     printName(os, nameAttribute_.get(), depth);
-    os << indent(depth) << "\"size\": " << sizeAttribute_.get() << "\n";
-    os << indent(--depth) << '}';
+    os << indent(depth) << "\"size\": " << sizeAttribute_.get();
+
+    if (logicalType().type() != LogicalType::NONE) {
+      os << ",\n" << indent(depth);
+      logicalType().printJson(os);
+    }
+
+    os << "\n" << indent(--depth) << '}';
 }
 
 } // namespace avro
diff --git a/lang/c++/test/SchemaTests.cc b/lang/c++/test/SchemaTests.cc
index 8ecde7a6e..b840e0c1f 100644
--- a/lang/c++/test/SchemaTests.cc
+++ b/lang/c++/test/SchemaTests.cc
@@ -17,6 +17,7 @@
  */
 
 #include "Compiler.hh"
+#include "GenericDatum.hh"
 #include "ValidSchema.hh"
 
 #include <boost/test/included/unit_test_framework.hpp>
@@ -182,9 +183,39 @@ const char* roundTripSchemas[] = {
     "{\"type\":\"fixed\",\"namespace\":\"org.apache.hadoop.avro\","
           "\"name\":\"MyFixed\",\"size\":1}",
     "{\"type\":\"fixed\",\"name\":\"Test\",\"size\":1}",
-    "{\"type\":\"fixed\",\"name\":\"Test\",\"size\":1}"
+    "{\"type\":\"fixed\",\"name\":\"Test\",\"size\":1}",
+
+    // Logical types
+    R"({"type": "bytes", "logicalType": "decimal",
+        "precision": 12, "scale": 6})",
+    R"({"type": "fixed", "name": "test", "size": 16,
+        "logicalType": "decimal", "precision": 38, "scale": 9})",
+    R"({"type": "int", "logicalType": "date"})",
+    R"({"type": "int", "logicalType": "time-millis"})",
+    R"({"type": "long", "logicalType": "time-micros"})",
+    R"({"type": "long", "logicalType": "timestamp-millis"})",
+    R"({"type": "long", "logicalType": "timestamp-micros"})",
+    R"({"type": "fixed", "name": "test", "size": 12,
+        "logicalType": "duration"})"
 };
 
+const char* malformedLogicalTypes[] = {
+    // Wrong base type.
+    R"({"type": "long", "logicalType": "decimal", "precision": 10})",
+    R"({"type": "string", "logicalType": "date"})",
+    R"({"type": "string", "logicalType": "time-millis"})",
+    R"({"type": "string", "logicalType": "time-micros"})",
+    R"({"type": "string", "logicalType": "timestamp-millis"})",
+    R"({"type": "string", "logicalType": "timestamp-micros"})",
+    R"({"type": "string", "logicalType": "duration"})",
+    // Missing the required field 'precision'.
+    R"({"type": "bytes", "logicalType": "decimal"})",
+    // The claimed precision is not supported by the size of the fixed type.
+    R"({"type": "fixed", "size": 4, "name": "a", "precision": 20})",
+    // Scale is larger than precision.
+    R"({"type": "bytes", "logicalType": "decimal",
+        "precision": 5, "scale": 10})"
+};
 
 
 static void testBasic(const char* schema)
@@ -214,8 +245,152 @@ static void testRoundTrip(const char* schema)
     std::ostringstream os;
     compiledSchema.toJson(os);
     std::string result = os.str();
+
+    std::string cleanedSchema(schema);
+    // Remove whitespace for comparison.
+    cleanedSchema.erase(std::remove_if(cleanedSchema.begin(),
+                                       cleanedSchema.end(),
+                                       ::isspace),
+                        cleanedSchema.end());
+
     result.erase(std::remove_if(result.begin(), result.end(), ::isspace), result.end()); // Remove whitespace
-    BOOST_CHECK(result == std::string(schema));
+    BOOST_CHECK(result == cleanedSchema);
+}
+
+static void testLogicalTypes()
+{
+    const char* bytesDecimalType = R"(
+    {
+        "type": "bytes",
+        "logicalType": "decimal",
+        "precision": 10,
+        "scale": 2
+    })";
+    const char* fixedDecimalType = R"(
+    {
+        "type": "fixed",
+        "size": 16,
+        "name": "fixedDecimalType",
+        "logicalType": "decimal",
+        "precision": 12,
+        "scale": 6
+    })";
+    const char* dateType = R"(
+    {
+        "type": "int", "logicalType": "date"
+    })";
+    const char* timeMillisType = R"(
+    {
+        "type": "int", "logicalType": "time-millis"
+    })";
+    const char* timeMicrosType = R"(
+    {
+        "type": "long", "logicalType": "time-micros"
+    })";
+    const char* timestampMillisType = R"(
+    {
+        "type": "long", "logicalType": "timestamp-millis"
+    })";
+    const char* timestampMicrosType = R"(
+    {
+        "type": "long", "logicalType": "timestamp-micros"
+    })";
+    const char* durationType = R"(
+    {
+        "type": "fixed",
+        "size": 12,
+        "name": "durationType",
+        "logicalType": "duration"
+    })";
+
+    {
+        BOOST_TEST_CHECKPOINT(bytesDecimalType);
+        ValidSchema schema1 = compileJsonSchemaFromString(bytesDecimalType);
+        BOOST_CHECK(schema1.root()->type() == AVRO_BYTES);
+        LogicalType logicalType = schema1.root()->logicalType();
+        BOOST_CHECK(logicalType.type() == LogicalType::DECIMAL);
+        BOOST_CHECK(logicalType.precision() == 10);
+        BOOST_CHECK(logicalType.scale() == 2);
+
+        BOOST_TEST_CHECKPOINT(fixedDecimalType);
+        ValidSchema schema2 = compileJsonSchemaFromString(fixedDecimalType);
+        BOOST_CHECK(schema2.root()->type() == AVRO_FIXED);
+        logicalType = schema2.root()->logicalType();
+        BOOST_CHECK(logicalType.type() == LogicalType::DECIMAL);
+        BOOST_CHECK(logicalType.precision() == 12);
+        BOOST_CHECK(logicalType.scale() == 6);
+
+        GenericDatum bytesDatum(schema1);
+        BOOST_CHECK(bytesDatum.logicalType().type() == LogicalType::DECIMAL);
+        GenericDatum fixedDatum(schema2);
+        BOOST_CHECK(fixedDatum.logicalType().type() == LogicalType::DECIMAL);
+    }
+    {
+        BOOST_TEST_CHECKPOINT(dateType);
+        ValidSchema schema = compileJsonSchemaFromString(dateType);
+        BOOST_CHECK(schema.root()->type() == AVRO_INT);
+        BOOST_CHECK(schema.root()->logicalType().type() == LogicalType::DATE);
+        GenericDatum datum(schema);
+        BOOST_CHECK(datum.logicalType().type() == LogicalType::DATE);
+    }
+    {
+        BOOST_TEST_CHECKPOINT(timeMillisType);
+        ValidSchema schema = compileJsonSchemaFromString(timeMillisType);
+        BOOST_CHECK(schema.root()->type() == AVRO_INT);
+        LogicalType logicalType = schema.root()->logicalType();
+        BOOST_CHECK(logicalType.type() == LogicalType::TIME_MILLIS);
+        GenericDatum datum(schema);
+        BOOST_CHECK(datum.logicalType().type() == LogicalType::TIME_MILLIS);
+    }
+    {
+        BOOST_TEST_CHECKPOINT(timeMicrosType);
+        ValidSchema schema = compileJsonSchemaFromString(timeMicrosType);
+        BOOST_CHECK(schema.root()->type() == AVRO_LONG);
+        LogicalType logicalType = schema.root()->logicalType();
+        BOOST_CHECK(logicalType.type() == LogicalType::TIME_MICROS);
+        GenericDatum datum(schema);
+        BOOST_CHECK(datum.logicalType().type() == LogicalType::TIME_MICROS);
+    }
+    {
+        BOOST_TEST_CHECKPOINT(timestampMillisType);
+        ValidSchema schema = compileJsonSchemaFromString(timestampMillisType);
+        BOOST_CHECK(schema.root()->type() == AVRO_LONG);
+        LogicalType logicalType = schema.root()->logicalType();
+        BOOST_CHECK(logicalType.type() == LogicalType::TIMESTAMP_MILLIS);
+        GenericDatum datum(schema);
+        BOOST_CHECK(datum.logicalType().type() ==
+                    LogicalType::TIMESTAMP_MILLIS);
+    }
+    {
+        BOOST_TEST_CHECKPOINT(timestampMicrosType);
+        ValidSchema schema = compileJsonSchemaFromString(timestampMicrosType);
+        BOOST_CHECK(schema.root()->type() == AVRO_LONG);
+        LogicalType logicalType = schema.root()->logicalType();
+        BOOST_CHECK(logicalType.type() == LogicalType::TIMESTAMP_MICROS);
+        GenericDatum datum(schema);
+        BOOST_CHECK(datum.logicalType().type() ==
+                    LogicalType::TIMESTAMP_MICROS);
+    }
+    {
+        BOOST_TEST_CHECKPOINT(durationType);
+        ValidSchema schema = compileJsonSchemaFromString(durationType);
+        BOOST_CHECK(schema.root()->type() == AVRO_FIXED);
+        BOOST_CHECK(schema.root()->fixedSize() == 12);
+        LogicalType logicalType = schema.root()->logicalType();
+        BOOST_CHECK(logicalType.type() == LogicalType::DURATION);
+        GenericDatum datum(schema);
+        BOOST_CHECK(datum.logicalType().type() == LogicalType::DURATION);
+    }
+}
+
+static void testMalformedLogicalTypes(const char* schema)
+{
+    BOOST_TEST_CHECKPOINT(schema);
+    ValidSchema parsedSchema = compileJsonSchemaFromString(schema);
+    LogicalType logicalType = parsedSchema.root()->logicalType();
+    BOOST_CHECK(logicalType.type() == LogicalType::NONE);
+    GenericDatum datum(parsedSchema);
+    BOOST_CHECK(datum.logicalType().type() == LogicalType::NONE);
 }
 
 }
@@ -238,5 +413,8 @@ init_unit_test_suite(int argc, char* argv[])
         avro::schema::basicSchemaErrors);
     ADD_PARAM_TEST(ts, avro::schema::testCompile, avro::schema::basicSchemas);
     ADD_PARAM_TEST(ts, avro::schema::testRoundTrip, avro::schema::roundTripSchemas);
+    ts->add(BOOST_TEST_CASE(&avro::schema::testLogicalTypes));
+    ADD_PARAM_TEST(ts, avro::schema::testMalformedLogicalTypes,
+                   avro::schema::malformedLogicalTypes);
     return ts;
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Add LogicalType support to c++ library
> --------------------------------------
>
>                 Key: AVRO-1702
>                 URL: https://issues.apache.org/jira/browse/AVRO-1702
>             Project: Apache Avro
>          Issue Type: New Feature
>          Components: c++
>            Reporter: peter liu
>            Assignee: Aniket Mokashi
>            Priority: Major
>             Fix For: 1.9.0
>
>
> I'd like to port the logicaltype support to c++ library



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)