You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2017/03/22 18:03:46 UTC

arrow git commit: ARROW-683: [C++/Python] Refactor to make Date32 and Date64 types for new metadata. Test IPC roundtrip

Repository: arrow
Updated Branches:
  refs/heads/master 36103143b -> 71424c20d


ARROW-683: [C++/Python] Refactor to make Date32 and Date64 types for new metadata. Test IPC roundtrip

Maintains existing Python behavior (datetime.date getting converted to milliseconds)

Author: Wes McKinney <we...@twosigma.com>

Closes #418 from wesm/ARROW-683 and squashes the following commits:

69f156b [Wes McKinney] Add autoconf-archive to README for system requirements
10988ad [Wes McKinney] Remove hacks for ax_cxx_compile_stdcxx_11
334558d [Wes McKinney] Fix glib for date32/date64. Add ax_cxx_compile_stdcxx_11.m4 macro for older autoconf
93cf8d6 [Wes McKinney] Refactor to make Date32 and Date64 types for new metadata. Test IPC roundtrips, maintain existing Python behavior


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/71424c20
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/71424c20
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/71424c20

Branch: refs/heads/master
Commit: 71424c20d31addb37cf7db56561790ca69db0430
Parents: 3610314
Author: Wes McKinney <we...@twosigma.com>
Authored: Wed Mar 22 14:03:40 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Wed Mar 22 14:03:40 2017 -0400

----------------------------------------------------------------------
 c_glib/README.md                             |  5 +--
 c_glib/arrow-glib/type.cpp                   |  6 ++--
 c_glib/arrow-glib/type.h                     |  6 ++--
 cpp/src/arrow/array.cc                       |  4 +--
 cpp/src/arrow/array.h                        |  4 +--
 cpp/src/arrow/builder.cc                     |  5 +--
 cpp/src/arrow/builder.h                      |  2 +-
 cpp/src/arrow/compare.cc                     | 10 +++---
 cpp/src/arrow/ipc/ipc-json-test.cc           |  8 +++--
 cpp/src/arrow/ipc/ipc-read-write-test.cc     |  8 ++---
 cpp/src/arrow/ipc/json-internal.cc           | 16 +++++----
 cpp/src/arrow/ipc/metadata.cc                | 18 +++++++---
 cpp/src/arrow/ipc/test-common.h              | 29 +++++-----------
 cpp/src/arrow/ipc/writer.cc                  |  2 +-
 cpp/src/arrow/loader.cc                      |  2 +-
 cpp/src/arrow/pretty_print.cc                |  4 +--
 cpp/src/arrow/type-test.cc                   |  8 +++++
 cpp/src/arrow/type.cc                        | 12 +++----
 cpp/src/arrow/type.h                         | 40 +++++++++++------------
 cpp/src/arrow/type_fwd.h                     |  8 ++---
 cpp/src/arrow/type_traits.h                  |  8 ++---
 python/pyarrow/__init__.py                   |  2 +-
 python/pyarrow/array.pyx                     |  9 +++--
 python/pyarrow/includes/libarrow.pxd         |  8 +++--
 python/pyarrow/scalar.pyx                    | 19 ++++++++---
 python/pyarrow/schema.pyx                    | 13 +++++---
 python/pyarrow/tests/test_convert_builtin.py |  2 +-
 python/pyarrow/tests/test_convert_pandas.py  |  2 +-
 python/src/pyarrow/adapters/builtin.cc       |  6 ++--
 python/src/pyarrow/adapters/pandas.cc        | 18 +++++-----
 python/src/pyarrow/helpers.cc                |  3 +-
 python/src/pyarrow/type_traits.h             |  2 +-
 32 files changed, 166 insertions(+), 123 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/c_glib/README.md
----------------------------------------------------------------------
diff --git a/c_glib/README.md b/c_glib/README.md
index 4008015..84027bf 100644
--- a/c_glib/README.md
+++ b/c_glib/README.md
@@ -58,7 +58,7 @@ to build Arrow GLib. You can install them by the followings:
 On Debian GNU/Linux or Ubuntu:
 
 ```text
-% sudo apt install -y -V gtk-doc-tools libgirepository1.0-dev
+% sudo apt install -y -V gtk-doc-tools autoconf-archive libgirepository1.0-dev
 ```
 
 On CentOS 7 or later:
@@ -76,7 +76,8 @@ On macOS with [Homebrew](https://brew.sh/):
 Now, you can build Arrow GLib:
 
 ```text
-% cd glib
+% cd c_glib
+% ./autogen.sh
 % ./configure --enable-gtk-doc
 % make
 % sudo make install

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/c_glib/arrow-glib/type.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/type.cpp b/c_glib/arrow-glib/type.cpp
index 56cbc21..2e59647 100644
--- a/c_glib/arrow-glib/type.cpp
+++ b/c_glib/arrow-glib/type.cpp
@@ -66,8 +66,10 @@ garrow_type_from_raw(arrow::Type::type type)
     return GARROW_TYPE_STRING;
   case arrow::Type::type::BINARY:
     return GARROW_TYPE_BINARY;
-  case arrow::Type::type::DATE:
-    return GARROW_TYPE_DATE;
+  case arrow::Type::type::DATE32:
+    return GARROW_TYPE_DATE32;
+  case arrow::Type::type::DATE64:
+    return GARROW_TYPE_DATE64;
   case arrow::Type::type::TIMESTAMP:
     return GARROW_TYPE_TIMESTAMP;
   case arrow::Type::type::TIME:

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/c_glib/arrow-glib/type.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/type.h b/c_glib/arrow-glib/type.h
index 48d2801..cd6137c 100644
--- a/c_glib/arrow-glib/type.h
+++ b/c_glib/arrow-glib/type.h
@@ -40,7 +40,8 @@ G_BEGIN_DECLS
  * @GARROW_TYPE_DOUBLE: 8-byte floating point value.
  * @GARROW_TYPE_STRING: UTF-8 variable-length string.
  * @GARROW_TYPE_BINARY: Variable-length bytes (no guarantee of UTF-8-ness).
- * @GARROW_TYPE_DATE: By default, int32 days since the UNIX epoch.
+ * @GARROW_TYPE_DATE32: int32 days since the UNIX epoch.
+ * @GARROW_TYPE_DATE64: int64 milliseconds since the UNIX epoch.
  * @GARROW_TYPE_TIMESTAMP: Exact timestamp encoded with int64 since UNIX epoch.
  *   Default unit millisecond.
  * @GARROW_TYPE_TIME: Exact time encoded with int64, default unit millisecond.
@@ -70,7 +71,8 @@ typedef enum {
   GARROW_TYPE_DOUBLE,
   GARROW_TYPE_STRING,
   GARROW_TYPE_BINARY,
-  GARROW_TYPE_DATE,
+  GARROW_TYPE_DATE32,
+  GARROW_TYPE_DATE64,
   GARROW_TYPE_TIMESTAMP,
   GARROW_TYPE_TIME,
   GARROW_TYPE_INTERVAL,

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/array.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array.cc b/cpp/src/arrow/array.cc
index 36b3fcc..4fa2b2b 100644
--- a/cpp/src/arrow/array.cc
+++ b/cpp/src/arrow/array.cc
@@ -489,8 +489,8 @@ ARRAY_VISITOR_DEFAULT(DoubleArray);
 ARRAY_VISITOR_DEFAULT(BinaryArray);
 ARRAY_VISITOR_DEFAULT(StringArray);
 ARRAY_VISITOR_DEFAULT(FixedWidthBinaryArray);
-ARRAY_VISITOR_DEFAULT(DateArray);
 ARRAY_VISITOR_DEFAULT(Date32Array);
+ARRAY_VISITOR_DEFAULT(Date64Array);
 ARRAY_VISITOR_DEFAULT(TimeArray);
 ARRAY_VISITOR_DEFAULT(TimestampArray);
 ARRAY_VISITOR_DEFAULT(IntervalArray);
@@ -515,8 +515,8 @@ template class NumericArray<Int16Type>;
 template class NumericArray<Int32Type>;
 template class NumericArray<Int64Type>;
 template class NumericArray<TimestampType>;
-template class NumericArray<DateType>;
 template class NumericArray<Date32Type>;
+template class NumericArray<Date64Type>;
 template class NumericArray<TimeType>;
 template class NumericArray<HalfFloatType>;
 template class NumericArray<FloatType>;

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/array.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array.h b/cpp/src/arrow/array.h
index 50faf08..e66ac50 100644
--- a/cpp/src/arrow/array.h
+++ b/cpp/src/arrow/array.h
@@ -58,8 +58,8 @@ class ARROW_EXPORT ArrayVisitor {
   virtual Status Visit(const StringArray& array);
   virtual Status Visit(const BinaryArray& array);
   virtual Status Visit(const FixedWidthBinaryArray& array);
-  virtual Status Visit(const DateArray& array);
   virtual Status Visit(const Date32Array& array);
+  virtual Status Visit(const Date64Array& array);
   virtual Status Visit(const TimeArray& array);
   virtual Status Visit(const TimestampArray& array);
   virtual Status Visit(const IntervalArray& array);
@@ -559,8 +559,8 @@ extern template class ARROW_EXPORT NumericArray<HalfFloatType>;
 extern template class ARROW_EXPORT NumericArray<FloatType>;
 extern template class ARROW_EXPORT NumericArray<DoubleType>;
 extern template class ARROW_EXPORT NumericArray<TimestampType>;
-extern template class ARROW_EXPORT NumericArray<DateType>;
 extern template class ARROW_EXPORT NumericArray<Date32Type>;
+extern template class ARROW_EXPORT NumericArray<Date64Type>;
 extern template class ARROW_EXPORT NumericArray<TimeType>;
 
 #if defined(__GNUC__) && !defined(__clang__)

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/builder.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/builder.cc b/cpp/src/arrow/builder.cc
index b65a492..483d6f0 100644
--- a/cpp/src/arrow/builder.cc
+++ b/cpp/src/arrow/builder.cc
@@ -238,8 +238,8 @@ template class PrimitiveBuilder<Int8Type>;
 template class PrimitiveBuilder<Int16Type>;
 template class PrimitiveBuilder<Int32Type>;
 template class PrimitiveBuilder<Int64Type>;
-template class PrimitiveBuilder<DateType>;
 template class PrimitiveBuilder<Date32Type>;
+template class PrimitiveBuilder<Date64Type>;
 template class PrimitiveBuilder<TimestampType>;
 template class PrimitiveBuilder<TimeType>;
 template class PrimitiveBuilder<HalfFloatType>;
@@ -531,7 +531,8 @@ Status MakeBuilder(MemoryPool* pool, const std::shared_ptr<DataType>& type,
     BUILDER_CASE(INT32, Int32Builder);
     BUILDER_CASE(UINT64, UInt64Builder);
     BUILDER_CASE(INT64, Int64Builder);
-    BUILDER_CASE(DATE, DateBuilder);
+    BUILDER_CASE(DATE32, Date32Builder);
+    BUILDER_CASE(DATE64, Date64Builder);
     case Type::TIMESTAMP:
       out->reset(new TimestampBuilder(pool, type));
       return Status::OK();

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/builder.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/builder.h b/cpp/src/arrow/builder.h
index 07b7cfc..7cefa64 100644
--- a/cpp/src/arrow/builder.h
+++ b/cpp/src/arrow/builder.h
@@ -232,8 +232,8 @@ using Int32Builder = NumericBuilder<Int32Type>;
 using Int64Builder = NumericBuilder<Int64Type>;
 using TimestampBuilder = NumericBuilder<TimestampType>;
 using TimeBuilder = NumericBuilder<TimeType>;
-using DateBuilder = NumericBuilder<DateType>;
 using Date32Builder = NumericBuilder<Date32Type>;
+using Date64Builder = NumericBuilder<Date64Type>;
 
 using HalfFloatBuilder = NumericBuilder<HalfFloatType>;
 using FloatBuilder = NumericBuilder<FloatType>;

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/compare.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/compare.cc b/cpp/src/arrow/compare.cc
index 86ed8cc..3e6ecef 100644
--- a/cpp/src/arrow/compare.cc
+++ b/cpp/src/arrow/compare.cc
@@ -169,12 +169,14 @@ class RangeEqualsVisitor : public ArrayVisitor {
     return Status::OK();
   }
 
-  Status Visit(const DateArray& left) override { return CompareValues<DateArray>(left); }
-
   Status Visit(const Date32Array& left) override {
     return CompareValues<Date32Array>(left);
   }
 
+  Status Visit(const Date64Array& left) override {
+    return CompareValues<Date64Array>(left);
+  }
+
   Status Visit(const TimeArray& left) override { return CompareValues<TimeArray>(left); }
 
   Status Visit(const TimestampArray& left) override {
@@ -409,10 +411,10 @@ class ArrayEqualsVisitor : public RangeEqualsVisitor {
 
   Status Visit(const DoubleArray& left) override { return ComparePrimitive(left); }
 
-  Status Visit(const DateArray& left) override { return ComparePrimitive(left); }
-
   Status Visit(const Date32Array& left) override { return ComparePrimitive(left); }
 
+  Status Visit(const Date64Array& left) override { return ComparePrimitive(left); }
+
   Status Visit(const TimeArray& left) override { return ComparePrimitive(left); }
 
   Status Visit(const TimestampArray& left) override { return ComparePrimitive(left); }

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/ipc/ipc-json-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/ipc-json-test.cc b/cpp/src/arrow/ipc/ipc-json-test.cc
index 4c18a49..fd35182 100644
--- a/cpp/src/arrow/ipc/ipc-json-test.cc
+++ b/cpp/src/arrow/ipc/ipc-json-test.cc
@@ -96,15 +96,17 @@ void CheckPrimitive(const std::shared_ptr<DataType>& type,
 }
 
 TEST(TestJsonSchemaWriter, FlatTypes) {
+  // TODO
+  // field("f14", date32())
   std::vector<std::shared_ptr<Field>> fields = {field("f0", int8()),
       field("f1", int16(), false), field("f2", int32()), field("f3", int64(), false),
       field("f4", uint8()), field("f5", uint16()), field("f6", uint32()),
       field("f7", uint64()), field("f8", float32()), field("f9", float64()),
       field("f10", utf8()), field("f11", binary()), field("f12", list(int32())),
       field("f13", struct_({field("s1", int32()), field("s2", utf8())})),
-      field("f14", date()), field("f15", timestamp(TimeUnit::NANO)),
-      field("f16", time(TimeUnit::MICRO)),
-      field("f17", union_({field("u1", int8()), field("u2", time(TimeUnit::MILLI))},
+      field("f15", date64()), field("f16", timestamp(TimeUnit::NANO)),
+      field("f17", time(TimeUnit::MICRO)),
+      field("f18", union_({field("u1", int8()), field("u2", time(TimeUnit::MILLI))},
                        {0, 1}, UnionMode::DENSE))};
 
   Schema schema(fields);

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/ipc/ipc-read-write-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/ipc-read-write-test.cc b/cpp/src/arrow/ipc/ipc-read-write-test.cc
index 261ca1d..0011844 100644
--- a/cpp/src/arrow/ipc/ipc-read-write-test.cc
+++ b/cpp/src/arrow/ipc/ipc-read-write-test.cc
@@ -117,10 +117,10 @@ TEST_F(TestSchemaMetadata, NestedFields) {
   CheckRoundtrip(schema, &memo);
 }
 
-#define BATCH_CASES()                                                                   \
-  ::testing::Values(&MakeIntRecordBatch, &MakeListRecordBatch, &MakeNonNullRecordBatch, \
-      &MakeZeroLengthRecordBatch, &MakeDeeplyNestedList, &MakeStringTypesRecordBatch,   \
-      &MakeStruct, &MakeUnion, &MakeDictionary, &MakeDate, &MakeTimestamps, &MakeTimes, \
+#define BATCH_CASES()                                                                    \
+  ::testing::Values(&MakeIntRecordBatch, &MakeListRecordBatch, &MakeNonNullRecordBatch,  \
+      &MakeZeroLengthRecordBatch, &MakeDeeplyNestedList, &MakeStringTypesRecordBatch,    \
+      &MakeStruct, &MakeUnion, &MakeDictionary, &MakeDates, &MakeTimestamps, &MakeTimes, \
       &MakeFWBinary);
 
 class IpcTestFixture : public io::MemoryMapFixture {

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/ipc/json-internal.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/json-internal.cc b/cpp/src/arrow/ipc/json-internal.cc
index 549b26b..08f0bdc 100644
--- a/cpp/src/arrow/ipc/json-internal.cc
+++ b/cpp/src/arrow/ipc/json-internal.cc
@@ -133,10 +133,7 @@ class JsonSchemaWriter : public TypeVisitor {
   }
 
   template <typename T>
-  typename std::enable_if<
-      std::is_base_of<NoExtraMeta, T>::value || std::is_base_of<BooleanType, T>::value ||
-          std::is_base_of<DateType, T>::value || std::is_base_of<NullType, T>::value,
-      void>::type
+  typename std::enable_if<std::is_base_of<NoExtraMeta, T>::value, void>::type
   WriteTypeMetadata(const T& type) {}
 
   template <typename T>
@@ -303,7 +300,10 @@ class JsonSchemaWriter : public TypeVisitor {
 
   Status Visit(const BinaryType& type) override { return WriteVarBytes("binary", type); }
 
-  Status Visit(const DateType& type) override { return WritePrimitive("date", type); }
+  // TODO
+  Status Visit(const Date32Type& type) override { return WritePrimitive("date", type); }
+
+  Status Visit(const Date64Type& type) override { return WritePrimitive("date", type); }
 
   Status Visit(const TimeType& type) override { return WritePrimitive("time", type); }
 
@@ -733,7 +733,8 @@ class JsonSchemaReader {
     } else if (type_name == "null") {
       *type = null();
     } else if (type_name == "date") {
-      *type = date();
+      // TODO
+      *type = date64();
     } else if (type_name == "time") {
       return GetTimeLike<TimeType>(json_type, type);
     } else if (type_name == "timestamp") {
@@ -1059,7 +1060,8 @@ class JsonArrayReader {
       TYPE_CASE(DoubleType);
       TYPE_CASE(StringType);
       TYPE_CASE(BinaryType);
-      NOT_IMPLEMENTED_CASE(DATE);
+      NOT_IMPLEMENTED_CASE(DATE32);
+      NOT_IMPLEMENTED_CASE(DATE64);
       NOT_IMPLEMENTED_CASE(TIMESTAMP);
       NOT_IMPLEMENTED_CASE(TIME);
       NOT_IMPLEMENTED_CASE(INTERVAL);

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/ipc/metadata.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/metadata.cc b/cpp/src/arrow/ipc/metadata.cc
index 4dfda54..c091bac 100644
--- a/cpp/src/arrow/ipc/metadata.cc
+++ b/cpp/src/arrow/ipc/metadata.cc
@@ -241,9 +241,15 @@ static Status TypeFromFlatbuffer(flatbuf::Type type, const void* type_data,
       return Status::OK();
     case flatbuf::Type_Decimal:
       return Status::NotImplemented("Decimal");
-    case flatbuf::Type_Date:
-      *out = date();
+    case flatbuf::Type_Date: {
+      auto date_type = static_cast<const flatbuf::Date*>(type_data);
+      if (date_type->unit() == flatbuf::DateUnit_DAY) {
+        *out = date32();
+      } else {
+        *out = date64();
+      }
       return Status::OK();
+    }
     case flatbuf::Type_Time: {
       auto time_type = static_cast<const flatbuf::Time*>(type_data);
       *out = time(FromFlatbufferUnit(time_type->unit()));
@@ -358,9 +364,13 @@ static Status TypeToFlatbuffer(FBB& fbb, const std::shared_ptr<DataType>& type,
       *out_type = flatbuf::Type_Utf8;
       *offset = flatbuf::CreateUtf8(fbb).Union();
       break;
-    case Type::DATE:
+    case Type::DATE32:
+      *out_type = flatbuf::Type_Date;
+      *offset = flatbuf::CreateDate(fbb, flatbuf::DateUnit_DAY).Union();
+      break;
+    case Type::DATE64:
       *out_type = flatbuf::Type_Date;
-      *offset = flatbuf::CreateDate(fbb).Union();
+      *offset = flatbuf::CreateDate(fbb, flatbuf::DateUnit_MILLISECOND).Union();
       break;
     case Type::TIME: {
       const auto& time_type = static_cast<const TimeType&>(*type);

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/ipc/test-common.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/test-common.h b/cpp/src/arrow/ipc/test-common.h
index 330af0c..4085ecf 100644
--- a/cpp/src/arrow/ipc/test-common.h
+++ b/cpp/src/arrow/ipc/test-common.h
@@ -463,33 +463,22 @@ Status MakeDictionaryFlat(std::shared_ptr<RecordBatch>* out) {
   return Status::OK();
 }
 
-Status MakeDate(std::shared_ptr<RecordBatch>* out) {
-  std::vector<bool> is_valid = {true, true, true, false, true, true, true};
-  auto f1 = field("f1", date());
-  std::shared_ptr<Schema> schema(new Schema({f1}));
-
-  std::vector<int64_t> date_values = {1489269000000, 1489270000000, 1489271000000,
-      1489272000000, 1489272000000, 1489273000000};
-
-  std::shared_ptr<Array> date_array;
-  ArrayFromVector<DateType, int64_t>(is_valid, date_values, &date_array);
-
-  std::vector<std::shared_ptr<Array>> arrays = {date_array};
-  *out = std::make_shared<RecordBatch>(schema, date_array->length(), arrays);
-  return Status::OK();
-}
-
-Status MakeDate32(std::shared_ptr<RecordBatch>* out) {
+Status MakeDates(std::shared_ptr<RecordBatch>* out) {
   std::vector<bool> is_valid = {true, true, true, false, true, true, true};
   auto f0 = field("f0", date32());
-  std::shared_ptr<Schema> schema(new Schema({f0}));
+  auto f1 = field("f1", date64());
+  std::shared_ptr<Schema> schema(new Schema({f0, f1}));
 
   std::vector<int32_t> date32_values = {0, 1, 2, 3, 4, 5, 6};
-
   std::shared_ptr<Array> date32_array;
   ArrayFromVector<Date32Type, int32_t>(is_valid, date32_values, &date32_array);
 
-  std::vector<std::shared_ptr<Array>> arrays = {date32_array};
+  std::vector<int64_t> date64_values = {1489269000000, 1489270000000, 1489271000000,
+      1489272000000, 1489272000000, 1489273000000};
+  std::shared_ptr<Array> date64_array;
+  ArrayFromVector<Date64Type, int64_t>(is_valid, date64_values, &date64_array);
+
+  std::vector<std::shared_ptr<Array>> arrays = {date32_array, date64_array};
   *out = std::make_shared<RecordBatch>(schema, date32_array->length(), arrays);
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/ipc/writer.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc
index 82c119e..ef59471 100644
--- a/cpp/src/arrow/ipc/writer.cc
+++ b/cpp/src/arrow/ipc/writer.cc
@@ -336,8 +336,8 @@ class RecordBatchWriter : public ArrayVisitor {
   VISIT_FIXED_WIDTH(HalfFloatArray);
   VISIT_FIXED_WIDTH(FloatArray);
   VISIT_FIXED_WIDTH(DoubleArray);
-  VISIT_FIXED_WIDTH(DateArray);
   VISIT_FIXED_WIDTH(Date32Array);
+  VISIT_FIXED_WIDTH(Date64Array);
   VISIT_FIXED_WIDTH(TimeArray);
   VISIT_FIXED_WIDTH(TimestampArray);
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/loader.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/loader.cc b/cpp/src/arrow/loader.cc
index fc37371..bc506be 100644
--- a/cpp/src/arrow/loader.cc
+++ b/cpp/src/arrow/loader.cc
@@ -146,8 +146,8 @@ class ArrayLoader : public TypeVisitor {
   VISIT_PRIMITIVE(HalfFloatType);
   VISIT_PRIMITIVE(FloatType);
   VISIT_PRIMITIVE(DoubleType);
-  VISIT_PRIMITIVE(DateType);
   VISIT_PRIMITIVE(Date32Type);
+  VISIT_PRIMITIVE(Date64Type);
   VISIT_PRIMITIVE(TimeType);
   VISIT_PRIMITIVE(TimestampType);
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/pretty_print.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/pretty_print.cc b/cpp/src/arrow/pretty_print.cc
index 87c1a1c..fc5eed1 100644
--- a/cpp/src/arrow/pretty_print.cc
+++ b/cpp/src/arrow/pretty_print.cc
@@ -171,10 +171,10 @@ class ArrayPrinter : public ArrayVisitor {
 
   Status Visit(const FixedWidthBinaryArray& array) override { return WriteArray(array); }
 
-  Status Visit(const DateArray& array) override { return WriteArray(array); }
-
   Status Visit(const Date32Array& array) override { return WriteArray(array); }
 
+  Status Visit(const Date64Array& array) override { return WriteArray(array); }
+
   Status Visit(const TimeArray& array) override { return WriteArray(array); }
 
   Status Visit(const TimestampArray& array) override {

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/type-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type-test.cc b/cpp/src/arrow/type-test.cc
index 22aa7eb..c2d115c 100644
--- a/cpp/src/arrow/type-test.cc
+++ b/cpp/src/arrow/type-test.cc
@@ -173,6 +173,14 @@ TEST(TestListType, Basics) {
   ASSERT_EQ("list<item: list<item: string>>", lt2.ToString());
 }
 
+TEST(TestDateTypes, ToString) {
+  auto t1 = date32();
+  auto t2 = date64();
+
+  ASSERT_EQ("date32[day]", t1->ToString());
+  ASSERT_EQ("date64[ms]", t2->ToString());
+}
+
 TEST(TestTimeType, Equals) {
   TimeType t1;
   TimeType t2;

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/type.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.cc b/cpp/src/arrow/type.cc
index 64070cb..937cbc5 100644
--- a/cpp/src/arrow/type.cc
+++ b/cpp/src/arrow/type.cc
@@ -110,12 +110,12 @@ std::string StructType::ToString() const {
   return s.str();
 }
 
-std::string DateType::ToString() const {
-  return std::string("date");
+std::string Date64Type::ToString() const {
+  return std::string("date64[ms]");
 }
 
 std::string Date32Type::ToString() const {
-  return std::string("date32");
+  return std::string("date32[day]");
 }
 
 std::string TimeType::ToString() const {
@@ -205,7 +205,7 @@ ACCEPT_VISITOR(ListType);
 ACCEPT_VISITOR(StructType);
 ACCEPT_VISITOR(DecimalType);
 ACCEPT_VISITOR(UnionType);
-ACCEPT_VISITOR(DateType);
+ACCEPT_VISITOR(Date64Type);
 ACCEPT_VISITOR(Date32Type);
 ACCEPT_VISITOR(TimeType);
 ACCEPT_VISITOR(TimestampType);
@@ -233,7 +233,7 @@ TYPE_FACTORY(float32, FloatType);
 TYPE_FACTORY(float64, DoubleType);
 TYPE_FACTORY(utf8, StringType);
 TYPE_FACTORY(binary, BinaryType);
-TYPE_FACTORY(date, DateType);
+TYPE_FACTORY(date64, Date64Type);
 TYPE_FACTORY(date32, Date32Type);
 
 std::shared_ptr<DataType> fixed_width_binary(int32_t byte_width) {
@@ -355,7 +355,7 @@ TYPE_VISITOR_DEFAULT(DoubleType);
 TYPE_VISITOR_DEFAULT(StringType);
 TYPE_VISITOR_DEFAULT(BinaryType);
 TYPE_VISITOR_DEFAULT(FixedWidthBinaryType);
-TYPE_VISITOR_DEFAULT(DateType);
+TYPE_VISITOR_DEFAULT(Date64Type);
 TYPE_VISITOR_DEFAULT(Date32Type);
 TYPE_VISITOR_DEFAULT(TimeType);
 TYPE_VISITOR_DEFAULT(TimestampType);

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/type.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index 27b28d2..c179bf3 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -71,12 +71,12 @@ struct Type {
     // Fixed-width binary. Each value occupies the same number of bytes
     FIXED_WIDTH_BINARY,
 
-    // int64_t milliseconds since the UNIX epoch
-    DATE,
-
     // int32_t days since the UNIX epoch
     DATE32,
 
+    // int64_t milliseconds since the UNIX epoch
+    DATE64,
+
     // Exact timestamp encoded with int64 since UNIX epoch
     // Default unit millisecond
     TIMESTAMP,
@@ -139,7 +139,7 @@ class ARROW_EXPORT TypeVisitor {
   virtual Status Visit(const StringType& type);
   virtual Status Visit(const BinaryType& type);
   virtual Status Visit(const FixedWidthBinaryType& type);
-  virtual Status Visit(const DateType& type);
+  virtual Status Visit(const Date64Type& type);
   virtual Status Visit(const Date32Type& type);
   virtual Status Visit(const TimeType& type);
   virtual Status Visit(const TimestampType& type);
@@ -245,7 +245,7 @@ struct ARROW_EXPORT CTypeImpl : public PrimitiveCType {
   std::string ToString() const override { return std::string(DERIVED::name()); }
 };
 
-struct ARROW_EXPORT NullType : public DataType {
+struct ARROW_EXPORT NullType : public DataType, public NoExtraMeta {
   static constexpr Type::type type_id = Type::NA;
 
   NullType() : DataType(Type::NA) {}
@@ -263,7 +263,7 @@ struct IntegerTypeImpl : public CTypeImpl<DERIVED, TYPE_ID, C_TYPE>, public Inte
   bool is_signed() const override { return std::is_signed<C_TYPE>::value; }
 };
 
-struct ARROW_EXPORT BooleanType : public FixedWidthType {
+struct ARROW_EXPORT BooleanType : public FixedWidthType, public NoExtraMeta {
   static constexpr Type::type type_id = Type::BOOL;
 
   BooleanType() : FixedWidthType(Type::BOOL) {}
@@ -455,33 +455,33 @@ struct ARROW_EXPORT UnionType : public DataType {
 // ----------------------------------------------------------------------
 // Date and time types
 
-/// Date as int64_t milliseconds since UNIX epoch
-struct ARROW_EXPORT DateType : public FixedWidthType {
-  static constexpr Type::type type_id = Type::DATE;
+/// Date as int32_t days since UNIX epoch
+struct ARROW_EXPORT Date32Type : public FixedWidthType, public NoExtraMeta {
+  static constexpr Type::type type_id = Type::DATE32;
 
-  using c_type = int64_t;
+  using c_type = int32_t;
 
-  DateType() : FixedWidthType(Type::DATE) {}
+  Date32Type() : FixedWidthType(Type::DATE32) {}
 
-  int bit_width() const override { return static_cast<int>(sizeof(c_type) * 8); }
+  int bit_width() const override { return static_cast<int>(sizeof(c_type) * 4); }
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
-  static std::string name() { return "date"; }
 };
 
-/// Date as int32_t days since UNIX epoch
-struct ARROW_EXPORT Date32Type : public FixedWidthType {
-  static constexpr Type::type type_id = Type::DATE32;
+/// Date as int64_t milliseconds since UNIX epoch
+struct ARROW_EXPORT Date64Type : public FixedWidthType, public NoExtraMeta {
+  static constexpr Type::type type_id = Type::DATE64;
 
-  using c_type = int32_t;
+  using c_type = int64_t;
 
-  Date32Type() : FixedWidthType(Type::DATE32) {}
+  Date64Type() : FixedWidthType(Type::DATE64) {}
 
-  int bit_width() const override { return static_cast<int>(sizeof(c_type) * 4); }
+  int bit_width() const override { return static_cast<int>(sizeof(c_type) * 8); }
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
+  static std::string name() { return "date"; }
 };
 
 enum class TimeUnit : char { SECOND = 0, MILLI = 1, MICRO = 2, NANO = 3 };
@@ -666,8 +666,8 @@ static inline bool is_primitive(Type::type type_id) {
     case Type::HALF_FLOAT:
     case Type::FLOAT:
     case Type::DOUBLE:
-    case Type::DATE:
     case Type::DATE32:
+    case Type::DATE64:
     case Type::TIMESTAMP:
     case Type::TIME:
     case Type::INTERVAL:

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/type_fwd.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type_fwd.h b/cpp/src/arrow/type_fwd.h
index 7fc36c4..ae85593 100644
--- a/cpp/src/arrow/type_fwd.h
+++ b/cpp/src/arrow/type_fwd.h
@@ -95,9 +95,9 @@ _NUMERIC_TYPE_DECL(Double);
 
 #undef _NUMERIC_TYPE_DECL
 
-struct DateType;
-using DateArray = NumericArray<DateType>;
-using DateBuilder = NumericBuilder<DateType>;
+struct Date64Type;
+using Date64Array = NumericArray<Date64Type>;
+using Date64Builder = NumericBuilder<Date64Type>;
 
 struct Date32Type;
 using Date32Array = NumericArray<Date32Type>;
@@ -132,8 +132,8 @@ std::shared_ptr<DataType> ARROW_EXPORT float32();
 std::shared_ptr<DataType> ARROW_EXPORT float64();
 std::shared_ptr<DataType> ARROW_EXPORT utf8();
 std::shared_ptr<DataType> ARROW_EXPORT binary();
-std::shared_ptr<DataType> ARROW_EXPORT date();
 std::shared_ptr<DataType> ARROW_EXPORT date32();
+std::shared_ptr<DataType> ARROW_EXPORT date64();
 
 }  // namespace arrow
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/cpp/src/arrow/type_traits.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type_traits.h b/cpp/src/arrow/type_traits.h
index 242e59d..e731913 100644
--- a/cpp/src/arrow/type_traits.h
+++ b/cpp/src/arrow/type_traits.h
@@ -119,15 +119,15 @@ struct TypeTraits<Int64Type> {
 };
 
 template <>
-struct TypeTraits<DateType> {
-  using ArrayType = DateArray;
-  using BuilderType = DateBuilder;
+struct TypeTraits<Date64Type> {
+  using ArrayType = Date64Array;
+  using BuilderType = Date64Builder;
 
   static inline int64_t bytes_required(int64_t elements) {
     return elements * sizeof(int64_t);
   }
   constexpr static bool is_parameter_free = true;
-  static inline std::shared_ptr<DataType> type_singleton() { return date(); }
+  static inline std::shared_ptr<DataType> type_singleton() { return date64(); }
 };
 
 template <>

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/python/pyarrow/__init__.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/__init__.py b/python/pyarrow/__init__.py
index a4aac44..c6f0be0 100644
--- a/python/pyarrow/__init__.py
+++ b/python/pyarrow/__init__.py
@@ -61,7 +61,7 @@ import pyarrow.schema as _schema
 from pyarrow.schema import (null, bool_,
                             int8, int16, int32, int64,
                             uint8, uint16, uint32, uint64,
-                            timestamp, date,
+                            timestamp, date32, date64,
                             float_, double, binary, string,
                             list_, struct, dictionary, field,
                             DataType, Field, Schema, schema)

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/python/pyarrow/array.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/array.pyx b/python/pyarrow/array.pyx
index 11244e7..6afeaa0 100644
--- a/python/pyarrow/array.pyx
+++ b/python/pyarrow/array.pyx
@@ -372,7 +372,11 @@ cdef class UInt64Array(IntegerArray):
     pass
 
 
-cdef class DateArray(NumericArray):
+cdef class Date32Array(NumericArray):
+    pass
+
+
+cdef class Date64Array(NumericArray):
     pass
 
 
@@ -459,7 +463,8 @@ cdef dict _array_classes = {
     Type_INT16: Int16Array,
     Type_INT32: Int32Array,
     Type_INT64: Int64Array,
-    Type_DATE: DateArray,
+    Type_DATE32: Date32Array,
+    Type_DATE64: Date64Array,
     Type_FLOAT: FloatArray,
     Type_DOUBLE: DoubleArray,
     Type_LIST: ListArray,

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/python/pyarrow/includes/libarrow.pxd
----------------------------------------------------------------------
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index 2d698d3..1d9c38e 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -39,7 +39,8 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         Type_DOUBLE" arrow::Type::DOUBLE"
 
         Type_TIMESTAMP" arrow::Type::TIMESTAMP"
-        Type_DATE" arrow::Type::DATE"
+        Type_DATE32" arrow::Type::DATE32"
+        Type_DATE64" arrow::Type::DATE64"
         Type_BINARY" arrow::Type::BINARY"
         Type_STRING" arrow::Type::STRING"
 
@@ -177,7 +178,10 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
     cdef cppclass CInt64Array" arrow::Int64Array"(CArray):
         int64_t Value(int i)
 
-    cdef cppclass CDateArray" arrow::DateArray"(CArray):
+    cdef cppclass CDate32Array" arrow::Date32Array"(CArray):
+        int32_t Value(int i)
+
+    cdef cppclass CDate64Array" arrow::Date64Array"(CArray):
         int64_t Value(int i)
 
     cdef cppclass CTimestampArray" arrow::TimestampArray"(CArray):

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/python/pyarrow/scalar.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/scalar.pyx b/python/pyarrow/scalar.pyx
index 1337b2b..8c88f90 100644
--- a/python/pyarrow/scalar.pyx
+++ b/python/pyarrow/scalar.pyx
@@ -124,11 +124,18 @@ cdef class UInt64Value(ArrayValue):
         return ap.Value(self.index)
 
 
-cdef class DateValue(ArrayValue):
+cdef class Date32Value(ArrayValue):
 
     def as_py(self):
-        cdef CDateArray* ap = <CDateArray*> self.sp_array.get()
-        return datetime.datetime.utcfromtimestamp(ap.Value(self.index) / 1000).date()
+        raise NotImplementedError
+
+
+cdef class Date64Value(ArrayValue):
+
+    def as_py(self):
+        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
+        return datetime.datetime.utcfromtimestamp(
+            ap.Value(self.index) / 1000).date()
 
 
 cdef class TimestampValue(ArrayValue):
@@ -147,7 +154,8 @@ cdef class TimestampValue(ArrayValue):
             return datetime.datetime.utcfromtimestamp(float(val) / 1000000)
         else:
             # TimeUnit_NANO
-            raise NotImplementedError("Cannot convert nanosecond timestamps to datetime.datetime")
+            raise NotImplementedError("Cannot convert nanosecond timestamps "
+                                      "to datetime.datetime")
 
 
 cdef class FloatValue(ArrayValue):
@@ -226,7 +234,8 @@ cdef dict _scalar_classes = {
     Type_INT16: Int16Value,
     Type_INT32: Int32Value,
     Type_INT64: Int64Value,
-    Type_DATE: DateValue,
+    Type_DATE32: Date32Value,
+    Type_DATE64: Date64Value,
     Type_TIMESTAMP: TimestampValue,
     Type_FLOAT: FloatValue,
     Type_DOUBLE: DoubleValue,

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/python/pyarrow/schema.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/schema.pyx b/python/pyarrow/schema.pyx
index ee38144..ab5ae5f 100644
--- a/python/pyarrow/schema.pyx
+++ b/python/pyarrow/schema.pyx
@@ -228,8 +228,9 @@ cdef set PRIMITIVE_TYPES = set([
     la.Type_UINT16, la.Type_INT16,
     la.Type_UINT32, la.Type_INT32,
     la.Type_UINT64, la.Type_INT64,
-    la.Type_TIMESTAMP, la.Type_DATE,
-    la.Type_FLOAT, la.Type_DOUBLE])
+    la.Type_TIMESTAMP, la.Type_DATE32,
+    la.Type_DATE64, la.Type_FLOAT,
+    la.Type_DOUBLE])
 
 
 def null():
@@ -319,8 +320,12 @@ def timestamp(unit_str, tz=None):
     return out
 
 
-def date():
-    return primitive_type(la.Type_DATE)
+def date32():
+    return primitive_type(la.Type_DATE32)
+
+
+def date64():
+    return primitive_type(la.Type_DATE64)
 
 
 def float_():

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/python/pyarrow/tests/test_convert_builtin.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_convert_builtin.py b/python/pyarrow/tests/test_convert_builtin.py
index c06d18d..7915f97 100644
--- a/python/pyarrow/tests/test_convert_builtin.py
+++ b/python/pyarrow/tests/test_convert_builtin.py
@@ -97,7 +97,7 @@ class TestConvertList(unittest.TestCase):
                 datetime.date(2040, 2, 26)]
         arr = pyarrow.from_pylist(data)
         assert len(arr) == 4
-        assert arr.type == pyarrow.date()
+        assert arr.type == pyarrow.date64()
         assert arr.null_count == 1
         assert arr[0].as_py() == datetime.date(2000, 1, 1)
         assert arr[1].as_py() is None

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/python/pyarrow/tests/test_convert_pandas.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_convert_pandas.py b/python/pyarrow/tests/test_convert_pandas.py
index 6b89444..ea7a892 100644
--- a/python/pyarrow/tests/test_convert_pandas.py
+++ b/python/pyarrow/tests/test_convert_pandas.py
@@ -326,7 +326,7 @@ class TestPandasConversion(unittest.TestCase):
                      datetime.date(1970, 1, 1),
                      datetime.date(2040, 2, 26)]})
         table = A.Table.from_pandas(df)
-        field = A.Field.from_py('date', A.date())
+        field = A.Field.from_py('date', A.date64())
         schema = A.Schema.from_fields([field])
         assert table.schema.equals(schema)
         result = table.to_pandas()

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/python/src/pyarrow/adapters/builtin.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/adapters/builtin.cc b/python/src/pyarrow/adapters/builtin.cc
index b197f58..06e098a 100644
--- a/python/src/pyarrow/adapters/builtin.cc
+++ b/python/src/pyarrow/adapters/builtin.cc
@@ -82,7 +82,7 @@ class ScalarVisitor {
       // TODO(wesm): tighter type later
       return int64();
     } else if (date_count_) {
-      return date();
+      return date64();
     } else if (timestamp_count_) {
       return timestamp(TimeUnit::MICRO);
     } else if (bool_count_) {
@@ -291,7 +291,7 @@ class Int64Converter : public TypedConverter<Int64Builder> {
   }
 };
 
-class DateConverter : public TypedConverter<DateBuilder> {
+class DateConverter : public TypedConverter<Date64Builder> {
  public:
   Status AppendData(PyObject* seq) override {
     Py_ssize_t size = PySequence_Size(seq);
@@ -457,7 +457,7 @@ std::shared_ptr<SeqConverter> GetConverter(const std::shared_ptr<DataType>& type
       return std::make_shared<BoolConverter>();
     case Type::INT64:
       return std::make_shared<Int64Converter>();
-    case Type::DATE:
+    case Type::DATE64:
       return std::make_shared<DateConverter>();
     case Type::TIMESTAMP:
       return std::make_shared<TimestampConverter>();

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/python/src/pyarrow/adapters/pandas.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/adapters/pandas.cc b/python/src/pyarrow/adapters/pandas.cc
index 863cf54..a7386ce 100644
--- a/python/src/pyarrow/adapters/pandas.cc
+++ b/python/src/pyarrow/adapters/pandas.cc
@@ -379,7 +379,7 @@ Status PandasConverter::ConvertDates(std::shared_ptr<Array>* out) {
   PyAcquireGIL lock;
 
   PyObject** objects = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
-  DateBuilder date_builder(pool_);
+  Date64Builder date_builder(pool_);
   RETURN_NOT_OK(date_builder.Resize(length_));
 
   Status s;
@@ -477,7 +477,7 @@ Status PandasConverter::ConvertObjects(std::shared_ptr<Array>* out) {
         return ConvertObjectStrings(out);
       case Type::BOOL:
         return ConvertBooleans(out);
-      case Type::DATE:
+      case Type::DATE64:
         return ConvertDates(out);
       case Type::LIST: {
         const auto& list_field = static_cast<const ListType&>(*type_);
@@ -725,7 +725,7 @@ inline void set_numpy_metadata(int type, DataType* datatype, PyArrayObject* out)
           break;
       }
     } else {
-      // datatype->type == Type::DATE
+      // datatype->type == Type::DATE64
       date_dtype->meta.base = NPY_FR_D;
     }
   }
@@ -1245,8 +1245,8 @@ class DatetimeBlock : public PandasBlock {
 
     const ChunkedArray& data = *col.get()->data();
 
-    if (type == Type::DATE) {
-      // DateType is millisecond timestamp stored as int64_t
+    if (type == Type::DATE64) {
+      // Date64Type is millisecond timestamp stored as int64_t
       // TODO(wesm): Do we want to make sure to zero out the milliseconds?
       ConvertDatetimeNanos<int64_t, 1000000L>(data, out_buffer);
     } else if (type == Type::TIMESTAMP) {
@@ -1490,7 +1490,7 @@ class DataFrameBlockCreator {
         case Type::BINARY:
           output_type = PandasBlock::OBJECT;
           break;
-        case Type::DATE:
+        case Type::DATE64:
           output_type = PandasBlock::DATETIME;
           break;
         case Type::TIMESTAMP: {
@@ -1752,7 +1752,7 @@ class ArrowDeserializer {
       CONVERT_CASE(DOUBLE);
       CONVERT_CASE(BINARY);
       CONVERT_CASE(STRING);
-      CONVERT_CASE(DATE);
+      CONVERT_CASE(DATE64);
       CONVERT_CASE(TIMESTAMP);
       CONVERT_CASE(DICTIONARY);
       CONVERT_CASE(LIST);
@@ -1771,7 +1771,7 @@ class ArrowDeserializer {
 
   template <int TYPE>
   inline typename std::enable_if<
-      (TYPE != Type::DATE) & arrow_traits<TYPE>::is_numeric_nullable, Status>::type
+      (TYPE != Type::DATE64) & arrow_traits<TYPE>::is_numeric_nullable, Status>::type
   ConvertValues() {
     typedef typename arrow_traits<TYPE>::T T;
     int npy_type = arrow_traits<TYPE>::npy_type;
@@ -1788,7 +1788,7 @@ class ArrowDeserializer {
   }
 
   template <int TYPE>
-  inline typename std::enable_if<TYPE == Type::DATE, Status>::type ConvertValues() {
+  inline typename std::enable_if<TYPE == Type::DATE64, Status>::type ConvertValues() {
     typedef typename arrow_traits<TYPE>::T T;
 
     RETURN_NOT_OK(AllocateOutput(arrow_traits<TYPE>::npy_type));

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/python/src/pyarrow/helpers.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/helpers.cc b/python/src/pyarrow/helpers.cc
index edebea6..43edf8a 100644
--- a/python/src/pyarrow/helpers.cc
+++ b/python/src/pyarrow/helpers.cc
@@ -39,7 +39,8 @@ std::shared_ptr<DataType> GetPrimitiveType(Type::type type) {
       GET_PRIMITIVE_TYPE(INT32, int32);
       GET_PRIMITIVE_TYPE(UINT64, uint64);
       GET_PRIMITIVE_TYPE(INT64, int64);
-      GET_PRIMITIVE_TYPE(DATE, date);
+      GET_PRIMITIVE_TYPE(DATE32, date32);
+      GET_PRIMITIVE_TYPE(DATE64, date64);
       GET_PRIMITIVE_TYPE(BOOL, boolean);
       GET_PRIMITIVE_TYPE(FLOAT, float32);
       GET_PRIMITIVE_TYPE(DOUBLE, float64);

http://git-wip-us.apache.org/repos/asf/arrow/blob/71424c20/python/src/pyarrow/type_traits.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/type_traits.h b/python/src/pyarrow/type_traits.h
index f4604d7..cc65d5c 100644
--- a/python/src/pyarrow/type_traits.h
+++ b/python/src/pyarrow/type_traits.h
@@ -180,7 +180,7 @@ struct arrow_traits<Type::TIMESTAMP> {
 };
 
 template <>
-struct arrow_traits<Type::DATE> {
+struct arrow_traits<Type::DATE64> {
   static constexpr int npy_type = NPY_DATETIME;
   static constexpr bool supports_nulls = true;
   static constexpr int64_t na_value = kPandasTimestampNull;