You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/04/16 15:49:41 UTC

[GitHub] [arrow] zeroshade opened a new pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

zeroshade opened a new pull request #10071:
URL: https://github.com/apache/arrow/pull/10071


   Following up from #9817  this is the next chunk of code for the Go Parquet port consisting of the Schema package, implementing the Converted and Logical types along with handling schema creation, manipulation, and printing.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619882602



##########
File path: go/parquet/schema/converted_types.go
##########
@@ -0,0 +1,191 @@
+// 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.
+
+package schema
+
+import (
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// ConvertedType corresponds to the ConvertedType in the parquet.Thrift,
+// with added values of None and NA for handling when these values are not
+// set in the metadata
+type ConvertedType format.ConvertedType
+
+var (
+	// ConvertedTypes is a struct containing the constants for the types
+	// to make it easy to reference them while making it clear what they are
+	ConvertedTypes = struct {
+		None            ConvertedType
+		UTF8            ConvertedType
+		Map             ConvertedType
+		MapKeyValue     ConvertedType
+		List            ConvertedType
+		Enum            ConvertedType
+		Decimal         ConvertedType
+		Date            ConvertedType
+		TimeMillis      ConvertedType
+		TimeMicros      ConvertedType
+		TimestampMillis ConvertedType
+		TimestampMicros ConvertedType
+		Uint8           ConvertedType
+		Uint16          ConvertedType
+		Uint32          ConvertedType
+		Uint64          ConvertedType
+		Int8            ConvertedType
+		Int16           ConvertedType
+		Int32           ConvertedType
+		Int64           ConvertedType
+		JSON            ConvertedType
+		BSON            ConvertedType
+		Interval        ConvertedType
+		NA              ConvertedType
+	}{
+		None:            -1, // thrift enum starts at 0, so we know this will not be used
+		UTF8:            ConvertedType(format.ConvertedType_UTF8),
+		Map:             ConvertedType(format.ConvertedType_MAP),
+		MapKeyValue:     ConvertedType(format.ConvertedType_MAP_KEY_VALUE),
+		List:            ConvertedType(format.ConvertedType_LIST),
+		Enum:            ConvertedType(format.ConvertedType_ENUM),
+		Decimal:         ConvertedType(format.ConvertedType_DECIMAL),
+		Date:            ConvertedType(format.ConvertedType_DATE),
+		TimeMillis:      ConvertedType(format.ConvertedType_TIME_MILLIS),
+		TimeMicros:      ConvertedType(format.ConvertedType_TIME_MICROS),
+		TimestampMillis: ConvertedType(format.ConvertedType_TIMESTAMP_MILLIS),
+		TimestampMicros: ConvertedType(format.ConvertedType_TIMESTAMP_MICROS),
+		Uint8:           ConvertedType(format.ConvertedType_UINT_8),
+		Uint16:          ConvertedType(format.ConvertedType_UINT_16),
+		Uint32:          ConvertedType(format.ConvertedType_UINT_32),
+		Uint64:          ConvertedType(format.ConvertedType_UINT_64),
+		Int8:            ConvertedType(format.ConvertedType_INT_8),
+		Int16:           ConvertedType(format.ConvertedType_INT_16),
+		Int32:           ConvertedType(format.ConvertedType_INT_32),
+		Int64:           ConvertedType(format.ConvertedType_INT_64),
+		JSON:            ConvertedType(format.ConvertedType_JSON),
+		BSON:            ConvertedType(format.ConvertedType_BSON),
+		Interval:        ConvertedType(format.ConvertedType_INTERVAL),
+		NA:              24, // should always be the last values after Interval
+	}
+)
+
+func (p ConvertedType) String() string {
+	switch p {
+	case ConvertedTypes.None:
+		return "NONE"
+	case ConvertedTypes.NA:
+		return "UNKNOWN"
+	default:
+		return format.ConvertedType(p).String()
+	}
+}
+
+// ToLogicalType returns the correct LogicalType for the given ConvertedType, using the decimal
+// metadata provided to define the precision/scale if necessary
+func (p ConvertedType) ToLogicalType(convertedDecimal DecimalMetadata) LogicalType {
+	switch p {
+	case ConvertedTypes.UTF8:
+		return StringLogicalType{}
+	case ConvertedTypes.Map, ConvertedTypes.MapKeyValue:
+		return MapLogicalType{}
+	case ConvertedTypes.List:
+		return ListLogicalType{}
+	case ConvertedTypes.Enum:
+		return EnumLogicalType{}
+	case ConvertedTypes.Decimal:
+		return NewDecimalLogicalType(convertedDecimal.Precision, convertedDecimal.Scale)
+	case ConvertedTypes.Date:
+		return DateLogicalType{}
+	case ConvertedTypes.TimeMillis:
+		return NewTimeLogicalType(true, TimeUnitMillis)
+	case ConvertedTypes.TimeMicros:
+		return NewTimeLogicalType(true, TimeUnitMicros)
+	case ConvertedTypes.TimestampMillis:
+		t := NewTimestampLogicalType(true, TimeUnitMillis)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.TimestampMicros:
+		t := NewTimestampLogicalType(true, TimeUnitMicros)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.Interval:
+		return IntervalLogicalType{}
+	case ConvertedTypes.Int8:
+		return NewIntLogicalType(8, true)

Review comment:
       comments for literals.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r633659739



##########
File path: go/parquet/schema/node.go
##########
@@ -0,0 +1,625 @@
+// 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.
+
+package schema
+
+import (
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/thrift/lib/go/thrift"
+	"golang.org/x/xerrors"
+)
+
+// NodeType describes whether the Node is a Primitive or Group node
+type NodeType int
+
+// the available constants for NodeType
+const (
+	Primitive NodeType = iota
+	Group
+)
+
+// Node is the interface for both Group and Primitive Nodes.
+// A logical schema type has a name, repetition level, and optionally
+// a logical type (converted type is the deprecated version of the logical
+// type concept, which is maintained for forward compatibility)
+type Node interface {
+	Name() string
+	Type() NodeType
+	RepetitionType() parquet.Repetition
+	ConvertedType() ConvertedType
+	LogicalType() LogicalType
+	FieldID() int32
+	Parent() Node
+	SetParent(Node)
+	Path() string
+	Equals(Node) bool
+	Visit(v Visitor)
+	toThrift() *format.SchemaElement
+}
+
+// Visitor is an interface for creating functionality to walk the schema tree.
+//
+// A visitor can be passed to the Visit function of a Node in order to walk
+// the tree. VisitPre is called the first time a node is encountered. If
+// it is a group node, the return is checked and if it is false, the children
+// will be skipped.
+//
+// VisitPost is called after visiting any children
+type Visitor interface {
+	VisitPre(Node) bool
+	VisitPost(Node)
+}
+
+// ColumnPathFromNode walks the parents of the given node to construct it's
+// column path
+func ColumnPathFromNode(n Node) parquet.ColumnPath {
+	if n == nil {
+		return nil
+	}
+
+	c := make([]string, 0)
+
+	// build the path in reverse order as we traverse nodes to the top
+	cursor := n
+	for cursor.Parent() != nil {
+		c = append(c, cursor.Name())
+		cursor = cursor.Parent()
+	}
+
+	// reverse the order of the list in place so that our result
+	// is in the proper, correct order.
+	for i := len(c)/2 - 1; i >= 0; i-- {

Review comment:
       go doesn't have a built in reverse function?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r623235858



##########
File path: go/parquet/schema/reflection_test.go
##########
@@ -0,0 +1,397 @@
+// 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.
+
+package schema_test
+
+import (
+	"log"
+	"os"
+	"reflect"
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/schema"
+	"github.com/stretchr/testify/assert"
+)
+
+func ExampleNewSchemaFromStruct_primitives() {
+	type Schema struct {
+		Bool              bool
+		Int8              int8
+		Uint16            uint16
+		Int32             int32
+		Int64             int64
+		Int96             parquet.Int96
+		Float             float32
+		Double            float64
+		ByteArray         string
+		FixedLenByteArray [10]byte
+	}
+
+	sc, err := schema.NewSchemaFromStruct(Schema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 Schema {
+	//   required boolean field_id=-1 Bool;
+	//   required int32 field_id=-1 Int8 (Int(bitWidth=8, isSigned=true));
+	//   required int32 field_id=-1 Uint16 (Int(bitWidth=16, isSigned=false));
+	//   required int32 field_id=-1 Int32 (Int(bitWidth=32, isSigned=true));
+	//   required int64 field_id=-1 Int64 (Int(bitWidth=64, isSigned=true));
+	//   required int96 field_id=-1 Int96;
+	//   required float field_id=-1 Float;
+	//   required double field_id=-1 Double;
+	//   required byte_array field_id=-1 ByteArray;
+	//   required fixed_len_byte_array field_id=-1 FixedLenByteArray;
+	// }
+}
+
+func ExampleNewSchemaFromStruct_convertedtypes() {
+	type ConvertedSchema struct {
+		Utf8           string        `parquet:"name=utf8, converted=UTF8"`
+		Uint32         uint32        `parquet:"converted=INT_32"`
+		Date           int32         `parquet:"name=date, converted=date"`
+		TimeMilli      int32         `parquet:"name=timemilli, converted=TIME_MILLIS"`
+		TimeMicro      int64         `parquet:"name=timemicro, converted=time_micros"`
+		TimeStampMilli int64         `parquet:"converted=timestamp_millis"`
+		TimeStampMicro int64         `parquet:"converted=timestamp_micros"`
+		Interval       parquet.Int96 `parquet:"converted=INTERVAL"`
+		Decimal1       int32         `parquet:"converted=decimal, scale=2, precision=9"`
+		Decimal2       int64         `parquet:"converted=decimal, scale=2, precision=18"`
+		Decimal3       [12]byte      `parquet:"converted=decimal, scale=2, precision=10"`
+		Decimal4       string        `parquet:"converted=decimal, scale=2, precision=20"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(&ConvertedSchema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 ConvertedSchema {
+	//   required byte_array field_id=-1 utf8 (String);
+	//   required int32 field_id=-1 Uint32 (Int(bitWidth=32, isSigned=true));
+	//   required int32 field_id=-1 date (Date);
+	//   required int32 field_id=-1 timemilli (Time(isAdjustedToUTC=true, timeUnit=milliseconds));
+	//   required int64 field_id=-1 timemicro (Time(isAdjustedToUTC=true, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeStampMilli (Timestamp(isAdjustedToUTC=true, timeUnit=milliseconds, is_from_converted_type=true, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimeStampMicro (Timestamp(isAdjustedToUTC=true, timeUnit=microseconds, is_from_converted_type=true, force_set_converted_type=false));
+	//   required int96 field_id=-1 Interval;
+	//   required int32 field_id=-1 Decimal1 (Decimal(precision=9, scale=2));
+	//   required int64 field_id=-1 Decimal2 (Decimal(precision=18, scale=2));
+	//   required fixed_len_byte_array field_id=-1 Decimal3 (Decimal(precision=10, scale=2));
+	//   required byte_array field_id=-1 Decimal4 (Decimal(precision=20, scale=2));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_repetition() {
+	type RepetitionSchema struct {
+		List     []int64 `parquet:"fieldid=1"`
+		Repeated []int64 `parquet:"repetition=repeated, fieldid=2"`
+		Optional *int64  `parquet:"fieldid=3"`
+		Required *int64  `parquet:"repetition=REQUIRED, fieldid=4"`
+		Opt      int64   `parquet:"repetition=OPTIONAL, fieldid=5"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(RepetitionSchema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 RepetitionSchema {
+	//   required group field_id=1 List (List) {
+	//     repeated group field_id=-1 list {
+	//       required int64 field_id=-1 element (Int(bitWidth=64, isSigned=true));
+	//     }
+	//   }
+	//   repeated int64 field_id=2 Repeated (Int(bitWidth=64, isSigned=true));
+	//   optional int64 field_id=3 Optional (Int(bitWidth=64, isSigned=true));
+	//   required int64 field_id=4 Required (Int(bitWidth=64, isSigned=true));
+	//   optional int64 field_id=5 Opt (Int(bitWidth=64, isSigned=true));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_logicaltypes() {
+	type LogicalTypes struct {
+		String                []byte   `parquet:"logical=String"`
+		Enum                  string   `parquet:"logical=enum"`
+		Date                  int32    `parquet:"logical=date"`
+		Decimal1              int32    `parquet:"logical=decimal, precision=9, scale=2"`
+		Decimal2              int32    `parquet:"logical=decimal, logical.precision=9, scale=2"`
+		Decimal3              int32    `parquet:"logical=decimal, precision=5, logical.precision=9, scale=1, logical.scale=3"`
+		TimeMilliUTC          int32    `parquet:"logical=TIME, logical.unit=millis"`
+		TimeMilli             int32    `parquet:"logical=Time, logical.unit=millis, logical.isadjustedutc=false"`
+		TimeMicros            int64    `parquet:"logical=time, logical.unit=micros, logical.isadjustedutc=false"`
+		TimeMicrosUTC         int64    `parquet:"logical=time, logical.unit=micros, logical.isadjustedutc=true"`
+		TimeNanos             int64    `parquet:"logical=time, logical.unit=nanos"`
+		TimestampMilli        int64    `parquet:"logical=timestamp, logical.unit=millis"`
+		TimestampMicrosNotUTC int64    `parquet:"logical=timestamp, logical.unit=micros, logical.isadjustedutc=false"`
+		TimestampNanos        int64    `parquet:"logical=timestamp, logical.unit=nanos"`
+		JSON                  string   `parquet:"logical=json"`
+		BSON                  []byte   `parquet:"logical=BSON"`
+		UUID                  [16]byte `parquet:"logical=uuid"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(LogicalTypes{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 LogicalTypes {
+	//   required byte_array field_id=-1 String (String);
+	//   required byte_array field_id=-1 Enum (Enum);
+	//   required int32 field_id=-1 Date (Date);
+	//   required int32 field_id=-1 Decimal1 (Decimal(precision=9, scale=2));
+	//   required int32 field_id=-1 Decimal2 (Decimal(precision=9, scale=2));
+	//   required int32 field_id=-1 Decimal3 (Decimal(precision=9, scale=3));
+	//   required int32 field_id=-1 TimeMilliUTC (Time(isAdjustedToUTC=true, timeUnit=milliseconds));
+	//   required int32 field_id=-1 TimeMilli (Time(isAdjustedToUTC=false, timeUnit=milliseconds));
+	//   required int64 field_id=-1 TimeMicros (Time(isAdjustedToUTC=false, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeMicrosUTC (Time(isAdjustedToUTC=true, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeNanos (Time(isAdjustedToUTC=true, timeUnit=nanoseconds));
+	//   required int64 field_id=-1 TimestampMilli (Timestamp(isAdjustedToUTC=true, timeUnit=milliseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimestampMicrosNotUTC (Timestamp(isAdjustedToUTC=false, timeUnit=microseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimestampNanos (Timestamp(isAdjustedToUTC=true, timeUnit=nanoseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required byte_array field_id=-1 JSON (JSON);
+	//   required byte_array field_id=-1 BSON (BSON);
+	//   required fixed_len_byte_array field_id=-1 UUID (UUID);
+	// }
+}
+
+func ExampleNewSchemaFromStruct_physicaltype() {
+	type ChangeTypes struct {
+		Int32        int64  `parquet:"type=int32"`
+		FixedLen     string `parquet:"type=fixed_len_byte_array, length=10"`
+		SliceAsFixed []byte `parquet:"type=fixed_len_byte_array, length=12"`
+		Int          int    `parquet:"type=int32"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(ChangeTypes{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 ChangeTypes {
+	//   required int32 field_id=-1 Int32 (Int(bitWidth=32, isSigned=true));
+	//   required fixed_len_byte_array field_id=-1 FixedLen;
+	//   required fixed_len_byte_array field_id=-1 SliceAsFixed;
+	//   required int32 field_id=-1 Int (Int(bitWidth=32, isSigned=true));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_nestedtypes() {
+	type Other struct {
+		OptionalMap *map[string]*string `parquet:"valuerepetition=required, keylogical=String, valueconverted=BSON"`
+	}
+
+	type MyMap map[int32]string
+
+	type Nested struct {
+		SimpleMap     map[int32]string
+		FixedLenMap   map[string][]byte `parquet:"keytype=fixed_len_byte_array, keyfieldid=10, valuefieldid=11, keylength=10"`
+		DecimalMap    map[int32]string  `parquet:"logical=map, keyconverted=DECIMAL, keyscale=3, keyprecision=7, valuetype=fixed_len_byte_array, valuelength=4, valuelogical=decimal, valuelogical.precision=9, valuescale=2"`
+		OtherList     []*Other
+		OtherRepeated []Other  `parquet:"repetition=repeated"`
+		DateArray     [5]int32 `parquet:"valuelogical=date, logical=list"`
+		DateMap       MyMap    `parquet:"keylogical=TIME, keylogical.unit=MILLIS, keylogical.isadjustedutc=false, valuelogical=enum"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(Nested{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 Nested {

Review comment:
       Ah gotcha. That makes sense.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r617006220



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16

Review comment:
       currently in the c++ code on master at least, they are still called `max_definition_level` and `max_repetition_level` https://github.com/apache/arrow/blob/master/cpp/src/parquet/schema.h#L406 which makes sense. They are the *max* definition and repetition level that exists in the column, since the individual levels would be on a per value basis.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616827828



##########
File path: go/parquet/internal/debug/log_off.go
##########
@@ -0,0 +1,21 @@
+// 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.
+
+// +build !debug
+
+package debug
+
+func Log(interface{}) {}

Review comment:
       this is using the build tags to have a debug logging interface that can be enabled by passing `-tags debug` when building so that they get optimized away in a build without that flag. It's the equivalent of using macros in C++ to check `#ifdef NDEBUG` etc. The go arrow library does the same.
   
   Go has a common library for logging, but it's designed to be used in conjunction with env vars controlling log level output rather than using the build tags to control optimizing away the debug logs like this.

##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16

Review comment:
       sure i'll add those comments.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-823296511


   bumping for some visibility to beg for reviews :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r633684009



##########
File path: go/parquet/schema/node.go
##########
@@ -0,0 +1,625 @@
+// 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.
+
+package schema
+
+import (
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/thrift/lib/go/thrift"
+	"golang.org/x/xerrors"
+)
+
+// NodeType describes whether the Node is a Primitive or Group node
+type NodeType int
+
+// the available constants for NodeType
+const (
+	Primitive NodeType = iota
+	Group
+)
+
+// Node is the interface for both Group and Primitive Nodes.
+// A logical schema type has a name, repetition level, and optionally
+// a logical type (converted type is the deprecated version of the logical
+// type concept, which is maintained for forward compatibility)
+type Node interface {
+	Name() string
+	Type() NodeType
+	RepetitionType() parquet.Repetition
+	ConvertedType() ConvertedType
+	LogicalType() LogicalType
+	FieldID() int32
+	Parent() Node
+	SetParent(Node)
+	Path() string
+	Equals(Node) bool
+	Visit(v Visitor)
+	toThrift() *format.SchemaElement
+}
+
+// Visitor is an interface for creating functionality to walk the schema tree.
+//
+// A visitor can be passed to the Visit function of a Node in order to walk
+// the tree. VisitPre is called the first time a node is encountered. If
+// it is a group node, the return is checked and if it is false, the children
+// will be skipped.
+//
+// VisitPost is called after visiting any children
+type Visitor interface {
+	VisitPre(Node) bool
+	VisitPost(Node)
+}
+
+// ColumnPathFromNode walks the parents of the given node to construct it's
+// column path
+func ColumnPathFromNode(n Node) parquet.ColumnPath {
+	if n == nil {
+		return nil
+	}
+
+	c := make([]string, 0)
+
+	// build the path in reverse order as we traverse nodes to the top
+	cursor := n
+	for cursor.Parent() != nil {
+		c = append(c, cursor.Name())
+		cursor = cursor.Parent()
+	}
+
+	// reverse the order of the list in place so that our result
+	// is in the proper, correct order.
+	for i := len(c)/2 - 1; i >= 0; i-- {

Review comment:
       Nope, there's no built-in reverse for slices. There's a few simple algorithmic things like that which exist in the C++ stdlib but Go doesn't have simply because of the philosophy Go has in trying not to hide complexity where it can. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r633676933



##########
File path: go/parquet/schema/reflection.go
##########
@@ -0,0 +1,827 @@
+// 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.
+
+package schema
+
+import (
+	"reflect"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+type taggedInfo struct {
+	Name string
+
+	Type      parquet.Type
+	KeyType   parquet.Type
+	ValueType parquet.Type
+
+	Length      int32
+	KeyLength   int32
+	ValueLength int32
+
+	Scale      int32
+	KeyScale   int32
+	ValueScale int32
+
+	Precision      int32
+	KeyPrecision   int32
+	ValuePrecision int32
+
+	FieldID      int32
+	KeyFieldID   int32
+	ValueFieldID int32
+
+	RepetitionType  parquet.Repetition
+	ValueRepetition parquet.Repetition
+
+	Converted      ConvertedType
+	KeyConverted   ConvertedType
+	ValueConverted ConvertedType
+
+	LogicalFields      map[string]string
+	KeyLogicalFields   map[string]string
+	ValueLogicalFields map[string]string
+
+	LogicalType      LogicalType
+	KeyLogicalType   LogicalType
+	ValueLogicalType LogicalType
+}
+
+func (t *taggedInfo) CopyForKey() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.KeyType
+	ret.Length = t.KeyLength
+	ret.Scale = t.KeyScale
+	ret.Precision = t.KeyPrecision
+	ret.FieldID = t.KeyFieldID
+	ret.RepetitionType = parquet.Repetitions.Required
+	ret.Converted = t.KeyConverted
+	ret.LogicalType = t.KeyLogicalType
+	return
+}
+
+func (t *taggedInfo) CopyForValue() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.ValueType
+	ret.Length = t.ValueLength
+	ret.Scale = t.ValueScale
+	ret.Precision = t.ValuePrecision
+	ret.FieldID = t.ValueFieldID
+	ret.RepetitionType = t.ValueRepetition
+	ret.Converted = t.ValueConverted
+	ret.LogicalType = t.ValueLogicalType
+	return
+}
+
+func (t *taggedInfo) UpdateLogicalTypes() {
+	processLogicalType := func(fields map[string]string, precision, scale int32) LogicalType {
+		t, ok := fields["type"]
+		if !ok {
+			return NoLogicalType{}
+		}
+
+		switch strings.ToLower(t) {
+		case "string":
+			return StringLogicalType{}
+		case "map":
+			return MapLogicalType{}
+		case "list":
+			return ListLogicalType{}
+		case "enum":
+			return EnumLogicalType{}
+		case "decimal":
+			if v, ok := fields["precision"]; ok {
+				precision = int32FromType(v)
+			}
+			if v, ok := fields["scale"]; ok {
+				scale = int32FromType(v)
+			}
+			return NewDecimalLogicalType(precision, scale)
+		case "date":
+			return DateLogicalType{}
+		case "time":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimeLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(strings.ToLower(unit)))
+		case "timestamp":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimestampLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(unit))
+		case "integer":
+			width, ok := fields["bitwidth"]
+			if !ok {
+				panic("must specify bitwidth if explicitly setting integer logical type")
+			}
+			signed, ok := fields["signed"]
+			if !ok {
+				signed = "true"
+			}
+
+			return NewIntLogicalType(int8(int32FromType(width)), boolFromStr(signed))
+		case "null":
+			return NullLogicalType{}
+		case "json":
+			return JSONLogicalType{}
+		case "bson":
+			return BSONLogicalType{}
+		case "uuid":
+			return UUIDLogicalType{}
+		default:
+			panic(xerrors.Errorf("invalid logical type specified: %s", t))
+		}
+	}
+
+	t.LogicalType = processLogicalType(t.LogicalFields, t.Precision, t.Scale)
+	t.KeyLogicalType = processLogicalType(t.KeyLogicalFields, t.KeyPrecision, t.KeyScale)
+	t.ValueLogicalType = processLogicalType(t.ValueLogicalFields, t.ValuePrecision, t.ValueScale)
+}
+
+func newTaggedInfo() taggedInfo {
+	return taggedInfo{
+		Type:               parquet.Types.Undefined,
+		KeyType:            parquet.Types.Undefined,
+		ValueType:          parquet.Types.Undefined,
+		RepetitionType:     parquet.Repetitions.Undefined,
+		ValueRepetition:    parquet.Repetitions.Undefined,
+		Converted:          ConvertedTypes.NA,
+		KeyConverted:       ConvertedTypes.NA,
+		ValueConverted:     ConvertedTypes.NA,
+		FieldID:            -1,
+		KeyFieldID:         -1,
+		ValueFieldID:       -1,
+		LogicalFields:      make(map[string]string),
+		KeyLogicalFields:   make(map[string]string),
+		ValueLogicalFields: make(map[string]string),
+		LogicalType:        NoLogicalType{},
+		KeyLogicalType:     NoLogicalType{},
+		ValueLogicalType:   NoLogicalType{},
+	}
+}
+
+var int32FromType = func(v string) int32 {
+	val, err := strconv.Atoi(v)
+	if err != nil {
+		panic(err)
+	}
+	return int32(val)
+}
+
+var boolFromStr = func(v string) bool {
+	val, err := strconv.ParseBool(v)
+	if err != nil {
+		panic(err)
+	}
+	return val
+}
+
+func infoFromTags(f reflect.StructTag) *taggedInfo {
+	typeFromStr := func(v string) parquet.Type {
+		t, err := format.TypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(xerrors.Errorf("invalid type specified: %s", v))
+		}
+		return parquet.Type(t)
+	}
+
+	repFromStr := func(v string) parquet.Repetition {
+		r, err := format.FieldRepetitionTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return parquet.Repetition(r)
+	}
+
+	convertedFromStr := func(v string) ConvertedType {
+		c, err := format.ConvertedTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return ConvertedType(c)
+	}
+
+	if ptags, ok := f.Lookup("parquet"); ok {
+		info := newTaggedInfo()
+		for _, tag := range strings.Split(strings.Replace(ptags, "\t", "", -1), ",") {
+			tag = strings.TrimSpace(tag)
+			kv := strings.SplitN(tag, "=", 2)
+			key := strings.TrimSpace(strings.ToLower(kv[0]))
+			value := strings.TrimSpace(kv[1])
+
+			switch key {
+			case "name":
+				info.Name = value
+			case "type":
+				info.Type = typeFromStr(value)
+			case "keytype":
+				info.KeyType = typeFromStr(value)
+			case "valuetype":
+				info.ValueType = typeFromStr(value)
+			case "length":
+				info.Length = int32FromType(value)
+			case "keylength":
+				info.KeyLength = int32FromType(value)
+			case "valuelength":
+				info.ValueLength = int32FromType(value)
+			case "scale":
+				info.Scale = int32FromType(value)
+			case "keyscale":
+				info.KeyScale = int32FromType(value)
+			case "valuescale":
+				info.ValueScale = int32FromType(value)
+			case "precision":
+				info.Precision = int32FromType(value)
+			case "keyprecision":
+				info.KeyPrecision = int32FromType(value)
+			case "valueprecision":
+				info.ValuePrecision = int32FromType(value)
+			case "fieldid":
+				info.FieldID = int32FromType(value)
+			case "keyfieldid":
+				info.KeyFieldID = int32FromType(value)
+			case "valuefieldid":
+				info.ValueFieldID = int32FromType(value)
+			case "repetition":
+				info.RepetitionType = repFromStr(value)
+			case "valuerepetition":
+				info.ValueRepetition = repFromStr(value)
+			case "converted":
+				info.Converted = convertedFromStr(value)
+			case "keyconverted":
+				info.KeyConverted = convertedFromStr(value)
+			case "valueconverted":
+				info.ValueConverted = convertedFromStr(value)
+			case "logical":
+				info.LogicalFields["type"] = value
+			case "keylogical":
+				info.KeyLogicalFields["type"] = value
+			case "valuelogical":
+				info.ValueLogicalFields["type"] = value
+			default:
+				switch {
+				case strings.HasPrefix(key, "logical."):
+					info.LogicalFields[strings.TrimPrefix(key, "logical.")] = value
+				case strings.HasPrefix(key, "keylogical."):
+					info.KeyLogicalFields[strings.TrimPrefix(key, "keylogical.")] = value
+				case strings.HasPrefix(key, "valuelogical."):
+					info.ValueLogicalFields[strings.TrimPrefix(key, "valuelogical.")] = value
+				}
+			}
+		}
+		info.UpdateLogicalTypes()
+		return &info
+	}
+	return nil
+}
+
+// typeToNode recurseively converts a physical type and the tag info into parquet Nodes
+//
+// to avoid having to propagate errors up potentially high numbers of recursive calls
+// we use panics and then recover in the public function NewSchemaFromStruct so that a
+// failure very far down the stack quickly unwinds.
+func typeToNode(name string, typ reflect.Type, repType parquet.Repetition, info *taggedInfo) Node {
+	// set up our default values for everything
+	var (
+		converted             = ConvertedTypes.None
+		logical   LogicalType = NoLogicalType{}
+		fieldID               = int32(-1)
+		physical              = parquet.Types.Undefined
+		typeLen               = 0
+		precision             = 0
+		scale                 = 0
+	)
+	if info != nil { // we have struct tag info to process
+		fieldID = info.FieldID

Review comment:
       Should I modify this to no longer auto-generate the fieldIDs? is there a specific reason why the C++ code won't generate them anymore?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-837560932


   Will do. thanks much


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-840831709


   The integration test failure i believe has nothing to do with this change as far as i can tell


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r631930961



##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval
+	IsSerialized() bool
+	// Returns true if not NoLogicalType
+	IsValid() bool
+	// Returns true if it is NoType
+	IsNone() bool
+	// returns a string representation of the Logical Type
+	String() string
+	toThrift() *format.LogicalType
+	// Return the equivalent ConvertedType for legacy Parquet systems
+	ToConvertedType() (ConvertedType, DecimalMetadata)
+	// Returns true if the specified ConvertedType is compatible with this
+	// logical type
+	IsCompatible(ConvertedType, DecimalMetadata) bool
+	// Returns true if this logical type can be used with the provided physical type
+	IsApplicable(t parquet.Type, tlen int32) bool
+	// Returns true if the logical types are the same
+	Equals(LogicalType) bool
+	// Returns the default stat sort order for this logical type
+	SortOrder() SortOrder
+}
+
+// TemporalLogicalType is a smaller interface for Time based logical types
+// like Time / Timestamp
+type TemporalLogicalType interface {
+	LogicalType
+	IsAdjustedToUTC() bool
+	TimeUnit() TimeUnitType
+}
+
+// SortOrder mirrors the parquet.thrift sort order type
+type SortOrder int8
+
+// Constants for the Stat sort order definitions
+const (
+	SortSIGNED SortOrder = iota
+	SortUNSIGNED
+	SortUNKNOWN
+)
+
+// DefaultSortOrder returns the default stat sort order for the given physical type
+func DefaultSortOrder(primitive format.Type) SortOrder {
+	switch primitive {
+	case format.Type_BOOLEAN, format.Type_INT32, format.Type_INT64, format.Type_FLOAT, format.Type_DOUBLE:
+		return SortSIGNED
+	case format.Type_BYTE_ARRAY, format.Type_FIXED_LEN_BYTE_ARRAY:
+		return SortUNSIGNED
+	case format.Type_INT96:
+		fallthrough
+	default:
+		return SortUNKNOWN
+	}
+}
+
+// GetLogicalSortOrder returns the default sort order for this logical type
+// or falls back to the default sort order for the physical type if not valid
+func GetLogicalSortOrder(logical LogicalType, primitive format.Type) SortOrder {
+	switch {
+	case logical == nil || !logical.IsValid():
+		return SortUNKNOWN
+	case logical.Equals(NoLogicalType{}):
+		return DefaultSortOrder(primitive)
+	default:
+		return logical.SortOrder()
+	}
+}
+
+type baseLogicalType struct{}
+
+func (baseLogicalType) IsSerialized() bool {
+	return true
+}
+
+func (baseLogicalType) IsValid() bool {
+	return true
+}
+
+func (baseLogicalType) IsNested() bool {
+	return false
+}
+
+func (baseLogicalType) IsNone() bool { return false }
+
+// StringLogicalType is a UTF8 string, only usable with ByteArray and FixedLenByteArray
+type StringLogicalType struct{ baseLogicalType }
+
+func (StringLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (StringLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": StringLogicalType{}.String()})
+}
+
+func (StringLogicalType) String() string {
+	return "String"
+}
+
+func (StringLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.UTF8, DecimalMetadata{}
+}
+
+func (StringLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.UTF8 && !dec.IsSet
+}
+
+func (StringLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (StringLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{STRING: format.NewStringType()}
+}
+
+func (StringLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(StringLogicalType)
+	return ok
+}
+
+// MapLogicalType represents a mapped type
+type MapLogicalType struct{ baseLogicalType }
+
+func (MapLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (MapLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": MapLogicalType{}.String()})
+}
+
+func (MapLogicalType) String() string {
+	return "Map"
+}
+
+func (MapLogicalType) IsNested() bool {
+	return true
+}
+
+func (MapLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Map, DecimalMetadata{}
+}
+
+func (MapLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return (t == ConvertedTypes.Map || t == ConvertedTypes.MapKeyValue) && !dec.IsSet
+}
+
+func (MapLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (MapLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{MAP: format.NewMapType()}
+}
+
+func (MapLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(MapLogicalType)
+	return ok
+}
+
+func NewListLogicalType() LogicalType {
+	return ListLogicalType{}
+}
+
+// ListLogicalType is used for columns which are themselves nested lists
+type ListLogicalType struct{ baseLogicalType }
+
+func (ListLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (ListLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": ListLogicalType{}.String()})
+}
+
+func (ListLogicalType) String() string {
+	return "List"
+}
+
+func (ListLogicalType) IsNested() bool {
+	return true
+}
+
+func (ListLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.List, DecimalMetadata{}
+}
+
+func (ListLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.List && !dec.IsSet
+}
+
+func (ListLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (ListLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{LIST: format.NewListType()}
+}
+
+func (ListLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(ListLogicalType)
+	return ok
+}
+
+// EnumLogicalType is for representing an enum, which should be a byte array type
+type EnumLogicalType struct{ baseLogicalType }
+
+func (EnumLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (EnumLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": EnumLogicalType{}.String()})
+}
+
+func (EnumLogicalType) String() string {
+	return "Enum"
+}
+
+func (EnumLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Enum, DecimalMetadata{}
+}
+
+func (EnumLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Enum && !dec.IsSet
+}
+
+func (EnumLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (EnumLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{ENUM: format.NewEnumType()}
+}
+
+func (EnumLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(EnumLogicalType)
+	return ok
+}
+
+// NewDecimalLogicalType returns a Decimal logical type with the given
+// precision and scale.
+//
+// Panics if precision < 1 or scale is not in the range (0, precision)
+func NewDecimalLogicalType(precision int32, scale int32) LogicalType {
+	if precision < 1 {
+		panic("parquet: precision must be greater than or equal to 1 for decimal logical type")
+	}
+	if scale < 0 || scale > precision {
+		panic("parquet: scale must be a non-negative integer that does not exceed precision for decimal logical type")
+	}
+	return &DecimalLogicalType{typ: &format.DecimalType{Precision: precision, Scale: scale}}
+}
+
+// DecimalLogicalType is used to represent a decimal value of a given
+// precision and scale
+type DecimalLogicalType struct {
+	baseLogicalType
+	typ *format.DecimalType
+}
+
+func (t DecimalLogicalType) Precision() int32 {
+	return t.typ.Precision
+}
+
+func (t DecimalLogicalType) Scale() int32 {
+	return t.typ.Scale
+}
+
+func (DecimalLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t DecimalLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{"Type": "Decimal", "precision": t.typ.Precision, "scale": t.typ.Scale})
+}
+
+func (t DecimalLogicalType) String() string {
+	return fmt.Sprintf("Decimal(precision=%d, scale=%d)", t.typ.Precision, t.typ.Scale)
+}
+
+func (t DecimalLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Decimal, DecimalMetadata{IsSet: true, Scale: t.typ.GetScale(), Precision: t.typ.GetPrecision()}
+}
+
+func (t DecimalLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.Decimal &&
+		dec.IsSet && dec.Scale == t.typ.Scale && dec.Precision == t.typ.Precision
+}
+
+func (t DecimalLogicalType) IsApplicable(typ parquet.Type, tlen int32) bool {
+	switch typ {
+	case parquet.Types.Int32:
+		return 1 <= t.typ.Precision && t.typ.Precision <= 9
+	case parquet.Types.Int64:
+		if t.typ.Precision < 10 {
+			debug.Log("int64 used for decimal logical, precision is small enough to use int32")
+		}
+		return 1 <= t.typ.Precision && t.typ.Precision <= 18
+	case parquet.Types.FixedLenByteArray:
+		return t.typ.Precision <= int32(math.Floor(math.Log10(math.Pow(2.0, (8.0*float64(tlen)-1.0)))))
+	case parquet.Types.ByteArray:
+		return true
+	}
+	return false
+}
+
+func (t DecimalLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DECIMAL: t.typ}
+}
+
+func (t DecimalLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*DecimalLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.Precision == other.typ.Precision && t.typ.Scale == other.typ.Scale
+}
+
+// DateLogicalType is an int32 representing the number of days since the Unix Epoch
+// 1 January 1970
+type DateLogicalType struct{ baseLogicalType }
+
+func (DateLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (DateLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": DateLogicalType{}.String()})
+}
+
+func (DateLogicalType) String() string {
+	return "Date"
+}
+
+func (DateLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Date, DecimalMetadata{}
+}
+
+func (DateLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Date && !dec.IsSet
+}
+
+func (DateLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int32
+}
+
+func (DateLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DATE: format.NewDateType()}
+}
+
+func (DateLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(DateLogicalType)
+	return ok
+}
+
+func timeUnitFromThrift(unit *format.TimeUnit) TimeUnitType {
+	switch {
+	case unit == nil:
+		return TimeUnitUnknown
+	case unit.IsSetMILLIS():
+		return TimeUnitMillis
+	case unit.IsSetMICROS():
+		return TimeUnitMicros
+	case unit.IsSetNANOS():
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func timeUnitToString(unit *format.TimeUnit) string {
+	switch {
+	case unit == nil:
+		return "unknown"
+	case unit.IsSetMILLIS():
+		return "milliseconds"
+	case unit.IsSetMICROS():
+		return "microseconds"
+	case unit.IsSetNANOS():
+		return "nanoseconds"
+	default:
+		return "unknown"
+	}
+}
+
+func timeUnitFromString(v string) TimeUnitType {
+	switch v {
+	case "millis":
+		return TimeUnitMillis
+	case "micros":
+		return TimeUnitMicros
+	case "nanos":
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func createTimeUnit(unit TimeUnitType) *format.TimeUnit {
+	tunit := format.NewTimeUnit()
+	switch unit {
+	case TimeUnitMicros:
+		tunit.MICROS = format.NewMicroSeconds()
+	case TimeUnitMillis:
+		tunit.MILLIS = format.NewMilliSeconds()
+	case TimeUnitNanos:
+		tunit.NANOS = format.NewNanoSeconds()
+	default:
+		panic("parquet: time unit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+	}
+	return tunit
+}
+
+// NewTimeLogicalType returns a time type of the given unit.
+func NewTimeLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimeLogicalType{typ: &format.TimeType{
+		IsAdjustedToUTC: isAdjustedToUTC,
+		Unit:            createTimeUnit(unit),
+	}}
+}
+
+// TimeLogicalType is a time type without a date and must be an
+// int32 for milliseconds, or an int64 for micro or nano seconds.
+type TimeLogicalType struct {
+	baseLogicalType
+	typ *format.TimeType
+}
+
+func (t TimeLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimeLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimeLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimeLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Time", "isAdjustedToUTC": t.typ.IsAdjustedToUTC, "timeUnit": timeUnitToString(t.typ.GetUnit())})
+}
+
+func (t TimeLogicalType) String() string {
+	return fmt.Sprintf("Time(isAdjustedToUTC=%t, timeUnit=%s)", t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()))
+}
+
+func (t TimeLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimeMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimeMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimeLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return c == ConvertedTypes.TimeMillis
+		case TimeUnitMicros:
+			return c == ConvertedTypes.TimeMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (t TimeLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetUnit().IsSetMILLIS()) ||
+		(typ == parquet.Types.Int64 &&
+			(t.typ.GetUnit().IsSetMICROS() || t.typ.GetUnit().IsSetNANOS()))
+}
+
+func (t TimeLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIME: t.typ}
+}
+
+func (t TimeLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimeLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewTimestampLogicalType returns a logical timestamp type with "forceConverted"
+// set to false
+func NewTimestampLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: false,
+		fromConverted:  false,
+	}
+}
+
+// NewTimestampLogicalTypeForce returns a timestamp logical type with
+// "forceConverted" set to true
+func NewTimestampLogicalTypeForce(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: true,
+		fromConverted:  false,
+	}
+}
+
+// TimestampLogicalType represents an int64 number that can be decoded
+// into a year, month, day, hour, minute, second, and subsecond
+type TimestampLogicalType struct {
+	baseLogicalType
+	typ            *format.TimestampType
+	forceConverted bool
+	fromConverted  bool
+}
+
+func (t TimestampLogicalType) IsFromConvertedType() bool {
+	return t.fromConverted
+}
+
+func (t TimestampLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimestampLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimestampLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimestampLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type":                     "Timestamp",
+		"isAdjustedToUTC":          t.typ.IsAdjustedToUTC,
+		"timeUnit":                 timeUnitToString(t.typ.GetUnit()),
+		"is_from_converted_type":   t.fromConverted,
+		"force_set_converted_type": t.forceConverted,
+	})
+}
+
+func (t TimestampLogicalType) IsSerialized() bool {
+	return !t.fromConverted
+}
+
+func (t TimestampLogicalType) String() string {
+	return fmt.Sprintf("Timestamp(isAdjustedToUTC=%t, timeUnit=%s, is_from_converted_type=%t, force_set_converted_type=%t)",
+		t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()), t.fromConverted, t.forceConverted)
+}
+
+func (t TimestampLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC || t.forceConverted {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimestampMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimestampMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimestampLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+
+	switch timeUnitFromThrift(t.typ.Unit) {
+	case TimeUnitMillis:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMillis
+		}
+	case TimeUnitMicros:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (TimestampLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int64
+}
+
+func (t TimestampLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIMESTAMP: t.typ}
+}
+
+func (t TimestampLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimestampLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewIntLogicalType creates an integer logical type of the desired bitwidth
+// and whether it is signed or not.
+//
+// Bit width must be exactly 8, 16, 32 or 64 for an integer logical type
+func NewIntLogicalType(bitWidth int8, signed bool) LogicalType {
+	switch bitWidth {
+	case 8, 16, 32, 64:
+	default:
+		panic("parquet: bit width must be exactly 8, 16, 32, or 64 for Int logical type")
+	}
+	return &IntLogicalType{
+		typ: &format.IntType{
+			BitWidth: bitWidth,
+			IsSigned: signed,
+		},
+	}
+}
+
+// IntLogicalType represents an integer type of a specific bit width and
+// is either signed or unsigned.
+type IntLogicalType struct {
+	baseLogicalType
+	typ *format.IntType
+}
+
+func (t IntLogicalType) BitWidth() int8 {
+	return t.typ.BitWidth
+}
+
+func (t IntLogicalType) IsSigned() bool {
+	return t.typ.IsSigned
+}
+
+func (t IntLogicalType) SortOrder() SortOrder {
+	if t.typ.IsSigned {
+		return SortSIGNED
+	}
+	return SortUNSIGNED
+}
+
+func (t IntLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Int", "bitWidth": t.typ.BitWidth, "isSigned": t.typ.IsSigned,
+	})
+}
+
+func (t IntLogicalType) String() string {
+	return fmt.Sprintf("Int(bitWidth=%d, isSigned=%t)", t.typ.GetBitWidth(), t.typ.GetIsSigned())
+}
+
+func (t IntLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	var d DecimalMetadata
+	if t.typ.IsSigned {
+		switch t.typ.BitWidth {
+		case 8:
+			return ConvertedTypes.Int8, d
+		case 16:
+			return ConvertedTypes.Int16, d
+		case 32:
+			return ConvertedTypes.Int32, d
+		case 64:
+			return ConvertedTypes.Int64, d
+		}
+	} else {
+		switch t.typ.BitWidth {
+		case 8:
+			return ConvertedTypes.Uint8, d
+		case 16:
+			return ConvertedTypes.Uint16, d
+		case 32:
+			return ConvertedTypes.Uint32, d
+		case 64:
+			return ConvertedTypes.Uint64, d
+		}
+	}
+	return ConvertedTypes.None, d
+}
+
+func (t IntLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	v, _ := t.ToConvertedType()
+	return c == v
+}
+
+func (t IntLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetBitWidth() <= 32) ||
+		(typ == parquet.Types.Int64 && t.typ.GetBitWidth() == 64)
+}
+
+func (t IntLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{INTEGER: t.typ}
+}
+
+func (t IntLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*IntLogicalType)
+	if !ok {
+		return false
+	}
+
+	return t.typ.GetIsSigned() == other.typ.GetIsSigned() &&
+		t.typ.GetBitWidth() == other.typ.GetBitWidth()
+}
+
+// UnknownLogicalType is a type that is essentially a placeholder for when
+// we don't know the type.
+type UnknownLogicalType struct{ baseLogicalType }
+
+func (UnknownLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (UnknownLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": UnknownLogicalType{}.String()})
+}
+
+func (UnknownLogicalType) IsValid() bool { return false }
+
+func (UnknownLogicalType) IsSerialized() bool { return false }
+
+func (UnknownLogicalType) String() string {
+	return "Unknown"
+}
+
+func (UnknownLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.NA, DecimalMetadata{}
+}
+
+func (UnknownLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.NA && !dec.IsSet
+}
+
+func (UnknownLogicalType) IsApplicable(parquet.Type, int32) bool { return true }
+
+func (UnknownLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{UNKNOWN: format.NewNullType()}
+}
+
+func (UnknownLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(UnknownLogicalType)
+	return ok
+}
+
+// JSONLogicalType represents a byte array column which is to be interpreted
+// as a JSON string.
+type JSONLogicalType struct{ baseLogicalType }
+
+func (JSONLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (JSONLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": JSONLogicalType{}.String()})
+}
+
+func (JSONLogicalType) String() string {
+	return "JSON"
+}
+
+func (JSONLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.JSON, DecimalMetadata{}
+}
+
+func (JSONLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.JSON && !dec.IsSet
+}
+
+func (JSONLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (JSONLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{JSON: format.NewJsonType()}
+}
+
+func (JSONLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(JSONLogicalType)
+	return ok
+}
+
+// BSONLogicalType represents a binary JSON string in the byte array
+type BSONLogicalType struct{ baseLogicalType }
+
+func (BSONLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (BSONLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": BSONLogicalType{}.String()})
+}
+
+func (BSONLogicalType) String() string {
+	return "BSON"
+}
+
+func (BSONLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.BSON, DecimalMetadata{}
+}
+
+func (BSONLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.BSON && !dec.IsSet
+}
+
+func (BSONLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (BSONLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{BSON: format.NewBsonType()}
+}
+
+func (BSONLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(BSONLogicalType)
+	return ok
+}
+
+// UUIDLogicalType can only be used with a FixedLength byte array column
+// that is exactly 16 bytes long
+type UUIDLogicalType struct{ baseLogicalType }
+
+func (UUIDLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (UUIDLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": UUIDLogicalType{}.String()})
+}
+
+func (UUIDLogicalType) String() string {
+	return "UUID"
+}
+
+func (UUIDLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (UUIDLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	switch c {
+	case ConvertedTypes.None, ConvertedTypes.NA:
+		return true
+	}
+	return false
+}
+
+func (UUIDLogicalType) IsApplicable(t parquet.Type, tlen int32) bool {
+	return t == parquet.Types.FixedLenByteArray && tlen == 16
+}
+
+func (UUIDLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{UUID: format.NewUUIDType()}
+}
+
+func (UUIDLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(UUIDLogicalType)
+	return ok
+}
+
+// IntervalLogicalType is not yet in the thrift spec, but represents
+// an interval time and needs to be a fixed length byte array of 12 bytes
+type IntervalLogicalType struct{ baseLogicalType }
+
+func (IntervalLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN

Review comment:
       i believe the parquet spec has this sort order unsigned (which doesn't really make sense to me)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-840670430


   @zeroshade left some comments but still going through this, will try to finish this evening.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r634028454



##########
File path: go/parquet/schema/reflection.go
##########
@@ -0,0 +1,827 @@
+// 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.
+
+package schema
+
+import (
+	"reflect"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+type taggedInfo struct {
+	Name string
+
+	Type      parquet.Type
+	KeyType   parquet.Type
+	ValueType parquet.Type
+
+	Length      int32
+	KeyLength   int32
+	ValueLength int32
+
+	Scale      int32
+	KeyScale   int32
+	ValueScale int32
+
+	Precision      int32
+	KeyPrecision   int32
+	ValuePrecision int32
+
+	FieldID      int32
+	KeyFieldID   int32
+	ValueFieldID int32
+
+	RepetitionType  parquet.Repetition
+	ValueRepetition parquet.Repetition
+
+	Converted      ConvertedType
+	KeyConverted   ConvertedType
+	ValueConverted ConvertedType
+
+	LogicalFields      map[string]string
+	KeyLogicalFields   map[string]string
+	ValueLogicalFields map[string]string
+
+	LogicalType      LogicalType
+	KeyLogicalType   LogicalType
+	ValueLogicalType LogicalType
+}
+
+func (t *taggedInfo) CopyForKey() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.KeyType
+	ret.Length = t.KeyLength
+	ret.Scale = t.KeyScale
+	ret.Precision = t.KeyPrecision
+	ret.FieldID = t.KeyFieldID
+	ret.RepetitionType = parquet.Repetitions.Required
+	ret.Converted = t.KeyConverted
+	ret.LogicalType = t.KeyLogicalType
+	return
+}
+
+func (t *taggedInfo) CopyForValue() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.ValueType
+	ret.Length = t.ValueLength
+	ret.Scale = t.ValueScale
+	ret.Precision = t.ValuePrecision
+	ret.FieldID = t.ValueFieldID
+	ret.RepetitionType = t.ValueRepetition
+	ret.Converted = t.ValueConverted
+	ret.LogicalType = t.ValueLogicalType
+	return
+}
+
+func (t *taggedInfo) UpdateLogicalTypes() {
+	processLogicalType := func(fields map[string]string, precision, scale int32) LogicalType {
+		t, ok := fields["type"]
+		if !ok {
+			return NoLogicalType{}
+		}
+
+		switch strings.ToLower(t) {
+		case "string":
+			return StringLogicalType{}
+		case "map":
+			return MapLogicalType{}
+		case "list":
+			return ListLogicalType{}
+		case "enum":
+			return EnumLogicalType{}
+		case "decimal":
+			if v, ok := fields["precision"]; ok {
+				precision = int32FromType(v)
+			}
+			if v, ok := fields["scale"]; ok {
+				scale = int32FromType(v)
+			}
+			return NewDecimalLogicalType(precision, scale)
+		case "date":
+			return DateLogicalType{}
+		case "time":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimeLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(strings.ToLower(unit)))
+		case "timestamp":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimestampLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(unit))
+		case "integer":
+			width, ok := fields["bitwidth"]
+			if !ok {
+				panic("must specify bitwidth if explicitly setting integer logical type")
+			}
+			signed, ok := fields["signed"]
+			if !ok {
+				signed = "true"
+			}
+
+			return NewIntLogicalType(int8(int32FromType(width)), boolFromStr(signed))
+		case "null":
+			return NullLogicalType{}
+		case "json":
+			return JSONLogicalType{}
+		case "bson":
+			return BSONLogicalType{}
+		case "uuid":
+			return UUIDLogicalType{}
+		default:
+			panic(xerrors.Errorf("invalid logical type specified: %s", t))
+		}
+	}
+
+	t.LogicalType = processLogicalType(t.LogicalFields, t.Precision, t.Scale)
+	t.KeyLogicalType = processLogicalType(t.KeyLogicalFields, t.KeyPrecision, t.KeyScale)
+	t.ValueLogicalType = processLogicalType(t.ValueLogicalFields, t.ValuePrecision, t.ValueScale)
+}
+
+func newTaggedInfo() taggedInfo {
+	return taggedInfo{
+		Type:               parquet.Types.Undefined,
+		KeyType:            parquet.Types.Undefined,
+		ValueType:          parquet.Types.Undefined,
+		RepetitionType:     parquet.Repetitions.Undefined,
+		ValueRepetition:    parquet.Repetitions.Undefined,
+		Converted:          ConvertedTypes.NA,
+		KeyConverted:       ConvertedTypes.NA,
+		ValueConverted:     ConvertedTypes.NA,
+		FieldID:            -1,
+		KeyFieldID:         -1,
+		ValueFieldID:       -1,
+		LogicalFields:      make(map[string]string),
+		KeyLogicalFields:   make(map[string]string),
+		ValueLogicalFields: make(map[string]string),
+		LogicalType:        NoLogicalType{},
+		KeyLogicalType:     NoLogicalType{},
+		ValueLogicalType:   NoLogicalType{},
+	}
+}
+
+var int32FromType = func(v string) int32 {
+	val, err := strconv.Atoi(v)
+	if err != nil {
+		panic(err)
+	}
+	return int32(val)
+}
+
+var boolFromStr = func(v string) bool {
+	val, err := strconv.ParseBool(v)
+	if err != nil {
+		panic(err)
+	}
+	return val
+}
+
+func infoFromTags(f reflect.StructTag) *taggedInfo {
+	typeFromStr := func(v string) parquet.Type {
+		t, err := format.TypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(xerrors.Errorf("invalid type specified: %s", v))
+		}
+		return parquet.Type(t)
+	}
+
+	repFromStr := func(v string) parquet.Repetition {
+		r, err := format.FieldRepetitionTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return parquet.Repetition(r)
+	}
+
+	convertedFromStr := func(v string) ConvertedType {
+		c, err := format.ConvertedTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return ConvertedType(c)
+	}
+
+	if ptags, ok := f.Lookup("parquet"); ok {
+		info := newTaggedInfo()
+		for _, tag := range strings.Split(strings.Replace(ptags, "\t", "", -1), ",") {
+			tag = strings.TrimSpace(tag)
+			kv := strings.SplitN(tag, "=", 2)
+			key := strings.TrimSpace(strings.ToLower(kv[0]))
+			value := strings.TrimSpace(kv[1])
+
+			switch key {
+			case "name":
+				info.Name = value
+			case "type":
+				info.Type = typeFromStr(value)
+			case "keytype":
+				info.KeyType = typeFromStr(value)
+			case "valuetype":
+				info.ValueType = typeFromStr(value)
+			case "length":
+				info.Length = int32FromType(value)
+			case "keylength":
+				info.KeyLength = int32FromType(value)
+			case "valuelength":
+				info.ValueLength = int32FromType(value)
+			case "scale":
+				info.Scale = int32FromType(value)
+			case "keyscale":
+				info.KeyScale = int32FromType(value)
+			case "valuescale":
+				info.ValueScale = int32FromType(value)
+			case "precision":
+				info.Precision = int32FromType(value)
+			case "keyprecision":
+				info.KeyPrecision = int32FromType(value)
+			case "valueprecision":
+				info.ValuePrecision = int32FromType(value)
+			case "fieldid":
+				info.FieldID = int32FromType(value)
+			case "keyfieldid":
+				info.KeyFieldID = int32FromType(value)
+			case "valuefieldid":
+				info.ValueFieldID = int32FromType(value)
+			case "repetition":
+				info.RepetitionType = repFromStr(value)
+			case "valuerepetition":
+				info.ValueRepetition = repFromStr(value)
+			case "converted":
+				info.Converted = convertedFromStr(value)
+			case "keyconverted":
+				info.KeyConverted = convertedFromStr(value)
+			case "valueconverted":
+				info.ValueConverted = convertedFromStr(value)
+			case "logical":
+				info.LogicalFields["type"] = value
+			case "keylogical":
+				info.KeyLogicalFields["type"] = value
+			case "valuelogical":
+				info.ValueLogicalFields["type"] = value
+			default:
+				switch {
+				case strings.HasPrefix(key, "logical."):
+					info.LogicalFields[strings.TrimPrefix(key, "logical.")] = value
+				case strings.HasPrefix(key, "keylogical."):
+					info.KeyLogicalFields[strings.TrimPrefix(key, "keylogical.")] = value
+				case strings.HasPrefix(key, "valuelogical."):
+					info.ValueLogicalFields[strings.TrimPrefix(key, "valuelogical.")] = value
+				}
+			}
+		}
+		info.UpdateLogicalTypes()
+		return &info
+	}
+	return nil
+}
+
+// typeToNode recurseively converts a physical type and the tag info into parquet Nodes
+//
+// to avoid having to propagate errors up potentially high numbers of recursive calls
+// we use panics and then recover in the public function NewSchemaFromStruct so that a
+// failure very far down the stack quickly unwinds.
+func typeToNode(name string, typ reflect.Type, repType parquet.Repetition, info *taggedInfo) Node {
+	// set up our default values for everything
+	var (
+		converted             = ConvertedTypes.None
+		logical   LogicalType = NoLogicalType{}
+		fieldID               = int32(-1)
+		physical              = parquet.Types.Undefined
+		typeLen               = 0
+		precision             = 0
+		scale                 = 0
+	)
+	if info != nil { // we have struct tag info to process
+		fieldID = info.FieldID

Review comment:
       See PR #10289 for details, essentially the  field IDs in thrift are mean for other system to set for schema evolution/conversion purposes.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616825788



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {
+			return true
+		}
+		n = n.Parent()
+	}
+	return false
+}
+
+// NewColumn returns a new column object for the given node with the provided
+// maximum definition and repetition levels.
+//
+// n MUST be a PrimitiveNode, otherwise this will panic.
+func NewColumn(n Node, maxDefinitionLvl, maxRepetitionLvl int16) *Column {
+	if n.Type() != Primitive {
+		panic("parquet: Column must be a primitive type")
+	}
+	return &Column{n.(*PrimitiveNode), maxDefinitionLvl, maxRepetitionLvl}
+}
+
+// Name is the column's name
+func (c *Column) Name() string { return c.pnode.Name() }
+
+// ColumnPath returns the full path to this column from the root of the schema
+func (c *Column) ColumnPath() parquet.ColumnPath { return c.pnode.columnPath() }
+
+// Path is equivalent to ColumnPath().String() returning the dot-string version of the path
+func (c *Column) Path() string { return c.pnode.Path() }
+
+// TypeLength is -1 if not a FixedLenByteArray, otherwise it is the length for the column
+func (c *Column) TypeLength() int { return c.pnode.TypeLength() }
+
+func (c *Column) MaxDefinitionLevel() int16        { return c.maxDefLvl }
+func (c *Column) MaxRepetitionLevel() int16        { return c.maxRepLvl }
+func (c *Column) PhysicalType() parquet.Type       { return c.pnode.PhysicalType() }
+func (c *Column) ConvertedType() ConvertedType     { return c.pnode.convertedType }
+func (c *Column) LogicalType() LogicalType         { return c.pnode.logicalType }
+func (c *Column) ColumnOrder() parquet.ColumnOrder { return c.pnode.ColumnOrder }
+func (c *Column) String() string {

Review comment:
       i'm surprised with Go's love of abreviations the String method is actually a full word :)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616853211



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {

Review comment:
       not sure if you saw comment above.  I think this is right since there is no repeated fields here.  I don't think this affects correctness but I think the check above isn't quite correct here or in C++.  really we need to walk the tree in both cases. Until either the top or a repeated field is hit




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616828894



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {
+			return true
+		}
+		n = n.Parent()
+	}
+	return false
+}
+
+// NewColumn returns a new column object for the given node with the provided
+// maximum definition and repetition levels.
+//
+// n MUST be a PrimitiveNode, otherwise this will panic.
+func NewColumn(n Node, maxDefinitionLvl, maxRepetitionLvl int16) *Column {
+	if n.Type() != Primitive {
+		panic("parquet: Column must be a primitive type")

Review comment:
       don't see why not, i'll add that to the message.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r633656815



##########
File path: go/parquet/schema/reflection.go
##########
@@ -0,0 +1,827 @@
+// 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.
+
+package schema
+
+import (
+	"reflect"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+type taggedInfo struct {
+	Name string
+
+	Type      parquet.Type
+	KeyType   parquet.Type
+	ValueType parquet.Type
+
+	Length      int32
+	KeyLength   int32
+	ValueLength int32
+
+	Scale      int32
+	KeyScale   int32
+	ValueScale int32
+
+	Precision      int32
+	KeyPrecision   int32
+	ValuePrecision int32
+
+	FieldID      int32
+	KeyFieldID   int32
+	ValueFieldID int32
+
+	RepetitionType  parquet.Repetition
+	ValueRepetition parquet.Repetition
+
+	Converted      ConvertedType
+	KeyConverted   ConvertedType
+	ValueConverted ConvertedType
+
+	LogicalFields      map[string]string
+	KeyLogicalFields   map[string]string
+	ValueLogicalFields map[string]string
+
+	LogicalType      LogicalType
+	KeyLogicalType   LogicalType
+	ValueLogicalType LogicalType
+}
+
+func (t *taggedInfo) CopyForKey() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.KeyType
+	ret.Length = t.KeyLength
+	ret.Scale = t.KeyScale
+	ret.Precision = t.KeyPrecision
+	ret.FieldID = t.KeyFieldID
+	ret.RepetitionType = parquet.Repetitions.Required
+	ret.Converted = t.KeyConverted
+	ret.LogicalType = t.KeyLogicalType
+	return
+}
+
+func (t *taggedInfo) CopyForValue() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.ValueType
+	ret.Length = t.ValueLength
+	ret.Scale = t.ValueScale
+	ret.Precision = t.ValuePrecision
+	ret.FieldID = t.ValueFieldID
+	ret.RepetitionType = t.ValueRepetition
+	ret.Converted = t.ValueConverted
+	ret.LogicalType = t.ValueLogicalType
+	return
+}
+
+func (t *taggedInfo) UpdateLogicalTypes() {
+	processLogicalType := func(fields map[string]string, precision, scale int32) LogicalType {
+		t, ok := fields["type"]
+		if !ok {
+			return NoLogicalType{}
+		}
+
+		switch strings.ToLower(t) {
+		case "string":
+			return StringLogicalType{}
+		case "map":
+			return MapLogicalType{}
+		case "list":
+			return ListLogicalType{}
+		case "enum":
+			return EnumLogicalType{}
+		case "decimal":
+			if v, ok := fields["precision"]; ok {
+				precision = int32FromType(v)
+			}
+			if v, ok := fields["scale"]; ok {
+				scale = int32FromType(v)
+			}
+			return NewDecimalLogicalType(precision, scale)
+		case "date":
+			return DateLogicalType{}
+		case "time":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimeLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(strings.ToLower(unit)))
+		case "timestamp":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimestampLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(unit))
+		case "integer":
+			width, ok := fields["bitwidth"]
+			if !ok {
+				panic("must specify bitwidth if explicitly setting integer logical type")
+			}
+			signed, ok := fields["signed"]
+			if !ok {
+				signed = "true"
+			}
+
+			return NewIntLogicalType(int8(int32FromType(width)), boolFromStr(signed))
+		case "null":
+			return NullLogicalType{}
+		case "json":
+			return JSONLogicalType{}
+		case "bson":
+			return BSONLogicalType{}
+		case "uuid":
+			return UUIDLogicalType{}
+		default:
+			panic(xerrors.Errorf("invalid logical type specified: %s", t))
+		}
+	}
+
+	t.LogicalType = processLogicalType(t.LogicalFields, t.Precision, t.Scale)
+	t.KeyLogicalType = processLogicalType(t.KeyLogicalFields, t.KeyPrecision, t.KeyScale)
+	t.ValueLogicalType = processLogicalType(t.ValueLogicalFields, t.ValuePrecision, t.ValueScale)
+}
+
+func newTaggedInfo() taggedInfo {
+	return taggedInfo{
+		Type:               parquet.Types.Undefined,
+		KeyType:            parquet.Types.Undefined,
+		ValueType:          parquet.Types.Undefined,
+		RepetitionType:     parquet.Repetitions.Undefined,
+		ValueRepetition:    parquet.Repetitions.Undefined,
+		Converted:          ConvertedTypes.NA,
+		KeyConverted:       ConvertedTypes.NA,
+		ValueConverted:     ConvertedTypes.NA,
+		FieldID:            -1,
+		KeyFieldID:         -1,
+		ValueFieldID:       -1,
+		LogicalFields:      make(map[string]string),
+		KeyLogicalFields:   make(map[string]string),
+		ValueLogicalFields: make(map[string]string),
+		LogicalType:        NoLogicalType{},
+		KeyLogicalType:     NoLogicalType{},
+		ValueLogicalType:   NoLogicalType{},
+	}
+}
+
+var int32FromType = func(v string) int32 {
+	val, err := strconv.Atoi(v)
+	if err != nil {
+		panic(err)
+	}
+	return int32(val)
+}
+
+var boolFromStr = func(v string) bool {
+	val, err := strconv.ParseBool(v)
+	if err != nil {
+		panic(err)
+	}
+	return val
+}
+
+func infoFromTags(f reflect.StructTag) *taggedInfo {
+	typeFromStr := func(v string) parquet.Type {
+		t, err := format.TypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(xerrors.Errorf("invalid type specified: %s", v))
+		}
+		return parquet.Type(t)
+	}
+
+	repFromStr := func(v string) parquet.Repetition {
+		r, err := format.FieldRepetitionTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return parquet.Repetition(r)
+	}
+
+	convertedFromStr := func(v string) ConvertedType {
+		c, err := format.ConvertedTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return ConvertedType(c)
+	}
+
+	if ptags, ok := f.Lookup("parquet"); ok {
+		info := newTaggedInfo()
+		for _, tag := range strings.Split(strings.Replace(ptags, "\t", "", -1), ",") {
+			tag = strings.TrimSpace(tag)
+			kv := strings.SplitN(tag, "=", 2)
+			key := strings.TrimSpace(strings.ToLower(kv[0]))
+			value := strings.TrimSpace(kv[1])
+
+			switch key {
+			case "name":
+				info.Name = value
+			case "type":
+				info.Type = typeFromStr(value)
+			case "keytype":
+				info.KeyType = typeFromStr(value)
+			case "valuetype":
+				info.ValueType = typeFromStr(value)
+			case "length":
+				info.Length = int32FromType(value)
+			case "keylength":
+				info.KeyLength = int32FromType(value)
+			case "valuelength":
+				info.ValueLength = int32FromType(value)
+			case "scale":
+				info.Scale = int32FromType(value)
+			case "keyscale":
+				info.KeyScale = int32FromType(value)
+			case "valuescale":
+				info.ValueScale = int32FromType(value)
+			case "precision":
+				info.Precision = int32FromType(value)
+			case "keyprecision":
+				info.KeyPrecision = int32FromType(value)
+			case "valueprecision":
+				info.ValuePrecision = int32FromType(value)
+			case "fieldid":
+				info.FieldID = int32FromType(value)
+			case "keyfieldid":
+				info.KeyFieldID = int32FromType(value)
+			case "valuefieldid":
+				info.ValueFieldID = int32FromType(value)
+			case "repetition":
+				info.RepetitionType = repFromStr(value)
+			case "valuerepetition":
+				info.ValueRepetition = repFromStr(value)
+			case "converted":
+				info.Converted = convertedFromStr(value)
+			case "keyconverted":
+				info.KeyConverted = convertedFromStr(value)
+			case "valueconverted":
+				info.ValueConverted = convertedFromStr(value)
+			case "logical":
+				info.LogicalFields["type"] = value
+			case "keylogical":
+				info.KeyLogicalFields["type"] = value
+			case "valuelogical":
+				info.ValueLogicalFields["type"] = value
+			default:
+				switch {
+				case strings.HasPrefix(key, "logical."):
+					info.LogicalFields[strings.TrimPrefix(key, "logical.")] = value
+				case strings.HasPrefix(key, "keylogical."):
+					info.KeyLogicalFields[strings.TrimPrefix(key, "keylogical.")] = value
+				case strings.HasPrefix(key, "valuelogical."):
+					info.ValueLogicalFields[strings.TrimPrefix(key, "valuelogical.")] = value
+				}
+			}
+		}
+		info.UpdateLogicalTypes()
+		return &info
+	}
+	return nil
+}
+
+// typeToNode recurseively converts a physical type and the tag info into parquet Nodes
+//
+// to avoid having to propagate errors up potentially high numbers of recursive calls
+// we use panics and then recover in the public function NewSchemaFromStruct so that a
+// failure very far down the stack quickly unwinds.
+func typeToNode(name string, typ reflect.Type, repType parquet.Repetition, info *taggedInfo) Node {
+	// set up our default values for everything
+	var (
+		converted             = ConvertedTypes.None
+		logical   LogicalType = NoLogicalType{}
+		fieldID               = int32(-1)
+		physical              = parquet.Types.Undefined
+		typeLen               = 0
+		precision             = 0
+		scale                 = 0
+	)
+	if info != nil { // we have struct tag info to process
+		fieldID = info.FieldID

Review comment:
       just a note there has been a recent issue/PR that is changing the logic around fieldID (c++ code will not generate them any more)

##########
File path: go/parquet/schema/node.go
##########
@@ -0,0 +1,625 @@
+// 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.
+
+package schema
+
+import (
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/thrift/lib/go/thrift"
+	"golang.org/x/xerrors"
+)
+
+// NodeType describes whether the Node is a Primitive or Group node
+type NodeType int
+
+// the available constants for NodeType
+const (
+	Primitive NodeType = iota
+	Group
+)
+
+// Node is the interface for both Group and Primitive Nodes.
+// A logical schema type has a name, repetition level, and optionally
+// a logical type (converted type is the deprecated version of the logical
+// type concept, which is maintained for forward compatibility)
+type Node interface {
+	Name() string
+	Type() NodeType
+	RepetitionType() parquet.Repetition
+	ConvertedType() ConvertedType
+	LogicalType() LogicalType
+	FieldID() int32
+	Parent() Node
+	SetParent(Node)
+	Path() string
+	Equals(Node) bool
+	Visit(v Visitor)
+	toThrift() *format.SchemaElement
+}
+
+// Visitor is an interface for creating functionality to walk the schema tree.
+//
+// A visitor can be passed to the Visit function of a Node in order to walk
+// the tree. VisitPre is called the first time a node is encountered. If
+// it is a group node, the return is checked and if it is false, the children
+// will be skipped.
+//
+// VisitPost is called after visiting any children
+type Visitor interface {
+	VisitPre(Node) bool
+	VisitPost(Node)
+}
+
+// ColumnPathFromNode walks the parents of the given node to construct it's
+// column path
+func ColumnPathFromNode(n Node) parquet.ColumnPath {
+	if n == nil {
+		return nil
+	}
+
+	c := make([]string, 0)
+
+	// build the path in reverse order as we traverse nodes to the top
+	cursor := n
+	for cursor.Parent() != nil {
+		c = append(c, cursor.Name())
+		cursor = cursor.Parent()
+	}
+
+	// reverse the order of the list in place so that our result
+	// is in the proper, correct order.
+	for i := len(c)/2 - 1; i >= 0; i-- {

Review comment:
       go doesn't have a built in reverse function?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616819468



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {

Review comment:
       docs.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r631954773



##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval
+	IsSerialized() bool
+	// Returns true if not NoLogicalType
+	IsValid() bool
+	// Returns true if it is NoType
+	IsNone() bool
+	// returns a string representation of the Logical Type
+	String() string
+	toThrift() *format.LogicalType
+	// Return the equivalent ConvertedType for legacy Parquet systems
+	ToConvertedType() (ConvertedType, DecimalMetadata)
+	// Returns true if the specified ConvertedType is compatible with this
+	// logical type
+	IsCompatible(ConvertedType, DecimalMetadata) bool
+	// Returns true if this logical type can be used with the provided physical type
+	IsApplicable(t parquet.Type, tlen int32) bool
+	// Returns true if the logical types are the same
+	Equals(LogicalType) bool
+	// Returns the default stat sort order for this logical type
+	SortOrder() SortOrder
+}
+
+// TemporalLogicalType is a smaller interface for Time based logical types
+// like Time / Timestamp
+type TemporalLogicalType interface {
+	LogicalType
+	IsAdjustedToUTC() bool
+	TimeUnit() TimeUnitType
+}
+
+// SortOrder mirrors the parquet.thrift sort order type
+type SortOrder int8
+
+// Constants for the Stat sort order definitions
+const (
+	SortSIGNED SortOrder = iota
+	SortUNSIGNED
+	SortUNKNOWN
+)
+
+// DefaultSortOrder returns the default stat sort order for the given physical type
+func DefaultSortOrder(primitive format.Type) SortOrder {
+	switch primitive {
+	case format.Type_BOOLEAN, format.Type_INT32, format.Type_INT64, format.Type_FLOAT, format.Type_DOUBLE:
+		return SortSIGNED
+	case format.Type_BYTE_ARRAY, format.Type_FIXED_LEN_BYTE_ARRAY:
+		return SortUNSIGNED
+	case format.Type_INT96:
+		fallthrough
+	default:
+		return SortUNKNOWN
+	}
+}
+
+// GetLogicalSortOrder returns the default sort order for this logical type
+// or falls back to the default sort order for the physical type if not valid
+func GetLogicalSortOrder(logical LogicalType, primitive format.Type) SortOrder {
+	switch {
+	case logical == nil || !logical.IsValid():
+		return SortUNKNOWN
+	case logical.Equals(NoLogicalType{}):
+		return DefaultSortOrder(primitive)
+	default:
+		return logical.SortOrder()
+	}
+}
+
+type baseLogicalType struct{}
+
+func (baseLogicalType) IsSerialized() bool {
+	return true
+}
+
+func (baseLogicalType) IsValid() bool {
+	return true
+}
+
+func (baseLogicalType) IsNested() bool {
+	return false
+}
+
+func (baseLogicalType) IsNone() bool { return false }
+
+// StringLogicalType is a UTF8 string, only usable with ByteArray and FixedLenByteArray
+type StringLogicalType struct{ baseLogicalType }
+
+func (StringLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (StringLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": StringLogicalType{}.String()})
+}
+
+func (StringLogicalType) String() string {
+	return "String"
+}
+
+func (StringLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.UTF8, DecimalMetadata{}
+}
+
+func (StringLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.UTF8 && !dec.IsSet
+}
+
+func (StringLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (StringLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{STRING: format.NewStringType()}
+}
+
+func (StringLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(StringLogicalType)
+	return ok
+}
+
+// MapLogicalType represents a mapped type
+type MapLogicalType struct{ baseLogicalType }
+
+func (MapLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (MapLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": MapLogicalType{}.String()})
+}
+
+func (MapLogicalType) String() string {
+	return "Map"
+}
+
+func (MapLogicalType) IsNested() bool {
+	return true
+}
+
+func (MapLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Map, DecimalMetadata{}
+}
+
+func (MapLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return (t == ConvertedTypes.Map || t == ConvertedTypes.MapKeyValue) && !dec.IsSet
+}
+
+func (MapLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (MapLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{MAP: format.NewMapType()}
+}
+
+func (MapLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(MapLogicalType)
+	return ok
+}
+
+func NewListLogicalType() LogicalType {
+	return ListLogicalType{}
+}
+
+// ListLogicalType is used for columns which are themselves nested lists
+type ListLogicalType struct{ baseLogicalType }
+
+func (ListLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (ListLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": ListLogicalType{}.String()})
+}
+
+func (ListLogicalType) String() string {
+	return "List"
+}
+
+func (ListLogicalType) IsNested() bool {
+	return true
+}
+
+func (ListLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.List, DecimalMetadata{}
+}
+
+func (ListLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.List && !dec.IsSet
+}
+
+func (ListLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (ListLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{LIST: format.NewListType()}
+}
+
+func (ListLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(ListLogicalType)
+	return ok
+}
+
+// EnumLogicalType is for representing an enum, which should be a byte array type
+type EnumLogicalType struct{ baseLogicalType }
+
+func (EnumLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (EnumLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": EnumLogicalType{}.String()})
+}
+
+func (EnumLogicalType) String() string {
+	return "Enum"
+}
+
+func (EnumLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Enum, DecimalMetadata{}
+}
+
+func (EnumLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Enum && !dec.IsSet
+}
+
+func (EnumLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (EnumLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{ENUM: format.NewEnumType()}
+}
+
+func (EnumLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(EnumLogicalType)
+	return ok
+}
+
+// NewDecimalLogicalType returns a Decimal logical type with the given
+// precision and scale.
+//
+// Panics if precision < 1 or scale is not in the range (0, precision)
+func NewDecimalLogicalType(precision int32, scale int32) LogicalType {
+	if precision < 1 {
+		panic("parquet: precision must be greater than or equal to 1 for decimal logical type")
+	}
+	if scale < 0 || scale > precision {
+		panic("parquet: scale must be a non-negative integer that does not exceed precision for decimal logical type")
+	}
+	return &DecimalLogicalType{typ: &format.DecimalType{Precision: precision, Scale: scale}}
+}
+
+// DecimalLogicalType is used to represent a decimal value of a given
+// precision and scale
+type DecimalLogicalType struct {
+	baseLogicalType
+	typ *format.DecimalType
+}
+
+func (t DecimalLogicalType) Precision() int32 {
+	return t.typ.Precision
+}
+
+func (t DecimalLogicalType) Scale() int32 {
+	return t.typ.Scale
+}
+
+func (DecimalLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t DecimalLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{"Type": "Decimal", "precision": t.typ.Precision, "scale": t.typ.Scale})
+}
+
+func (t DecimalLogicalType) String() string {
+	return fmt.Sprintf("Decimal(precision=%d, scale=%d)", t.typ.Precision, t.typ.Scale)
+}
+
+func (t DecimalLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Decimal, DecimalMetadata{IsSet: true, Scale: t.typ.GetScale(), Precision: t.typ.GetPrecision()}
+}
+
+func (t DecimalLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.Decimal &&
+		dec.IsSet && dec.Scale == t.typ.Scale && dec.Precision == t.typ.Precision
+}
+
+func (t DecimalLogicalType) IsApplicable(typ parquet.Type, tlen int32) bool {
+	switch typ {
+	case parquet.Types.Int32:
+		return 1 <= t.typ.Precision && t.typ.Precision <= 9
+	case parquet.Types.Int64:
+		if t.typ.Precision < 10 {
+			debug.Log("int64 used for decimal logical, precision is small enough to use int32")
+		}
+		return 1 <= t.typ.Precision && t.typ.Precision <= 18
+	case parquet.Types.FixedLenByteArray:
+		return t.typ.Precision <= int32(math.Floor(math.Log10(math.Pow(2.0, (8.0*float64(tlen)-1.0)))))
+	case parquet.Types.ByteArray:
+		return true
+	}
+	return false
+}
+
+func (t DecimalLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DECIMAL: t.typ}
+}
+
+func (t DecimalLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*DecimalLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.Precision == other.typ.Precision && t.typ.Scale == other.typ.Scale
+}
+
+// DateLogicalType is an int32 representing the number of days since the Unix Epoch
+// 1 January 1970
+type DateLogicalType struct{ baseLogicalType }
+
+func (DateLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (DateLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": DateLogicalType{}.String()})
+}
+
+func (DateLogicalType) String() string {
+	return "Date"
+}
+
+func (DateLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Date, DecimalMetadata{}
+}
+
+func (DateLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Date && !dec.IsSet
+}
+
+func (DateLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int32
+}
+
+func (DateLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DATE: format.NewDateType()}
+}
+
+func (DateLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(DateLogicalType)
+	return ok
+}
+
+func timeUnitFromThrift(unit *format.TimeUnit) TimeUnitType {
+	switch {
+	case unit == nil:
+		return TimeUnitUnknown
+	case unit.IsSetMILLIS():
+		return TimeUnitMillis
+	case unit.IsSetMICROS():
+		return TimeUnitMicros
+	case unit.IsSetNANOS():
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func timeUnitToString(unit *format.TimeUnit) string {
+	switch {
+	case unit == nil:
+		return "unknown"
+	case unit.IsSetMILLIS():
+		return "milliseconds"
+	case unit.IsSetMICROS():
+		return "microseconds"
+	case unit.IsSetNANOS():
+		return "nanoseconds"
+	default:
+		return "unknown"
+	}
+}
+
+func timeUnitFromString(v string) TimeUnitType {
+	switch v {
+	case "millis":
+		return TimeUnitMillis
+	case "micros":
+		return TimeUnitMicros
+	case "nanos":
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func createTimeUnit(unit TimeUnitType) *format.TimeUnit {
+	tunit := format.NewTimeUnit()
+	switch unit {
+	case TimeUnitMicros:
+		tunit.MICROS = format.NewMicroSeconds()
+	case TimeUnitMillis:
+		tunit.MILLIS = format.NewMilliSeconds()
+	case TimeUnitNanos:
+		tunit.NANOS = format.NewNanoSeconds()
+	default:
+		panic("parquet: time unit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+	}
+	return tunit
+}
+
+// NewTimeLogicalType returns a time type of the given unit.
+func NewTimeLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimeLogicalType{typ: &format.TimeType{
+		IsAdjustedToUTC: isAdjustedToUTC,
+		Unit:            createTimeUnit(unit),
+	}}
+}
+
+// TimeLogicalType is a time type without a date and must be an
+// int32 for milliseconds, or an int64 for micro or nano seconds.
+type TimeLogicalType struct {
+	baseLogicalType
+	typ *format.TimeType
+}
+
+func (t TimeLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimeLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimeLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimeLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Time", "isAdjustedToUTC": t.typ.IsAdjustedToUTC, "timeUnit": timeUnitToString(t.typ.GetUnit())})
+}
+
+func (t TimeLogicalType) String() string {
+	return fmt.Sprintf("Time(isAdjustedToUTC=%t, timeUnit=%s)", t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()))
+}
+
+func (t TimeLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimeMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimeMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimeLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return c == ConvertedTypes.TimeMillis
+		case TimeUnitMicros:
+			return c == ConvertedTypes.TimeMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (t TimeLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetUnit().IsSetMILLIS()) ||
+		(typ == parquet.Types.Int64 &&
+			(t.typ.GetUnit().IsSetMICROS() || t.typ.GetUnit().IsSetNANOS()))
+}
+
+func (t TimeLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIME: t.typ}
+}
+
+func (t TimeLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimeLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewTimestampLogicalType returns a logical timestamp type with "forceConverted"
+// set to false
+func NewTimestampLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: false,
+		fromConverted:  false,
+	}
+}
+
+// NewTimestampLogicalTypeForce returns a timestamp logical type with
+// "forceConverted" set to true
+func NewTimestampLogicalTypeForce(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: true,
+		fromConverted:  false,
+	}
+}
+
+// TimestampLogicalType represents an int64 number that can be decoded
+// into a year, month, day, hour, minute, second, and subsecond
+type TimestampLogicalType struct {
+	baseLogicalType
+	typ            *format.TimestampType
+	forceConverted bool
+	fromConverted  bool
+}
+
+func (t TimestampLogicalType) IsFromConvertedType() bool {
+	return t.fromConverted
+}
+
+func (t TimestampLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimestampLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimestampLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimestampLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type":                     "Timestamp",
+		"isAdjustedToUTC":          t.typ.IsAdjustedToUTC,
+		"timeUnit":                 timeUnitToString(t.typ.GetUnit()),
+		"is_from_converted_type":   t.fromConverted,
+		"force_set_converted_type": t.forceConverted,
+	})
+}
+
+func (t TimestampLogicalType) IsSerialized() bool {
+	return !t.fromConverted
+}
+
+func (t TimestampLogicalType) String() string {
+	return fmt.Sprintf("Timestamp(isAdjustedToUTC=%t, timeUnit=%s, is_from_converted_type=%t, force_set_converted_type=%t)",
+		t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()), t.fromConverted, t.forceConverted)
+}
+
+func (t TimestampLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC || t.forceConverted {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimestampMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimestampMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimestampLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+
+	switch timeUnitFromThrift(t.typ.Unit) {
+	case TimeUnitMillis:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMillis
+		}
+	case TimeUnitMicros:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (TimestampLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int64
+}
+
+func (t TimestampLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIMESTAMP: t.typ}
+}
+
+func (t TimestampLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimestampLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewIntLogicalType creates an integer logical type of the desired bitwidth
+// and whether it is signed or not.
+//
+// Bit width must be exactly 8, 16, 32 or 64 for an integer logical type
+func NewIntLogicalType(bitWidth int8, signed bool) LogicalType {
+	switch bitWidth {
+	case 8, 16, 32, 64:
+	default:
+		panic("parquet: bit width must be exactly 8, 16, 32, or 64 for Int logical type")
+	}
+	return &IntLogicalType{
+		typ: &format.IntType{
+			BitWidth: bitWidth,
+			IsSigned: signed,
+		},
+	}
+}
+
+// IntLogicalType represents an integer type of a specific bit width and
+// is either signed or unsigned.
+type IntLogicalType struct {
+	baseLogicalType
+	typ *format.IntType
+}
+
+func (t IntLogicalType) BitWidth() int8 {
+	return t.typ.BitWidth
+}
+
+func (t IntLogicalType) IsSigned() bool {
+	return t.typ.IsSigned
+}
+
+func (t IntLogicalType) SortOrder() SortOrder {
+	if t.typ.IsSigned {
+		return SortSIGNED
+	}
+	return SortUNSIGNED
+}
+
+func (t IntLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Int", "bitWidth": t.typ.BitWidth, "isSigned": t.typ.IsSigned,
+	})
+}
+
+func (t IntLogicalType) String() string {
+	return fmt.Sprintf("Int(bitWidth=%d, isSigned=%t)", t.typ.GetBitWidth(), t.typ.GetIsSigned())
+}
+
+func (t IntLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	var d DecimalMetadata
+	if t.typ.IsSigned {
+		switch t.typ.BitWidth {
+		case 8:
+			return ConvertedTypes.Int8, d
+		case 16:
+			return ConvertedTypes.Int16, d
+		case 32:
+			return ConvertedTypes.Int32, d
+		case 64:
+			return ConvertedTypes.Int64, d
+		}
+	} else {
+		switch t.typ.BitWidth {
+		case 8:
+			return ConvertedTypes.Uint8, d
+		case 16:
+			return ConvertedTypes.Uint16, d
+		case 32:
+			return ConvertedTypes.Uint32, d
+		case 64:
+			return ConvertedTypes.Uint64, d
+		}
+	}
+	return ConvertedTypes.None, d
+}
+
+func (t IntLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	v, _ := t.ToConvertedType()
+	return c == v
+}
+
+func (t IntLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetBitWidth() <= 32) ||
+		(typ == parquet.Types.Int64 && t.typ.GetBitWidth() == 64)
+}
+
+func (t IntLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{INTEGER: t.typ}
+}
+
+func (t IntLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*IntLogicalType)
+	if !ok {
+		return false
+	}
+
+	return t.typ.GetIsSigned() == other.typ.GetIsSigned() &&
+		t.typ.GetBitWidth() == other.typ.GetBitWidth()
+}
+
+// UnknownLogicalType is a type that is essentially a placeholder for when
+// we don't know the type.
+type UnknownLogicalType struct{ baseLogicalType }
+
+func (UnknownLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (UnknownLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": UnknownLogicalType{}.String()})
+}
+
+func (UnknownLogicalType) IsValid() bool { return false }
+
+func (UnknownLogicalType) IsSerialized() bool { return false }
+
+func (UnknownLogicalType) String() string {
+	return "Unknown"
+}
+
+func (UnknownLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.NA, DecimalMetadata{}
+}
+
+func (UnknownLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.NA && !dec.IsSet
+}
+
+func (UnknownLogicalType) IsApplicable(parquet.Type, int32) bool { return true }
+
+func (UnknownLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{UNKNOWN: format.NewNullType()}
+}
+
+func (UnknownLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(UnknownLogicalType)
+	return ok
+}
+
+// JSONLogicalType represents a byte array column which is to be interpreted
+// as a JSON string.
+type JSONLogicalType struct{ baseLogicalType }
+
+func (JSONLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (JSONLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": JSONLogicalType{}.String()})
+}
+
+func (JSONLogicalType) String() string {
+	return "JSON"
+}
+
+func (JSONLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.JSON, DecimalMetadata{}
+}
+
+func (JSONLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.JSON && !dec.IsSet
+}
+
+func (JSONLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (JSONLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{JSON: format.NewJsonType()}
+}
+
+func (JSONLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(JSONLogicalType)
+	return ok
+}
+
+// BSONLogicalType represents a binary JSON string in the byte array
+type BSONLogicalType struct{ baseLogicalType }
+
+func (BSONLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (BSONLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": BSONLogicalType{}.String()})
+}
+
+func (BSONLogicalType) String() string {
+	return "BSON"
+}
+
+func (BSONLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.BSON, DecimalMetadata{}
+}
+
+func (BSONLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.BSON && !dec.IsSet
+}
+
+func (BSONLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (BSONLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{BSON: format.NewBsonType()}
+}
+
+func (BSONLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(BSONLogicalType)
+	return ok
+}
+
+// UUIDLogicalType can only be used with a FixedLength byte array column
+// that is exactly 16 bytes long
+type UUIDLogicalType struct{ baseLogicalType }
+
+func (UUIDLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (UUIDLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": UUIDLogicalType{}.String()})
+}
+
+func (UUIDLogicalType) String() string {
+	return "UUID"
+}
+
+func (UUIDLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (UUIDLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	switch c {
+	case ConvertedTypes.None, ConvertedTypes.NA:
+		return true
+	}
+	return false
+}
+
+func (UUIDLogicalType) IsApplicable(t parquet.Type, tlen int32) bool {
+	return t == parquet.Types.FixedLenByteArray && tlen == 16
+}
+
+func (UUIDLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{UUID: format.NewUUIDType()}
+}
+
+func (UUIDLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(UUIDLogicalType)
+	return ok
+}
+
+// IntervalLogicalType is not yet in the thrift spec, but represents
+// an interval time and needs to be a fixed length byte array of 12 bytes
+type IntervalLogicalType struct{ baseLogicalType }
+
+func (IntervalLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN

Review comment:
       are you saying that the comment in parquet.thrift needs to be updated? Or the docs I pointed at need to be updated and I should change this? Sorry, it's just unclear from your comment and I'm not sure if parquet.thrift is considered the source of truth or if the parquet-format repo is lol




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r623152901



##########
File path: go/parquet/schema/reflection_test.go
##########
@@ -0,0 +1,397 @@
+// 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.
+
+package schema_test
+
+import (
+	"log"
+	"os"
+	"reflect"
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/schema"
+	"github.com/stretchr/testify/assert"
+)
+
+func ExampleNewSchemaFromStruct_primitives() {
+	type Schema struct {
+		Bool              bool
+		Int8              int8
+		Uint16            uint16
+		Int32             int32
+		Int64             int64
+		Int96             parquet.Int96
+		Float             float32
+		Double            float64
+		ByteArray         string
+		FixedLenByteArray [10]byte
+	}
+
+	sc, err := schema.NewSchemaFromStruct(Schema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 Schema {
+	//   required boolean field_id=-1 Bool;
+	//   required int32 field_id=-1 Int8 (Int(bitWidth=8, isSigned=true));
+	//   required int32 field_id=-1 Uint16 (Int(bitWidth=16, isSigned=false));
+	//   required int32 field_id=-1 Int32 (Int(bitWidth=32, isSigned=true));
+	//   required int64 field_id=-1 Int64 (Int(bitWidth=64, isSigned=true));
+	//   required int96 field_id=-1 Int96;
+	//   required float field_id=-1 Float;
+	//   required double field_id=-1 Double;
+	//   required byte_array field_id=-1 ByteArray;
+	//   required fixed_len_byte_array field_id=-1 FixedLenByteArray;
+	// }
+}
+
+func ExampleNewSchemaFromStruct_convertedtypes() {
+	type ConvertedSchema struct {
+		Utf8           string        `parquet:"name=utf8, converted=UTF8"`
+		Uint32         uint32        `parquet:"converted=INT_32"`
+		Date           int32         `parquet:"name=date, converted=date"`
+		TimeMilli      int32         `parquet:"name=timemilli, converted=TIME_MILLIS"`
+		TimeMicro      int64         `parquet:"name=timemicro, converted=time_micros"`
+		TimeStampMilli int64         `parquet:"converted=timestamp_millis"`
+		TimeStampMicro int64         `parquet:"converted=timestamp_micros"`
+		Interval       parquet.Int96 `parquet:"converted=INTERVAL"`
+		Decimal1       int32         `parquet:"converted=decimal, scale=2, precision=9"`
+		Decimal2       int64         `parquet:"converted=decimal, scale=2, precision=18"`
+		Decimal3       [12]byte      `parquet:"converted=decimal, scale=2, precision=10"`
+		Decimal4       string        `parquet:"converted=decimal, scale=2, precision=20"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(&ConvertedSchema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 ConvertedSchema {
+	//   required byte_array field_id=-1 utf8 (String);
+	//   required int32 field_id=-1 Uint32 (Int(bitWidth=32, isSigned=true));
+	//   required int32 field_id=-1 date (Date);
+	//   required int32 field_id=-1 timemilli (Time(isAdjustedToUTC=true, timeUnit=milliseconds));
+	//   required int64 field_id=-1 timemicro (Time(isAdjustedToUTC=true, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeStampMilli (Timestamp(isAdjustedToUTC=true, timeUnit=milliseconds, is_from_converted_type=true, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimeStampMicro (Timestamp(isAdjustedToUTC=true, timeUnit=microseconds, is_from_converted_type=true, force_set_converted_type=false));
+	//   required int96 field_id=-1 Interval;
+	//   required int32 field_id=-1 Decimal1 (Decimal(precision=9, scale=2));
+	//   required int64 field_id=-1 Decimal2 (Decimal(precision=18, scale=2));
+	//   required fixed_len_byte_array field_id=-1 Decimal3 (Decimal(precision=10, scale=2));
+	//   required byte_array field_id=-1 Decimal4 (Decimal(precision=20, scale=2));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_repetition() {
+	type RepetitionSchema struct {
+		List     []int64 `parquet:"fieldid=1"`
+		Repeated []int64 `parquet:"repetition=repeated, fieldid=2"`
+		Optional *int64  `parquet:"fieldid=3"`
+		Required *int64  `parquet:"repetition=REQUIRED, fieldid=4"`
+		Opt      int64   `parquet:"repetition=OPTIONAL, fieldid=5"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(RepetitionSchema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 RepetitionSchema {
+	//   required group field_id=1 List (List) {
+	//     repeated group field_id=-1 list {
+	//       required int64 field_id=-1 element (Int(bitWidth=64, isSigned=true));
+	//     }
+	//   }
+	//   repeated int64 field_id=2 Repeated (Int(bitWidth=64, isSigned=true));
+	//   optional int64 field_id=3 Optional (Int(bitWidth=64, isSigned=true));
+	//   required int64 field_id=4 Required (Int(bitWidth=64, isSigned=true));
+	//   optional int64 field_id=5 Opt (Int(bitWidth=64, isSigned=true));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_logicaltypes() {
+	type LogicalTypes struct {
+		String                []byte   `parquet:"logical=String"`
+		Enum                  string   `parquet:"logical=enum"`
+		Date                  int32    `parquet:"logical=date"`
+		Decimal1              int32    `parquet:"logical=decimal, precision=9, scale=2"`
+		Decimal2              int32    `parquet:"logical=decimal, logical.precision=9, scale=2"`
+		Decimal3              int32    `parquet:"logical=decimal, precision=5, logical.precision=9, scale=1, logical.scale=3"`
+		TimeMilliUTC          int32    `parquet:"logical=TIME, logical.unit=millis"`
+		TimeMilli             int32    `parquet:"logical=Time, logical.unit=millis, logical.isadjustedutc=false"`
+		TimeMicros            int64    `parquet:"logical=time, logical.unit=micros, logical.isadjustedutc=false"`
+		TimeMicrosUTC         int64    `parquet:"logical=time, logical.unit=micros, logical.isadjustedutc=true"`
+		TimeNanos             int64    `parquet:"logical=time, logical.unit=nanos"`
+		TimestampMilli        int64    `parquet:"logical=timestamp, logical.unit=millis"`
+		TimestampMicrosNotUTC int64    `parquet:"logical=timestamp, logical.unit=micros, logical.isadjustedutc=false"`
+		TimestampNanos        int64    `parquet:"logical=timestamp, logical.unit=nanos"`
+		JSON                  string   `parquet:"logical=json"`
+		BSON                  []byte   `parquet:"logical=BSON"`
+		UUID                  [16]byte `parquet:"logical=uuid"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(LogicalTypes{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 LogicalTypes {
+	//   required byte_array field_id=-1 String (String);
+	//   required byte_array field_id=-1 Enum (Enum);
+	//   required int32 field_id=-1 Date (Date);
+	//   required int32 field_id=-1 Decimal1 (Decimal(precision=9, scale=2));
+	//   required int32 field_id=-1 Decimal2 (Decimal(precision=9, scale=2));
+	//   required int32 field_id=-1 Decimal3 (Decimal(precision=9, scale=3));
+	//   required int32 field_id=-1 TimeMilliUTC (Time(isAdjustedToUTC=true, timeUnit=milliseconds));
+	//   required int32 field_id=-1 TimeMilli (Time(isAdjustedToUTC=false, timeUnit=milliseconds));
+	//   required int64 field_id=-1 TimeMicros (Time(isAdjustedToUTC=false, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeMicrosUTC (Time(isAdjustedToUTC=true, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeNanos (Time(isAdjustedToUTC=true, timeUnit=nanoseconds));
+	//   required int64 field_id=-1 TimestampMilli (Timestamp(isAdjustedToUTC=true, timeUnit=milliseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimestampMicrosNotUTC (Timestamp(isAdjustedToUTC=false, timeUnit=microseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimestampNanos (Timestamp(isAdjustedToUTC=true, timeUnit=nanoseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required byte_array field_id=-1 JSON (JSON);
+	//   required byte_array field_id=-1 BSON (BSON);
+	//   required fixed_len_byte_array field_id=-1 UUID (UUID);
+	// }
+}
+
+func ExampleNewSchemaFromStruct_physicaltype() {
+	type ChangeTypes struct {
+		Int32        int64  `parquet:"type=int32"`
+		FixedLen     string `parquet:"type=fixed_len_byte_array, length=10"`
+		SliceAsFixed []byte `parquet:"type=fixed_len_byte_array, length=12"`
+		Int          int    `parquet:"type=int32"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(ChangeTypes{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 ChangeTypes {
+	//   required int32 field_id=-1 Int32 (Int(bitWidth=32, isSigned=true));
+	//   required fixed_len_byte_array field_id=-1 FixedLen;
+	//   required fixed_len_byte_array field_id=-1 SliceAsFixed;
+	//   required int32 field_id=-1 Int (Int(bitWidth=32, isSigned=true));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_nestedtypes() {
+	type Other struct {
+		OptionalMap *map[string]*string `parquet:"valuerepetition=required, keylogical=String, valueconverted=BSON"`
+	}
+
+	type MyMap map[int32]string
+
+	type Nested struct {
+		SimpleMap     map[int32]string
+		FixedLenMap   map[string][]byte `parquet:"keytype=fixed_len_byte_array, keyfieldid=10, valuefieldid=11, keylength=10"`
+		DecimalMap    map[int32]string  `parquet:"logical=map, keyconverted=DECIMAL, keyscale=3, keyprecision=7, valuetype=fixed_len_byte_array, valuelength=4, valuelogical=decimal, valuelogical.precision=9, valuescale=2"`
+		OtherList     []*Other
+		OtherRepeated []Other  `parquet:"repetition=repeated"`
+		DateArray     [5]int32 `parquet:"valuelogical=date, logical=list"`
+		DateMap       MyMap    `parquet:"keylogical=TIME, keylogical.unit=MILLIS, keylogical.isadjustedutc=false, valuelogical=enum"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(Nested{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 Nested {

Review comment:
       I didn't understand that formatting the comment in this way was an assertion of output.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619923761



##########
File path: go/parquet/schema/helpers.go
##########
@@ -0,0 +1,102 @@
+// 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.
+
+package schema
+
+import "github.com/apache/arrow/go/parquet"
+
+// ListOf is a convenience helper function to create a properly structured
+// list structure according to the Parquet Spec.
+//
+// <list-repetition> group <name> (LIST) {
+//   repeated group list {
+//     <element-repetition> <element-type> element;
+//   }
+// }
+//
+// <list-repetition> can only be optional or required. panics if repeated.
+// <element-repetition> can only be optional or required. panics if repeated.
+func ListOf(n Node, rep parquet.Repetition, fieldID int32) *GroupNode {
+	if rep == parquet.Repetitions.Repeated || n.RepetitionType() == parquet.Repetitions.Repeated {
+		panic("parquet: listof repetition and element repetition must not be repeated.")
+	}
+	listName := n.Name()
+
+	switch n := n.(type) {
+	case *PrimitiveNode:
+		n.name = "element"
+	case *GroupNode:
+		n.name = "element"
+	}
+
+	return NewGroupNodeLogical(listName, rep, FieldList{
+		NewGroupNode("list", parquet.Repetitions.Repeated, FieldList{n}, -1)},

Review comment:
       done

##########
File path: go/parquet/schema/helpers.go
##########
@@ -0,0 +1,102 @@
+// 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.
+
+package schema
+
+import "github.com/apache/arrow/go/parquet"
+
+// ListOf is a convenience helper function to create a properly structured
+// list structure according to the Parquet Spec.
+//
+// <list-repetition> group <name> (LIST) {
+//   repeated group list {
+//     <element-repetition> <element-type> element;
+//   }
+// }
+//
+// <list-repetition> can only be optional or required. panics if repeated.
+// <element-repetition> can only be optional or required. panics if repeated.
+func ListOf(n Node, rep parquet.Repetition, fieldID int32) *GroupNode {
+	if rep == parquet.Repetitions.Repeated || n.RepetitionType() == parquet.Repetitions.Repeated {
+		panic("parquet: listof repetition and element repetition must not be repeated.")
+	}
+	listName := n.Name()
+
+	switch n := n.(type) {
+	case *PrimitiveNode:
+		n.name = "element"
+	case *GroupNode:
+		n.name = "element"
+	}
+
+	return NewGroupNodeLogical(listName, rep, FieldList{
+		NewGroupNode("list", parquet.Repetitions.Repeated, FieldList{n}, -1)},
+		ListLogicalType{}, fieldID)
+}
+
+// MapOf is a convenience helper function to create a properly structured
+// parquet map node setup according to the Parquet Spec.
+//
+// <map-repetition> group <name> (MAP) {
+// 	 repeated group key_value {
+// 	   required <key-type> key;
+//     <value-repetition> <value-type> value;
+//   }
+// }
+//
+// key node will be renamed to "key", value node if not nil will be renamed to "value"
+//
+// <map-repetition> must be only optional or required. panics if repeated is passed.
+//
+// the key node *must* be required repetition. panics if optional or repeated
+//
+// value node can be nil (omitted) or have a repetition of required or optional *only*.
+// panics if value node is not nil and has a repetition of repeated.
+func MapOf(name string, key Node, value Node, mapRep parquet.Repetition, fieldID int32) *GroupNode {
+	if mapRep == parquet.Repetitions.Repeated {
+		panic("parquet: map repetition cannot be Repeated")
+	}
+	if key.RepetitionType() != parquet.Repetitions.Required {
+		panic("parquet: map key repetition must be Required")
+	}
+	if value != nil {
+		if value.RepetitionType() == parquet.Repetitions.Repeated {
+			panic("parquet: map value cannot have repetition Repeated")
+		}
+		switch value := value.(type) {
+		case *PrimitiveNode:
+			value.name = "value"
+		case *GroupNode:
+			value.name = "value"
+		}
+	}
+
+	switch key := key.(type) {
+	case *PrimitiveNode:
+		key.name = "key"
+	case *GroupNode:
+		key.name = "key"
+	}
+
+	keyval := FieldList{key}
+	if value != nil {
+		keyval = append(keyval, value)
+	}
+
+	return NewGroupNodeLogical(name, mapRep, FieldList{
+		NewGroupNode("key_value", parquet.Repetitions.Repeated, keyval, -1),

Review comment:
       done

##########
File path: go/parquet/schema/logical_types_test.go
##########
@@ -0,0 +1,551 @@
+// 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.
+
+package schema_test
+
+import (
+	"encoding/json"
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/schema"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestConvertedLogicalEquivalences(t *testing.T) {
+	tests := []struct {
+		name      string
+		converted schema.ConvertedType
+		logical   schema.LogicalType
+		expected  schema.LogicalType
+	}{
+		{"utf8", schema.ConvertedTypes.UTF8, schema.StringLogicalType{}, schema.StringLogicalType{}},
+		{"map", schema.ConvertedTypes.Map, schema.MapLogicalType{}, schema.MapLogicalType{}},
+		{"mapkeyval", schema.ConvertedTypes.MapKeyValue, schema.MapLogicalType{}, schema.MapLogicalType{}},
+		{"list", schema.ConvertedTypes.List, schema.NewListLogicalType(), schema.NewListLogicalType()},
+		{"enum", schema.ConvertedTypes.Enum, schema.EnumLogicalType{}, schema.EnumLogicalType{}},
+		{"date", schema.ConvertedTypes.Date, schema.DateLogicalType{}, schema.DateLogicalType{}},
+		{"timemilli", schema.ConvertedTypes.TimeMillis, schema.NewTimeLogicalType(true, schema.TimeUnitMillis), &schema.TimeLogicalType{}},
+		{"timemicro", schema.ConvertedTypes.TimeMicros, schema.NewTimeLogicalType(true, schema.TimeUnitMicros), &schema.TimeLogicalType{}},
+		{"timestampmilli", schema.ConvertedTypes.TimestampMillis, schema.NewTimestampLogicalType(true, schema.TimeUnitMillis), &schema.TimestampLogicalType{}},
+		{"timestampmicro", schema.ConvertedTypes.TimestampMicros, schema.NewTimestampLogicalType(true, schema.TimeUnitMicros), &schema.TimestampLogicalType{}},
+		{"uint8", schema.ConvertedTypes.Uint8, schema.NewIntLogicalType(8, false), &schema.IntLogicalType{}},
+		{"uint16", schema.ConvertedTypes.Uint16, schema.NewIntLogicalType(16, false), &schema.IntLogicalType{}},
+		{"uint32", schema.ConvertedTypes.Uint32, schema.NewIntLogicalType(32, false), &schema.IntLogicalType{}},
+		{"uint64", schema.ConvertedTypes.Uint64, schema.NewIntLogicalType(64, false), &schema.IntLogicalType{}},
+		{"int8", schema.ConvertedTypes.Int8, schema.NewIntLogicalType(8, true), &schema.IntLogicalType{}},

Review comment:
       done

##########
File path: go/parquet/schema/node.go
##########
@@ -0,0 +1,591 @@
+// 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.
+
+package schema
+
+import (
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/thrift/lib/go/thrift"
+	"golang.org/x/xerrors"
+)
+
+// NodeType describes whether the Node is a Primitive or Group node
+type NodeType int
+
+// the available constants for NodeType
+const (
+	Primitive NodeType = iota
+	Group
+)
+
+// Node is the interface for both Group and Primitive Nodes.
+// A logical schema type has a name, repetition level, and optionally
+// a logical type (ConvertedType in Parquet metadata parlance)

Review comment:
       fixed

##########
File path: go/parquet/schema/node.go
##########
@@ -0,0 +1,591 @@
+// 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.
+
+package schema
+
+import (
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/thrift/lib/go/thrift"
+	"golang.org/x/xerrors"
+)
+
+// NodeType describes whether the Node is a Primitive or Group node
+type NodeType int
+
+// the available constants for NodeType
+const (
+	Primitive NodeType = iota
+	Group
+)
+
+// Node is the interface for both Group and Primitive Nodes.
+// A logical schema type has a name, repetition level, and optionally
+// a logical type (ConvertedType in Parquet metadata parlance)
+type Node interface {
+	Name() string
+	Type() NodeType
+	RepetitionType() parquet.Repetition
+	ConvertedType() ConvertedType
+	LogicalType() LogicalType
+	FieldID() int32
+	Parent() Node
+	SetParent(Node)
+	Path() string
+	Equals(Node) bool
+	Visit(v Visitor)
+	toThrift() *format.SchemaElement
+}
+
+// Visitor is an interface for creating functionality to walk the schema tree.
+//
+// A visitor can be passed to the Visit function of a Node in order to walk
+// the tree. VisitPre is called the first time a node is encountered. If
+// it is a group node, the return is checked and if it is false, the children
+// will be skipped.
+//
+// VisitPost is called after visiting any children
+type Visitor interface {
+	VisitPre(Node) bool
+	VisitPost(Node)
+}
+
+// ColumnPathFromNode walks the parents of the given node to construct it's
+// column path
+func ColumnPathFromNode(n Node) parquet.ColumnPath {
+	if n == nil {
+		return nil
+	}
+
+	c := make([]string, 0)
+
+	cursor := n
+	for cursor.Parent() != nil {
+		c = append(c, cursor.Name())
+		cursor = cursor.Parent()
+	}
+
+	for i := len(c)/2 - 1; i >= 0; i-- {

Review comment:
       added comments




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619882315



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,110 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode *PrimitiveNode
+	// the maximum definition level in this column
+	maxDefLvl int16
+	// the maximum repetition level in this column

Review comment:
       might be worth adding.  when repetiation levels in the file equal this value, it indicates additional elements in the inner most list.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r633656815



##########
File path: go/parquet/schema/reflection.go
##########
@@ -0,0 +1,827 @@
+// 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.
+
+package schema
+
+import (
+	"reflect"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+type taggedInfo struct {
+	Name string
+
+	Type      parquet.Type
+	KeyType   parquet.Type
+	ValueType parquet.Type
+
+	Length      int32
+	KeyLength   int32
+	ValueLength int32
+
+	Scale      int32
+	KeyScale   int32
+	ValueScale int32
+
+	Precision      int32
+	KeyPrecision   int32
+	ValuePrecision int32
+
+	FieldID      int32
+	KeyFieldID   int32
+	ValueFieldID int32
+
+	RepetitionType  parquet.Repetition
+	ValueRepetition parquet.Repetition
+
+	Converted      ConvertedType
+	KeyConverted   ConvertedType
+	ValueConverted ConvertedType
+
+	LogicalFields      map[string]string
+	KeyLogicalFields   map[string]string
+	ValueLogicalFields map[string]string
+
+	LogicalType      LogicalType
+	KeyLogicalType   LogicalType
+	ValueLogicalType LogicalType
+}
+
+func (t *taggedInfo) CopyForKey() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.KeyType
+	ret.Length = t.KeyLength
+	ret.Scale = t.KeyScale
+	ret.Precision = t.KeyPrecision
+	ret.FieldID = t.KeyFieldID
+	ret.RepetitionType = parquet.Repetitions.Required
+	ret.Converted = t.KeyConverted
+	ret.LogicalType = t.KeyLogicalType
+	return
+}
+
+func (t *taggedInfo) CopyForValue() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.ValueType
+	ret.Length = t.ValueLength
+	ret.Scale = t.ValueScale
+	ret.Precision = t.ValuePrecision
+	ret.FieldID = t.ValueFieldID
+	ret.RepetitionType = t.ValueRepetition
+	ret.Converted = t.ValueConverted
+	ret.LogicalType = t.ValueLogicalType
+	return
+}
+
+func (t *taggedInfo) UpdateLogicalTypes() {
+	processLogicalType := func(fields map[string]string, precision, scale int32) LogicalType {
+		t, ok := fields["type"]
+		if !ok {
+			return NoLogicalType{}
+		}
+
+		switch strings.ToLower(t) {
+		case "string":
+			return StringLogicalType{}
+		case "map":
+			return MapLogicalType{}
+		case "list":
+			return ListLogicalType{}
+		case "enum":
+			return EnumLogicalType{}
+		case "decimal":
+			if v, ok := fields["precision"]; ok {
+				precision = int32FromType(v)
+			}
+			if v, ok := fields["scale"]; ok {
+				scale = int32FromType(v)
+			}
+			return NewDecimalLogicalType(precision, scale)
+		case "date":
+			return DateLogicalType{}
+		case "time":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimeLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(strings.ToLower(unit)))
+		case "timestamp":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimestampLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(unit))
+		case "integer":
+			width, ok := fields["bitwidth"]
+			if !ok {
+				panic("must specify bitwidth if explicitly setting integer logical type")
+			}
+			signed, ok := fields["signed"]
+			if !ok {
+				signed = "true"
+			}
+
+			return NewIntLogicalType(int8(int32FromType(width)), boolFromStr(signed))
+		case "null":
+			return NullLogicalType{}
+		case "json":
+			return JSONLogicalType{}
+		case "bson":
+			return BSONLogicalType{}
+		case "uuid":
+			return UUIDLogicalType{}
+		default:
+			panic(xerrors.Errorf("invalid logical type specified: %s", t))
+		}
+	}
+
+	t.LogicalType = processLogicalType(t.LogicalFields, t.Precision, t.Scale)
+	t.KeyLogicalType = processLogicalType(t.KeyLogicalFields, t.KeyPrecision, t.KeyScale)
+	t.ValueLogicalType = processLogicalType(t.ValueLogicalFields, t.ValuePrecision, t.ValueScale)
+}
+
+func newTaggedInfo() taggedInfo {
+	return taggedInfo{
+		Type:               parquet.Types.Undefined,
+		KeyType:            parquet.Types.Undefined,
+		ValueType:          parquet.Types.Undefined,
+		RepetitionType:     parquet.Repetitions.Undefined,
+		ValueRepetition:    parquet.Repetitions.Undefined,
+		Converted:          ConvertedTypes.NA,
+		KeyConverted:       ConvertedTypes.NA,
+		ValueConverted:     ConvertedTypes.NA,
+		FieldID:            -1,
+		KeyFieldID:         -1,
+		ValueFieldID:       -1,
+		LogicalFields:      make(map[string]string),
+		KeyLogicalFields:   make(map[string]string),
+		ValueLogicalFields: make(map[string]string),
+		LogicalType:        NoLogicalType{},
+		KeyLogicalType:     NoLogicalType{},
+		ValueLogicalType:   NoLogicalType{},
+	}
+}
+
+var int32FromType = func(v string) int32 {
+	val, err := strconv.Atoi(v)
+	if err != nil {
+		panic(err)
+	}
+	return int32(val)
+}
+
+var boolFromStr = func(v string) bool {
+	val, err := strconv.ParseBool(v)
+	if err != nil {
+		panic(err)
+	}
+	return val
+}
+
+func infoFromTags(f reflect.StructTag) *taggedInfo {
+	typeFromStr := func(v string) parquet.Type {
+		t, err := format.TypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(xerrors.Errorf("invalid type specified: %s", v))
+		}
+		return parquet.Type(t)
+	}
+
+	repFromStr := func(v string) parquet.Repetition {
+		r, err := format.FieldRepetitionTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return parquet.Repetition(r)
+	}
+
+	convertedFromStr := func(v string) ConvertedType {
+		c, err := format.ConvertedTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return ConvertedType(c)
+	}
+
+	if ptags, ok := f.Lookup("parquet"); ok {
+		info := newTaggedInfo()
+		for _, tag := range strings.Split(strings.Replace(ptags, "\t", "", -1), ",") {
+			tag = strings.TrimSpace(tag)
+			kv := strings.SplitN(tag, "=", 2)
+			key := strings.TrimSpace(strings.ToLower(kv[0]))
+			value := strings.TrimSpace(kv[1])
+
+			switch key {
+			case "name":
+				info.Name = value
+			case "type":
+				info.Type = typeFromStr(value)
+			case "keytype":
+				info.KeyType = typeFromStr(value)
+			case "valuetype":
+				info.ValueType = typeFromStr(value)
+			case "length":
+				info.Length = int32FromType(value)
+			case "keylength":
+				info.KeyLength = int32FromType(value)
+			case "valuelength":
+				info.ValueLength = int32FromType(value)
+			case "scale":
+				info.Scale = int32FromType(value)
+			case "keyscale":
+				info.KeyScale = int32FromType(value)
+			case "valuescale":
+				info.ValueScale = int32FromType(value)
+			case "precision":
+				info.Precision = int32FromType(value)
+			case "keyprecision":
+				info.KeyPrecision = int32FromType(value)
+			case "valueprecision":
+				info.ValuePrecision = int32FromType(value)
+			case "fieldid":
+				info.FieldID = int32FromType(value)
+			case "keyfieldid":
+				info.KeyFieldID = int32FromType(value)
+			case "valuefieldid":
+				info.ValueFieldID = int32FromType(value)
+			case "repetition":
+				info.RepetitionType = repFromStr(value)
+			case "valuerepetition":
+				info.ValueRepetition = repFromStr(value)
+			case "converted":
+				info.Converted = convertedFromStr(value)
+			case "keyconverted":
+				info.KeyConverted = convertedFromStr(value)
+			case "valueconverted":
+				info.ValueConverted = convertedFromStr(value)
+			case "logical":
+				info.LogicalFields["type"] = value
+			case "keylogical":
+				info.KeyLogicalFields["type"] = value
+			case "valuelogical":
+				info.ValueLogicalFields["type"] = value
+			default:
+				switch {
+				case strings.HasPrefix(key, "logical."):
+					info.LogicalFields[strings.TrimPrefix(key, "logical.")] = value
+				case strings.HasPrefix(key, "keylogical."):
+					info.KeyLogicalFields[strings.TrimPrefix(key, "keylogical.")] = value
+				case strings.HasPrefix(key, "valuelogical."):
+					info.ValueLogicalFields[strings.TrimPrefix(key, "valuelogical.")] = value
+				}
+			}
+		}
+		info.UpdateLogicalTypes()
+		return &info
+	}
+	return nil
+}
+
+// typeToNode recurseively converts a physical type and the tag info into parquet Nodes
+//
+// to avoid having to propagate errors up potentially high numbers of recursive calls
+// we use panics and then recover in the public function NewSchemaFromStruct so that a
+// failure very far down the stack quickly unwinds.
+func typeToNode(name string, typ reflect.Type, repType parquet.Repetition, info *taggedInfo) Node {
+	// set up our default values for everything
+	var (
+		converted             = ConvertedTypes.None
+		logical   LogicalType = NoLogicalType{}
+		fieldID               = int32(-1)
+		physical              = parquet.Types.Undefined
+		typeLen               = 0
+		precision             = 0
+		scale                 = 0
+	)
+	if info != nil { // we have struct tag info to process
+		fieldID = info.FieldID

Review comment:
       just a note there has been a recent issue/PR that is changing the logic around fieldID (c++ code will not generate them any more)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616825171



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {
+			return true
+		}
+		n = n.Parent()
+	}
+	return false
+}
+
+// NewColumn returns a new column object for the given node with the provided
+// maximum definition and repetition levels.
+//
+// n MUST be a PrimitiveNode, otherwise this will panic.
+func NewColumn(n Node, maxDefinitionLvl, maxRepetitionLvl int16) *Column {
+	if n.Type() != Primitive {
+		panic("parquet: Column must be a primitive type")
+	}
+	return &Column{n.(*PrimitiveNode), maxDefinitionLvl, maxRepetitionLvl}
+}
+
+// Name is the column's name
+func (c *Column) Name() string { return c.pnode.Name() }
+
+// ColumnPath returns the full path to this column from the root of the schema
+func (c *Column) ColumnPath() parquet.ColumnPath { return c.pnode.columnPath() }
+
+// Path is equivalent to ColumnPath().String() returning the dot-string version of the path
+func (c *Column) Path() string { return c.pnode.Path() }
+
+// TypeLength is -1 if not a FixedLenByteArray, otherwise it is the length for the column

Review comment:
       nit: elements in the column.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619882571



##########
File path: go/parquet/schema/converted_types.go
##########
@@ -0,0 +1,191 @@
+// 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.
+
+package schema
+
+import (
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// ConvertedType corresponds to the ConvertedType in the parquet.Thrift,
+// with added values of None and NA for handling when these values are not
+// set in the metadata
+type ConvertedType format.ConvertedType
+
+var (
+	// ConvertedTypes is a struct containing the constants for the types
+	// to make it easy to reference them while making it clear what they are
+	ConvertedTypes = struct {
+		None            ConvertedType
+		UTF8            ConvertedType
+		Map             ConvertedType
+		MapKeyValue     ConvertedType
+		List            ConvertedType
+		Enum            ConvertedType
+		Decimal         ConvertedType
+		Date            ConvertedType
+		TimeMillis      ConvertedType
+		TimeMicros      ConvertedType
+		TimestampMillis ConvertedType
+		TimestampMicros ConvertedType
+		Uint8           ConvertedType
+		Uint16          ConvertedType
+		Uint32          ConvertedType
+		Uint64          ConvertedType
+		Int8            ConvertedType
+		Int16           ConvertedType
+		Int32           ConvertedType
+		Int64           ConvertedType
+		JSON            ConvertedType
+		BSON            ConvertedType
+		Interval        ConvertedType
+		NA              ConvertedType
+	}{
+		None:            -1, // thrift enum starts at 0, so we know this will not be used
+		UTF8:            ConvertedType(format.ConvertedType_UTF8),
+		Map:             ConvertedType(format.ConvertedType_MAP),
+		MapKeyValue:     ConvertedType(format.ConvertedType_MAP_KEY_VALUE),
+		List:            ConvertedType(format.ConvertedType_LIST),
+		Enum:            ConvertedType(format.ConvertedType_ENUM),
+		Decimal:         ConvertedType(format.ConvertedType_DECIMAL),
+		Date:            ConvertedType(format.ConvertedType_DATE),
+		TimeMillis:      ConvertedType(format.ConvertedType_TIME_MILLIS),
+		TimeMicros:      ConvertedType(format.ConvertedType_TIME_MICROS),
+		TimestampMillis: ConvertedType(format.ConvertedType_TIMESTAMP_MILLIS),
+		TimestampMicros: ConvertedType(format.ConvertedType_TIMESTAMP_MICROS),
+		Uint8:           ConvertedType(format.ConvertedType_UINT_8),
+		Uint16:          ConvertedType(format.ConvertedType_UINT_16),
+		Uint32:          ConvertedType(format.ConvertedType_UINT_32),
+		Uint64:          ConvertedType(format.ConvertedType_UINT_64),
+		Int8:            ConvertedType(format.ConvertedType_INT_8),
+		Int16:           ConvertedType(format.ConvertedType_INT_16),
+		Int32:           ConvertedType(format.ConvertedType_INT_32),
+		Int64:           ConvertedType(format.ConvertedType_INT_64),
+		JSON:            ConvertedType(format.ConvertedType_JSON),
+		BSON:            ConvertedType(format.ConvertedType_BSON),
+		Interval:        ConvertedType(format.ConvertedType_INTERVAL),
+		NA:              24, // should always be the last values after Interval
+	}
+)
+
+func (p ConvertedType) String() string {
+	switch p {
+	case ConvertedTypes.None:
+		return "NONE"
+	case ConvertedTypes.NA:
+		return "UNKNOWN"
+	default:
+		return format.ConvertedType(p).String()
+	}
+}
+
+// ToLogicalType returns the correct LogicalType for the given ConvertedType, using the decimal
+// metadata provided to define the precision/scale if necessary
+func (p ConvertedType) ToLogicalType(convertedDecimal DecimalMetadata) LogicalType {
+	switch p {
+	case ConvertedTypes.UTF8:
+		return StringLogicalType{}
+	case ConvertedTypes.Map, ConvertedTypes.MapKeyValue:
+		return MapLogicalType{}
+	case ConvertedTypes.List:
+		return ListLogicalType{}
+	case ConvertedTypes.Enum:
+		return EnumLogicalType{}
+	case ConvertedTypes.Decimal:
+		return NewDecimalLogicalType(convertedDecimal.Precision, convertedDecimal.Scale)
+	case ConvertedTypes.Date:
+		return DateLogicalType{}
+	case ConvertedTypes.TimeMillis:
+		return NewTimeLogicalType(true, TimeUnitMillis)

Review comment:
       add comments for literal values plase.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619923938



##########
File path: go/parquet/schema/schema_element_test.go
##########
@@ -0,0 +1,432 @@
+// 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.
+
+package schema
+
+import (
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/suite"
+)
+
+type schemaElementConstruction struct {
+	node            Node
+	element         *format.SchemaElement
+	name            string
+	expectConverted bool
+	converted       ConvertedType
+	expectLogical   bool
+	checkLogical    func(*format.SchemaElement) bool
+}
+
+type decimalSchemaElementConstruction struct {
+	schemaElementConstruction
+	precision int
+	scale     int
+}
+
+type temporalSchemaElementConstruction struct {
+	schemaElementConstruction
+	adjusted bool
+	unit     TimeUnitType
+	getUnit  func(*format.SchemaElement) *format.TimeUnit
+}
+
+type intSchemaElementConstruction struct {
+	schemaElementConstruction
+	width  int8
+	signed bool
+}
+
+type legacySchemaElementConstructArgs struct {
+	name            string
+	physical        parquet.Type
+	len             int
+	expectConverted bool
+	converted       ConvertedType
+	expectLogical   bool
+	checkLogical    func(*format.SchemaElement) bool
+}
+
+type schemaElementConstructArgs struct {
+	name            string
+	logical         LogicalType
+	physical        parquet.Type
+	len             int
+	expectConverted bool
+	converted       ConvertedType
+	expectLogical   bool
+	checkLogical    func(*format.SchemaElement) bool
+}
+type SchemaElementConstructionSuite struct {
+	suite.Suite
+}
+
+func (s *SchemaElementConstructionSuite) reconstruct(c schemaElementConstructArgs) *schemaElementConstruction {
+	ret := &schemaElementConstruction{
+		node:            NewPrimitiveNodeLogical(c.name, parquet.Repetitions.Required, c.logical, c.physical, c.len, -1),
+		name:            c.name,
+		expectConverted: c.expectConverted,
+		converted:       c.converted,
+		expectLogical:   c.expectLogical,
+		checkLogical:    c.checkLogical,
+	}
+	ret.element = ret.node.toThrift()
+	return ret
+}
+
+func (s *SchemaElementConstructionSuite) legacyReconstruct(c legacySchemaElementConstructArgs) *schemaElementConstruction {
+	ret := &schemaElementConstruction{
+		node:            NewPrimitiveNodeConverted(c.name, parquet.Repetitions.Required, c.physical, c.converted, c.len, 0, 0, -1),
+		name:            c.name,
+		expectConverted: c.expectConverted,
+		converted:       c.converted,
+		expectLogical:   c.expectLogical,
+		checkLogical:    c.checkLogical,
+	}
+	ret.element = ret.node.toThrift()
+	return ret
+}
+
+func (s *SchemaElementConstructionSuite) inspect(c *schemaElementConstruction) {
+	if c.expectConverted {
+		s.True(c.element.IsSetConvertedType())
+		s.Equal(c.converted, ConvertedType(*c.element.ConvertedType))
+	} else {
+		s.False(c.element.IsSetConvertedType())
+	}
+	if c.expectLogical {
+		s.True(c.element.IsSetLogicalType())
+		s.True(c.checkLogical(c.element))
+	} else {
+		s.False(c.element.IsSetLogicalType())
+	}
+}
+
+func (s *SchemaElementConstructionSuite) TestSimple() {
+	checkNone := func(*format.SchemaElement) bool { return true }
+
+	tests := []struct {
+		name   string
+		args   *schemaElementConstructArgs
+		legacy *legacySchemaElementConstructArgs
+	}{
+		{"string", &schemaElementConstructArgs{
+			"string", StringLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.UTF8, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetSTRING() },
+		}, nil},
+		{"enum", &schemaElementConstructArgs{
+			"enum", EnumLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.Enum, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetENUM() },
+		}, nil},
+		{"date", &schemaElementConstructArgs{
+			"date", DateLogicalType{}, parquet.Types.Int32, -1, true, ConvertedTypes.Date, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetDATE() },
+		}, nil},
+		{"interval", &schemaElementConstructArgs{
+			"interval", IntervalLogicalType{}, parquet.Types.FixedLenByteArray, 12, true, ConvertedTypes.Interval, false,
+			checkNone,
+		}, nil},
+		{"null", &schemaElementConstructArgs{
+			"null", NullLogicalType{}, parquet.Types.Double, -1, false, ConvertedTypes.NA, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetUNKNOWN() },
+		}, nil},
+		{"json", &schemaElementConstructArgs{
+			"json", JSONLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.JSON, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetJSON() },
+		}, nil},
+		{"bson", &schemaElementConstructArgs{
+			"bson", BSONLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.BSON, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetBSON() },
+		}, nil},
+		{"uuid", &schemaElementConstructArgs{
+			"uuid", UUIDLogicalType{}, parquet.Types.FixedLenByteArray, 16, false, ConvertedTypes.NA, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetUUID() },
+		}, nil},
+		{"none", &schemaElementConstructArgs{
+			"none", NoLogicalType{}, parquet.Types.Int64, -1, false, ConvertedTypes.NA, false,
+			checkNone,
+		}, nil},
+		{"unknown", &schemaElementConstructArgs{
+			"unknown", UnknownLogicalType{}, parquet.Types.Int64, -1, true, ConvertedTypes.NA, false,
+			checkNone,
+		}, nil},
+		{"timestamp_ms", nil, &legacySchemaElementConstructArgs{
+			"timestamp_ms", parquet.Types.Int64, -1, true, ConvertedTypes.TimestampMillis, false, checkNone}},
+		{"timestamp_us", nil, &legacySchemaElementConstructArgs{
+			"timestamp_us", parquet.Types.Int64, -1, true, ConvertedTypes.TimestampMicros, false, checkNone}},
+	}
+	for _, tt := range tests {
+		s.Run(tt.name, func() {
+			var sc *schemaElementConstruction
+			if tt.args != nil {
+				sc = s.reconstruct(*tt.args)
+			} else {
+				sc = s.legacyReconstruct(*tt.legacy)
+			}
+			s.Equal(tt.name, sc.element.Name)
+			s.inspect(sc)
+		})
+	}
+}
+
+func (s *SchemaElementConstructionSuite) reconstructDecimal(c schemaElementConstructArgs) *decimalSchemaElementConstruction {
+	ret := s.reconstruct(c)
+	dec := c.logical.(*DecimalLogicalType)
+	return &decimalSchemaElementConstruction{*ret, int(dec.Precision()), int(dec.Scale())}
+}
+
+func (s *SchemaElementConstructionSuite) inspectDecimal(d *decimalSchemaElementConstruction) {
+	s.inspect(&d.schemaElementConstruction)
+	s.EqualValues(d.precision, d.element.GetPrecision())
+	s.EqualValues(d.scale, d.element.GetScale())
+	s.EqualValues(d.precision, d.element.LogicalType.DECIMAL.Precision)
+	s.EqualValues(d.scale, d.element.LogicalType.DECIMAL.Scale)
+}
+
+func (s *SchemaElementConstructionSuite) TestDecimal() {
+	checkDecimal := func(p *format.SchemaElement) bool { return p.LogicalType.IsSetDECIMAL() }
+
+	tests := []schemaElementConstructArgs{
+		{"decimal16_6", NewDecimalLogicalType(16, 6), parquet.Types.Int64, -1, true, ConvertedTypes.Decimal, true, checkDecimal},
+		{"decimal1_0", NewDecimalLogicalType(1, 0), parquet.Types.Int32, -1, true, ConvertedTypes.Decimal, true, checkDecimal},
+		{"decimal10", NewDecimalLogicalType(10, 0), parquet.Types.Int64, -1, true, ConvertedTypes.Decimal, true, checkDecimal},

Review comment:
       added

##########
File path: go/parquet/schema/schema_flatten_test.go
##########
@@ -0,0 +1,150 @@
+// 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.
+
+package schema
+
+import (
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/suite"
+)
+
+func NewPrimitive(name string, repetition format.FieldRepetitionType, typ format.Type, fieldID int32) *format.SchemaElement {
+	ret := &format.SchemaElement{
+		Name:           name,
+		RepetitionType: format.FieldRepetitionTypePtr(repetition),
+		Type:           format.TypePtr(typ),
+	}
+	if fieldID >= 0 {
+		ret.FieldID = &fieldID
+	}
+	return ret
+}
+
+func NewGroup(name string, repetition format.FieldRepetitionType, numChildren, fieldID int32) *format.SchemaElement {
+	ret := &format.SchemaElement{
+		Name:           name,
+		RepetitionType: format.FieldRepetitionTypePtr(repetition),
+		NumChildren:    &numChildren,
+	}
+	if fieldID >= 0 {
+		ret.FieldID = &fieldID
+	}
+	return ret
+}
+
+type SchemaFlattenSuite struct {
+	suite.Suite
+
+	name string
+}
+
+func (s *SchemaFlattenSuite) SetupSuite() {
+	s.name = "parquet_schema"
+}
+
+func (s *SchemaFlattenSuite) TestDecimalMetadata() {
+	group := NewGroupNodeConverted("group", parquet.Repetitions.Repeated, FieldList{
+		NewPrimitiveNodeConverted("decimal", parquet.Repetitions.Required, parquet.Types.Int64, ConvertedTypes.Decimal, 0, 8, 4, -1),

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-829654974


   @nickpoorman so the tact I've taken in here has been essentially if something is a *programmer* error such as trying to create an integer node with string logical type, and is unrecoverable I might panic. But if something fails based on user inputs or in normal operation, then I return an error. 
   
   I've done a lot of looking to try to reduce the panics in here, is there any specific ones or patterns in here you think should be an error where I'm doing a panic?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] nickpoorman commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
nickpoorman commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-829607714


   I'm not a fan of the panics instead of returning errors. Having the panics means if I'm running this in a production environment I have to wrap everything I do with Arrow in a rescue defer which is going to slow things down.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r617007391



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {

Review comment:
       so this looks like an artifact from earlier versions of my code, I didn't notice that I don't call it anymore and even my go-staticcheck is pointing out that this is no longer called at all, even in my finished version of the repo. So i'll remove this function entirely.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619906330



##########
File path: go/parquet/schema/reflection_test.go
##########
@@ -0,0 +1,397 @@
+// 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.
+
+package schema_test
+
+import (
+	"log"
+	"os"
+	"reflect"
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/schema"
+	"github.com/stretchr/testify/assert"
+)
+
+func ExampleNewSchemaFromStruct_primitives() {
+	type Schema struct {
+		Bool              bool
+		Int8              int8
+		Uint16            uint16
+		Int32             int32
+		Int64             int64
+		Int96             parquet.Int96
+		Float             float32
+		Double            float64
+		ByteArray         string
+		FixedLenByteArray [10]byte
+	}
+
+	sc, err := schema.NewSchemaFromStruct(Schema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 Schema {
+	//   required boolean field_id=-1 Bool;
+	//   required int32 field_id=-1 Int8 (Int(bitWidth=8, isSigned=true));
+	//   required int32 field_id=-1 Uint16 (Int(bitWidth=16, isSigned=false));
+	//   required int32 field_id=-1 Int32 (Int(bitWidth=32, isSigned=true));
+	//   required int64 field_id=-1 Int64 (Int(bitWidth=64, isSigned=true));
+	//   required int96 field_id=-1 Int96;
+	//   required float field_id=-1 Float;
+	//   required double field_id=-1 Double;
+	//   required byte_array field_id=-1 ByteArray;
+	//   required fixed_len_byte_array field_id=-1 FixedLenByteArray;
+	// }
+}
+
+func ExampleNewSchemaFromStruct_convertedtypes() {
+	type ConvertedSchema struct {
+		Utf8           string        `parquet:"name=utf8, converted=UTF8"`
+		Uint32         uint32        `parquet:"converted=INT_32"`
+		Date           int32         `parquet:"name=date, converted=date"`
+		TimeMilli      int32         `parquet:"name=timemilli, converted=TIME_MILLIS"`
+		TimeMicro      int64         `parquet:"name=timemicro, converted=time_micros"`
+		TimeStampMilli int64         `parquet:"converted=timestamp_millis"`
+		TimeStampMicro int64         `parquet:"converted=timestamp_micros"`
+		Interval       parquet.Int96 `parquet:"converted=INTERVAL"`
+		Decimal1       int32         `parquet:"converted=decimal, scale=2, precision=9"`
+		Decimal2       int64         `parquet:"converted=decimal, scale=2, precision=18"`
+		Decimal3       [12]byte      `parquet:"converted=decimal, scale=2, precision=10"`
+		Decimal4       string        `parquet:"converted=decimal, scale=2, precision=20"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(&ConvertedSchema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 ConvertedSchema {
+	//   required byte_array field_id=-1 utf8 (String);
+	//   required int32 field_id=-1 Uint32 (Int(bitWidth=32, isSigned=true));
+	//   required int32 field_id=-1 date (Date);
+	//   required int32 field_id=-1 timemilli (Time(isAdjustedToUTC=true, timeUnit=milliseconds));
+	//   required int64 field_id=-1 timemicro (Time(isAdjustedToUTC=true, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeStampMilli (Timestamp(isAdjustedToUTC=true, timeUnit=milliseconds, is_from_converted_type=true, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimeStampMicro (Timestamp(isAdjustedToUTC=true, timeUnit=microseconds, is_from_converted_type=true, force_set_converted_type=false));
+	//   required int96 field_id=-1 Interval;
+	//   required int32 field_id=-1 Decimal1 (Decimal(precision=9, scale=2));
+	//   required int64 field_id=-1 Decimal2 (Decimal(precision=18, scale=2));
+	//   required fixed_len_byte_array field_id=-1 Decimal3 (Decimal(precision=10, scale=2));
+	//   required byte_array field_id=-1 Decimal4 (Decimal(precision=20, scale=2));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_repetition() {
+	type RepetitionSchema struct {
+		List     []int64 `parquet:"fieldid=1"`
+		Repeated []int64 `parquet:"repetition=repeated, fieldid=2"`
+		Optional *int64  `parquet:"fieldid=3"`
+		Required *int64  `parquet:"repetition=REQUIRED, fieldid=4"`
+		Opt      int64   `parquet:"repetition=OPTIONAL, fieldid=5"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(RepetitionSchema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 RepetitionSchema {
+	//   required group field_id=1 List (List) {
+	//     repeated group field_id=-1 list {
+	//       required int64 field_id=-1 element (Int(bitWidth=64, isSigned=true));
+	//     }
+	//   }
+	//   repeated int64 field_id=2 Repeated (Int(bitWidth=64, isSigned=true));
+	//   optional int64 field_id=3 Optional (Int(bitWidth=64, isSigned=true));
+	//   required int64 field_id=4 Required (Int(bitWidth=64, isSigned=true));
+	//   optional int64 field_id=5 Opt (Int(bitWidth=64, isSigned=true));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_logicaltypes() {
+	type LogicalTypes struct {
+		String                []byte   `parquet:"logical=String"`
+		Enum                  string   `parquet:"logical=enum"`
+		Date                  int32    `parquet:"logical=date"`
+		Decimal1              int32    `parquet:"logical=decimal, precision=9, scale=2"`
+		Decimal2              int32    `parquet:"logical=decimal, logical.precision=9, scale=2"`
+		Decimal3              int32    `parquet:"logical=decimal, precision=5, logical.precision=9, scale=1, logical.scale=3"`
+		TimeMilliUTC          int32    `parquet:"logical=TIME, logical.unit=millis"`
+		TimeMilli             int32    `parquet:"logical=Time, logical.unit=millis, logical.isadjustedutc=false"`
+		TimeMicros            int64    `parquet:"logical=time, logical.unit=micros, logical.isadjustedutc=false"`
+		TimeMicrosUTC         int64    `parquet:"logical=time, logical.unit=micros, logical.isadjustedutc=true"`
+		TimeNanos             int64    `parquet:"logical=time, logical.unit=nanos"`
+		TimestampMilli        int64    `parquet:"logical=timestamp, logical.unit=millis"`
+		TimestampMicrosNotUTC int64    `parquet:"logical=timestamp, logical.unit=micros, logical.isadjustedutc=false"`
+		TimestampNanos        int64    `parquet:"logical=timestamp, logical.unit=nanos"`
+		JSON                  string   `parquet:"logical=json"`
+		BSON                  []byte   `parquet:"logical=BSON"`
+		UUID                  [16]byte `parquet:"logical=uuid"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(LogicalTypes{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 LogicalTypes {
+	//   required byte_array field_id=-1 String (String);
+	//   required byte_array field_id=-1 Enum (Enum);
+	//   required int32 field_id=-1 Date (Date);
+	//   required int32 field_id=-1 Decimal1 (Decimal(precision=9, scale=2));
+	//   required int32 field_id=-1 Decimal2 (Decimal(precision=9, scale=2));
+	//   required int32 field_id=-1 Decimal3 (Decimal(precision=9, scale=3));
+	//   required int32 field_id=-1 TimeMilliUTC (Time(isAdjustedToUTC=true, timeUnit=milliseconds));
+	//   required int32 field_id=-1 TimeMilli (Time(isAdjustedToUTC=false, timeUnit=milliseconds));
+	//   required int64 field_id=-1 TimeMicros (Time(isAdjustedToUTC=false, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeMicrosUTC (Time(isAdjustedToUTC=true, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeNanos (Time(isAdjustedToUTC=true, timeUnit=nanoseconds));
+	//   required int64 field_id=-1 TimestampMilli (Timestamp(isAdjustedToUTC=true, timeUnit=milliseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimestampMicrosNotUTC (Timestamp(isAdjustedToUTC=false, timeUnit=microseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimestampNanos (Timestamp(isAdjustedToUTC=true, timeUnit=nanoseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required byte_array field_id=-1 JSON (JSON);
+	//   required byte_array field_id=-1 BSON (BSON);
+	//   required fixed_len_byte_array field_id=-1 UUID (UUID);
+	// }
+}
+
+func ExampleNewSchemaFromStruct_physicaltype() {
+	type ChangeTypes struct {
+		Int32        int64  `parquet:"type=int32"`
+		FixedLen     string `parquet:"type=fixed_len_byte_array, length=10"`
+		SliceAsFixed []byte `parquet:"type=fixed_len_byte_array, length=12"`
+		Int          int    `parquet:"type=int32"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(ChangeTypes{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 ChangeTypes {
+	//   required int32 field_id=-1 Int32 (Int(bitWidth=32, isSigned=true));
+	//   required fixed_len_byte_array field_id=-1 FixedLen;
+	//   required fixed_len_byte_array field_id=-1 SliceAsFixed;
+	//   required int32 field_id=-1 Int (Int(bitWidth=32, isSigned=true));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_nestedtypes() {
+	type Other struct {
+		OptionalMap *map[string]*string `parquet:"valuerepetition=required, keylogical=String, valueconverted=BSON"`
+	}
+
+	type MyMap map[int32]string
+
+	type Nested struct {
+		SimpleMap     map[int32]string
+		FixedLenMap   map[string][]byte `parquet:"keytype=fixed_len_byte_array, keyfieldid=10, valuefieldid=11, keylength=10"`
+		DecimalMap    map[int32]string  `parquet:"logical=map, keyconverted=DECIMAL, keyscale=3, keyprecision=7, valuetype=fixed_len_byte_array, valuelength=4, valuelogical=decimal, valuelogical.precision=9, valuescale=2"`
+		OtherList     []*Other
+		OtherRepeated []Other  `parquet:"repetition=repeated"`
+		DateArray     [5]int32 `parquet:"valuelogical=date, logical=list"`
+		DateMap       MyMap    `parquet:"keylogical=TIME, keylogical.unit=MILLIS, keylogical.isadjustedutc=false, valuelogical=enum"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(Nested{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 Nested {

Review comment:
       I'm not sure what you mean, what assertion are you referring to?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619882252



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16

Review comment:
       OK.  might be worth adding, that at this level, a corresponding value is present.

##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,110 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode *PrimitiveNode
+	// the maximum definition level in this column
+	maxDefLvl int16
+	// the maximum repetition level in this column

Review comment:
       might be worth adding.  when repetiation levels in the file equal this value, it indicates additional elements in the inner most list.

##########
File path: go/parquet/schema/converted_types.go
##########
@@ -0,0 +1,191 @@
+// 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.
+
+package schema
+
+import (
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// ConvertedType corresponds to the ConvertedType in the parquet.Thrift,
+// with added values of None and NA for handling when these values are not
+// set in the metadata
+type ConvertedType format.ConvertedType
+
+var (
+	// ConvertedTypes is a struct containing the constants for the types
+	// to make it easy to reference them while making it clear what they are
+	ConvertedTypes = struct {
+		None            ConvertedType
+		UTF8            ConvertedType
+		Map             ConvertedType
+		MapKeyValue     ConvertedType
+		List            ConvertedType
+		Enum            ConvertedType
+		Decimal         ConvertedType
+		Date            ConvertedType
+		TimeMillis      ConvertedType
+		TimeMicros      ConvertedType
+		TimestampMillis ConvertedType
+		TimestampMicros ConvertedType
+		Uint8           ConvertedType
+		Uint16          ConvertedType
+		Uint32          ConvertedType
+		Uint64          ConvertedType
+		Int8            ConvertedType
+		Int16           ConvertedType
+		Int32           ConvertedType
+		Int64           ConvertedType
+		JSON            ConvertedType
+		BSON            ConvertedType
+		Interval        ConvertedType
+		NA              ConvertedType
+	}{
+		None:            -1, // thrift enum starts at 0, so we know this will not be used
+		UTF8:            ConvertedType(format.ConvertedType_UTF8),
+		Map:             ConvertedType(format.ConvertedType_MAP),
+		MapKeyValue:     ConvertedType(format.ConvertedType_MAP_KEY_VALUE),
+		List:            ConvertedType(format.ConvertedType_LIST),
+		Enum:            ConvertedType(format.ConvertedType_ENUM),
+		Decimal:         ConvertedType(format.ConvertedType_DECIMAL),
+		Date:            ConvertedType(format.ConvertedType_DATE),
+		TimeMillis:      ConvertedType(format.ConvertedType_TIME_MILLIS),
+		TimeMicros:      ConvertedType(format.ConvertedType_TIME_MICROS),
+		TimestampMillis: ConvertedType(format.ConvertedType_TIMESTAMP_MILLIS),
+		TimestampMicros: ConvertedType(format.ConvertedType_TIMESTAMP_MICROS),
+		Uint8:           ConvertedType(format.ConvertedType_UINT_8),
+		Uint16:          ConvertedType(format.ConvertedType_UINT_16),
+		Uint32:          ConvertedType(format.ConvertedType_UINT_32),
+		Uint64:          ConvertedType(format.ConvertedType_UINT_64),
+		Int8:            ConvertedType(format.ConvertedType_INT_8),
+		Int16:           ConvertedType(format.ConvertedType_INT_16),
+		Int32:           ConvertedType(format.ConvertedType_INT_32),
+		Int64:           ConvertedType(format.ConvertedType_INT_64),
+		JSON:            ConvertedType(format.ConvertedType_JSON),
+		BSON:            ConvertedType(format.ConvertedType_BSON),
+		Interval:        ConvertedType(format.ConvertedType_INTERVAL),
+		NA:              24, // should always be the last values after Interval
+	}
+)
+
+func (p ConvertedType) String() string {
+	switch p {
+	case ConvertedTypes.None:
+		return "NONE"
+	case ConvertedTypes.NA:
+		return "UNKNOWN"
+	default:
+		return format.ConvertedType(p).String()
+	}
+}
+
+// ToLogicalType returns the correct LogicalType for the given ConvertedType, using the decimal
+// metadata provided to define the precision/scale if necessary
+func (p ConvertedType) ToLogicalType(convertedDecimal DecimalMetadata) LogicalType {
+	switch p {
+	case ConvertedTypes.UTF8:
+		return StringLogicalType{}
+	case ConvertedTypes.Map, ConvertedTypes.MapKeyValue:
+		return MapLogicalType{}
+	case ConvertedTypes.List:
+		return ListLogicalType{}
+	case ConvertedTypes.Enum:
+		return EnumLogicalType{}
+	case ConvertedTypes.Decimal:
+		return NewDecimalLogicalType(convertedDecimal.Precision, convertedDecimal.Scale)
+	case ConvertedTypes.Date:
+		return DateLogicalType{}
+	case ConvertedTypes.TimeMillis:
+		return NewTimeLogicalType(true, TimeUnitMillis)

Review comment:
       add comments for literal values plase.

##########
File path: go/parquet/schema/converted_types.go
##########
@@ -0,0 +1,191 @@
+// 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.
+
+package schema
+
+import (
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// ConvertedType corresponds to the ConvertedType in the parquet.Thrift,
+// with added values of None and NA for handling when these values are not
+// set in the metadata
+type ConvertedType format.ConvertedType
+
+var (
+	// ConvertedTypes is a struct containing the constants for the types
+	// to make it easy to reference them while making it clear what they are
+	ConvertedTypes = struct {
+		None            ConvertedType
+		UTF8            ConvertedType
+		Map             ConvertedType
+		MapKeyValue     ConvertedType
+		List            ConvertedType
+		Enum            ConvertedType
+		Decimal         ConvertedType
+		Date            ConvertedType
+		TimeMillis      ConvertedType
+		TimeMicros      ConvertedType
+		TimestampMillis ConvertedType
+		TimestampMicros ConvertedType
+		Uint8           ConvertedType
+		Uint16          ConvertedType
+		Uint32          ConvertedType
+		Uint64          ConvertedType
+		Int8            ConvertedType
+		Int16           ConvertedType
+		Int32           ConvertedType
+		Int64           ConvertedType
+		JSON            ConvertedType
+		BSON            ConvertedType
+		Interval        ConvertedType
+		NA              ConvertedType
+	}{
+		None:            -1, // thrift enum starts at 0, so we know this will not be used
+		UTF8:            ConvertedType(format.ConvertedType_UTF8),
+		Map:             ConvertedType(format.ConvertedType_MAP),
+		MapKeyValue:     ConvertedType(format.ConvertedType_MAP_KEY_VALUE),
+		List:            ConvertedType(format.ConvertedType_LIST),
+		Enum:            ConvertedType(format.ConvertedType_ENUM),
+		Decimal:         ConvertedType(format.ConvertedType_DECIMAL),
+		Date:            ConvertedType(format.ConvertedType_DATE),
+		TimeMillis:      ConvertedType(format.ConvertedType_TIME_MILLIS),
+		TimeMicros:      ConvertedType(format.ConvertedType_TIME_MICROS),
+		TimestampMillis: ConvertedType(format.ConvertedType_TIMESTAMP_MILLIS),
+		TimestampMicros: ConvertedType(format.ConvertedType_TIMESTAMP_MICROS),
+		Uint8:           ConvertedType(format.ConvertedType_UINT_8),
+		Uint16:          ConvertedType(format.ConvertedType_UINT_16),
+		Uint32:          ConvertedType(format.ConvertedType_UINT_32),
+		Uint64:          ConvertedType(format.ConvertedType_UINT_64),
+		Int8:            ConvertedType(format.ConvertedType_INT_8),
+		Int16:           ConvertedType(format.ConvertedType_INT_16),
+		Int32:           ConvertedType(format.ConvertedType_INT_32),
+		Int64:           ConvertedType(format.ConvertedType_INT_64),
+		JSON:            ConvertedType(format.ConvertedType_JSON),
+		BSON:            ConvertedType(format.ConvertedType_BSON),
+		Interval:        ConvertedType(format.ConvertedType_INTERVAL),
+		NA:              24, // should always be the last values after Interval
+	}
+)
+
+func (p ConvertedType) String() string {
+	switch p {
+	case ConvertedTypes.None:
+		return "NONE"
+	case ConvertedTypes.NA:
+		return "UNKNOWN"
+	default:
+		return format.ConvertedType(p).String()
+	}
+}
+
+// ToLogicalType returns the correct LogicalType for the given ConvertedType, using the decimal
+// metadata provided to define the precision/scale if necessary
+func (p ConvertedType) ToLogicalType(convertedDecimal DecimalMetadata) LogicalType {
+	switch p {
+	case ConvertedTypes.UTF8:
+		return StringLogicalType{}
+	case ConvertedTypes.Map, ConvertedTypes.MapKeyValue:
+		return MapLogicalType{}
+	case ConvertedTypes.List:
+		return ListLogicalType{}
+	case ConvertedTypes.Enum:
+		return EnumLogicalType{}
+	case ConvertedTypes.Decimal:
+		return NewDecimalLogicalType(convertedDecimal.Precision, convertedDecimal.Scale)
+	case ConvertedTypes.Date:
+		return DateLogicalType{}
+	case ConvertedTypes.TimeMillis:
+		return NewTimeLogicalType(true, TimeUnitMillis)
+	case ConvertedTypes.TimeMicros:
+		return NewTimeLogicalType(true, TimeUnitMicros)
+	case ConvertedTypes.TimestampMillis:
+		t := NewTimestampLogicalType(true, TimeUnitMillis)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.TimestampMicros:
+		t := NewTimestampLogicalType(true, TimeUnitMicros)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.Interval:
+		return IntervalLogicalType{}
+	case ConvertedTypes.Int8:
+		return NewIntLogicalType(8, true)

Review comment:
       comments for literals.

##########
File path: go/parquet/schema/converted_types.go
##########
@@ -0,0 +1,191 @@
+// 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.
+
+package schema
+
+import (
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// ConvertedType corresponds to the ConvertedType in the parquet.Thrift,
+// with added values of None and NA for handling when these values are not
+// set in the metadata
+type ConvertedType format.ConvertedType
+
+var (
+	// ConvertedTypes is a struct containing the constants for the types
+	// to make it easy to reference them while making it clear what they are
+	ConvertedTypes = struct {
+		None            ConvertedType
+		UTF8            ConvertedType
+		Map             ConvertedType
+		MapKeyValue     ConvertedType
+		List            ConvertedType
+		Enum            ConvertedType
+		Decimal         ConvertedType
+		Date            ConvertedType
+		TimeMillis      ConvertedType
+		TimeMicros      ConvertedType
+		TimestampMillis ConvertedType
+		TimestampMicros ConvertedType
+		Uint8           ConvertedType
+		Uint16          ConvertedType
+		Uint32          ConvertedType
+		Uint64          ConvertedType
+		Int8            ConvertedType
+		Int16           ConvertedType
+		Int32           ConvertedType
+		Int64           ConvertedType
+		JSON            ConvertedType
+		BSON            ConvertedType
+		Interval        ConvertedType
+		NA              ConvertedType
+	}{
+		None:            -1, // thrift enum starts at 0, so we know this will not be used
+		UTF8:            ConvertedType(format.ConvertedType_UTF8),
+		Map:             ConvertedType(format.ConvertedType_MAP),
+		MapKeyValue:     ConvertedType(format.ConvertedType_MAP_KEY_VALUE),
+		List:            ConvertedType(format.ConvertedType_LIST),
+		Enum:            ConvertedType(format.ConvertedType_ENUM),
+		Decimal:         ConvertedType(format.ConvertedType_DECIMAL),
+		Date:            ConvertedType(format.ConvertedType_DATE),
+		TimeMillis:      ConvertedType(format.ConvertedType_TIME_MILLIS),
+		TimeMicros:      ConvertedType(format.ConvertedType_TIME_MICROS),
+		TimestampMillis: ConvertedType(format.ConvertedType_TIMESTAMP_MILLIS),
+		TimestampMicros: ConvertedType(format.ConvertedType_TIMESTAMP_MICROS),
+		Uint8:           ConvertedType(format.ConvertedType_UINT_8),
+		Uint16:          ConvertedType(format.ConvertedType_UINT_16),
+		Uint32:          ConvertedType(format.ConvertedType_UINT_32),
+		Uint64:          ConvertedType(format.ConvertedType_UINT_64),
+		Int8:            ConvertedType(format.ConvertedType_INT_8),
+		Int16:           ConvertedType(format.ConvertedType_INT_16),
+		Int32:           ConvertedType(format.ConvertedType_INT_32),
+		Int64:           ConvertedType(format.ConvertedType_INT_64),
+		JSON:            ConvertedType(format.ConvertedType_JSON),
+		BSON:            ConvertedType(format.ConvertedType_BSON),
+		Interval:        ConvertedType(format.ConvertedType_INTERVAL),
+		NA:              24, // should always be the last values after Interval
+	}
+)
+
+func (p ConvertedType) String() string {
+	switch p {
+	case ConvertedTypes.None:
+		return "NONE"
+	case ConvertedTypes.NA:
+		return "UNKNOWN"
+	default:
+		return format.ConvertedType(p).String()
+	}
+}
+
+// ToLogicalType returns the correct LogicalType for the given ConvertedType, using the decimal
+// metadata provided to define the precision/scale if necessary
+func (p ConvertedType) ToLogicalType(convertedDecimal DecimalMetadata) LogicalType {
+	switch p {
+	case ConvertedTypes.UTF8:
+		return StringLogicalType{}
+	case ConvertedTypes.Map, ConvertedTypes.MapKeyValue:
+		return MapLogicalType{}
+	case ConvertedTypes.List:
+		return ListLogicalType{}
+	case ConvertedTypes.Enum:
+		return EnumLogicalType{}
+	case ConvertedTypes.Decimal:
+		return NewDecimalLogicalType(convertedDecimal.Precision, convertedDecimal.Scale)
+	case ConvertedTypes.Date:
+		return DateLogicalType{}
+	case ConvertedTypes.TimeMillis:
+		return NewTimeLogicalType(true, TimeUnitMillis)
+	case ConvertedTypes.TimeMicros:
+		return NewTimeLogicalType(true, TimeUnitMicros)
+	case ConvertedTypes.TimestampMillis:
+		t := NewTimestampLogicalType(true, TimeUnitMillis)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.TimestampMicros:
+		t := NewTimestampLogicalType(true, TimeUnitMicros)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.Interval:
+		return IntervalLogicalType{}
+	case ConvertedTypes.Int8:
+		return NewIntLogicalType(8, true)
+	case ConvertedTypes.Int16:
+		return NewIntLogicalType(16, true)
+	case ConvertedTypes.Int32:
+		return NewIntLogicalType(32, true)
+	case ConvertedTypes.Int64:
+		return NewIntLogicalType(64, true)
+	case ConvertedTypes.Uint8:
+		return NewIntLogicalType(8, false)
+	case ConvertedTypes.Uint16:
+		return NewIntLogicalType(16, false)
+	case ConvertedTypes.Uint32:
+		return NewIntLogicalType(32, false)
+	case ConvertedTypes.Uint64:
+		return NewIntLogicalType(64, false)
+	case ConvertedTypes.JSON:
+		return JSONLogicalType{}
+	case ConvertedTypes.BSON:
+		return BSONLogicalType{}
+	case ConvertedTypes.None:
+		return NoLogicalType{}
+	case ConvertedTypes.NA:
+		fallthrough
+	default:
+		return UnknownLogicalType{}
+	}
+}
+
+// GetSortOrder defaults to the sort order based on the physical type if convert
+// is ConvertedTypes.None, otherwise determines the sort order by the converted type.
+func GetSortOrder(convert ConvertedType, primitive format.Type) SortOrder {
+	if convert == ConvertedTypes.None {
+		return DefaultSortOrder(primitive)
+	}
+	switch convert {
+	case ConvertedTypes.Int8,
+		ConvertedTypes.Int16,
+		ConvertedTypes.Int32,
+		ConvertedTypes.Int64,
+		ConvertedTypes.Date,
+		ConvertedTypes.TimeMicros,
+		ConvertedTypes.TimeMillis,
+		ConvertedTypes.TimestampMicros,
+		ConvertedTypes.TimestampMillis:
+		return SortSIGNED
+	case ConvertedTypes.Uint8,
+		ConvertedTypes.Uint16,
+		ConvertedTypes.Uint32,
+		ConvertedTypes.Uint64,
+		ConvertedTypes.Enum,
+		ConvertedTypes.UTF8,
+		ConvertedTypes.BSON,
+		ConvertedTypes.JSON:
+		return SortUNSIGNED
+	case ConvertedTypes.Decimal,

Review comment:
       this doesn't look right for decimal.  I believe it should be signed?  Or might vary based on underlying type.

##########
File path: go/parquet/schema/helpers.go
##########
@@ -0,0 +1,102 @@
+// 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.
+
+package schema
+
+import "github.com/apache/arrow/go/parquet"
+
+// ListOf is a convenience helper function to create a properly structured
+// list structure according to the Parquet Spec.
+//
+// <list-repetition> group <name> (LIST) {
+//   repeated group list {
+//     <element-repetition> <element-type> element;
+//   }
+// }
+//
+// <list-repetition> can only be optional or required. panics if repeated.
+// <element-repetition> can only be optional or required. panics if repeated.
+func ListOf(n Node, rep parquet.Repetition, fieldID int32) *GroupNode {
+	if rep == parquet.Repetitions.Repeated || n.RepetitionType() == parquet.Repetitions.Repeated {
+		panic("parquet: listof repetition and element repetition must not be repeated.")
+	}
+	listName := n.Name()
+
+	switch n := n.(type) {
+	case *PrimitiveNode:
+		n.name = "element"
+	case *GroupNode:
+		n.name = "element"
+	}
+
+	return NewGroupNodeLogical(listName, rep, FieldList{
+		NewGroupNode("list", parquet.Repetitions.Repeated, FieldList{n}, -1)},

Review comment:
       literal comment please.

##########
File path: go/parquet/schema/helpers.go
##########
@@ -0,0 +1,102 @@
+// 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.
+
+package schema
+
+import "github.com/apache/arrow/go/parquet"
+
+// ListOf is a convenience helper function to create a properly structured
+// list structure according to the Parquet Spec.
+//
+// <list-repetition> group <name> (LIST) {
+//   repeated group list {
+//     <element-repetition> <element-type> element;
+//   }
+// }
+//
+// <list-repetition> can only be optional or required. panics if repeated.
+// <element-repetition> can only be optional or required. panics if repeated.
+func ListOf(n Node, rep parquet.Repetition, fieldID int32) *GroupNode {
+	if rep == parquet.Repetitions.Repeated || n.RepetitionType() == parquet.Repetitions.Repeated {
+		panic("parquet: listof repetition and element repetition must not be repeated.")
+	}
+	listName := n.Name()
+
+	switch n := n.(type) {
+	case *PrimitiveNode:
+		n.name = "element"
+	case *GroupNode:
+		n.name = "element"
+	}
+
+	return NewGroupNodeLogical(listName, rep, FieldList{
+		NewGroupNode("list", parquet.Repetitions.Repeated, FieldList{n}, -1)},
+		ListLogicalType{}, fieldID)
+}
+
+// MapOf is a convenience helper function to create a properly structured
+// parquet map node setup according to the Parquet Spec.
+//
+// <map-repetition> group <name> (MAP) {
+// 	 repeated group key_value {
+// 	   required <key-type> key;
+//     <value-repetition> <value-type> value;
+//   }
+// }
+//
+// key node will be renamed to "key", value node if not nil will be renamed to "value"
+//
+// <map-repetition> must be only optional or required. panics if repeated is passed.

Review comment:
       Thanks for the comments.

##########
File path: go/parquet/schema/helpers.go
##########
@@ -0,0 +1,102 @@
+// 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.
+
+package schema
+
+import "github.com/apache/arrow/go/parquet"
+
+// ListOf is a convenience helper function to create a properly structured
+// list structure according to the Parquet Spec.
+//
+// <list-repetition> group <name> (LIST) {
+//   repeated group list {
+//     <element-repetition> <element-type> element;
+//   }
+// }
+//
+// <list-repetition> can only be optional or required. panics if repeated.
+// <element-repetition> can only be optional or required. panics if repeated.
+func ListOf(n Node, rep parquet.Repetition, fieldID int32) *GroupNode {
+	if rep == parquet.Repetitions.Repeated || n.RepetitionType() == parquet.Repetitions.Repeated {
+		panic("parquet: listof repetition and element repetition must not be repeated.")
+	}
+	listName := n.Name()
+
+	switch n := n.(type) {
+	case *PrimitiveNode:
+		n.name = "element"
+	case *GroupNode:
+		n.name = "element"
+	}
+
+	return NewGroupNodeLogical(listName, rep, FieldList{
+		NewGroupNode("list", parquet.Repetitions.Repeated, FieldList{n}, -1)},
+		ListLogicalType{}, fieldID)
+}
+
+// MapOf is a convenience helper function to create a properly structured
+// parquet map node setup according to the Parquet Spec.
+//
+// <map-repetition> group <name> (MAP) {
+// 	 repeated group key_value {
+// 	   required <key-type> key;
+//     <value-repetition> <value-type> value;
+//   }
+// }
+//
+// key node will be renamed to "key", value node if not nil will be renamed to "value"
+//
+// <map-repetition> must be only optional or required. panics if repeated is passed.
+//
+// the key node *must* be required repetition. panics if optional or repeated
+//
+// value node can be nil (omitted) or have a repetition of required or optional *only*.
+// panics if value node is not nil and has a repetition of repeated.
+func MapOf(name string, key Node, value Node, mapRep parquet.Repetition, fieldID int32) *GroupNode {
+	if mapRep == parquet.Repetitions.Repeated {
+		panic("parquet: map repetition cannot be Repeated")
+	}
+	if key.RepetitionType() != parquet.Repetitions.Required {
+		panic("parquet: map key repetition must be Required")
+	}
+	if value != nil {
+		if value.RepetitionType() == parquet.Repetitions.Repeated {
+			panic("parquet: map value cannot have repetition Repeated")
+		}
+		switch value := value.(type) {
+		case *PrimitiveNode:
+			value.name = "value"
+		case *GroupNode:
+			value.name = "value"
+		}
+	}
+
+	switch key := key.(type) {
+	case *PrimitiveNode:
+		key.name = "key"
+	case *GroupNode:
+		key.name = "key"
+	}
+
+	keyval := FieldList{key}
+	if value != nil {
+		keyval = append(keyval, value)
+	}
+
+	return NewGroupNodeLogical(name, mapRep, FieldList{
+		NewGroupNode("key_value", parquet.Repetitions.Repeated, keyval, -1),

Review comment:
       literal comment.

##########
File path: go/parquet/schema/schema_flatten_test.go
##########
@@ -0,0 +1,150 @@
+// 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.
+
+package schema
+
+import (
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/suite"
+)
+
+func NewPrimitive(name string, repetition format.FieldRepetitionType, typ format.Type, fieldID int32) *format.SchemaElement {
+	ret := &format.SchemaElement{
+		Name:           name,
+		RepetitionType: format.FieldRepetitionTypePtr(repetition),
+		Type:           format.TypePtr(typ),
+	}
+	if fieldID >= 0 {
+		ret.FieldID = &fieldID
+	}
+	return ret
+}
+
+func NewGroup(name string, repetition format.FieldRepetitionType, numChildren, fieldID int32) *format.SchemaElement {
+	ret := &format.SchemaElement{
+		Name:           name,
+		RepetitionType: format.FieldRepetitionTypePtr(repetition),
+		NumChildren:    &numChildren,
+	}
+	if fieldID >= 0 {
+		ret.FieldID = &fieldID
+	}
+	return ret
+}
+
+type SchemaFlattenSuite struct {
+	suite.Suite
+
+	name string
+}
+
+func (s *SchemaFlattenSuite) SetupSuite() {
+	s.name = "parquet_schema"
+}
+
+func (s *SchemaFlattenSuite) TestDecimalMetadata() {
+	group := NewGroupNodeConverted("group", parquet.Repetitions.Repeated, FieldList{
+		NewPrimitiveNodeConverted("decimal", parquet.Repetitions.Required, parquet.Types.Int64, ConvertedTypes.Decimal, 0, 8, 4, -1),

Review comment:
       literal comments here would definitely help readability.

##########
File path: go/parquet/schema/reflection_test.go
##########
@@ -0,0 +1,397 @@
+// 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.
+
+package schema_test
+
+import (
+	"log"
+	"os"
+	"reflect"
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/schema"
+	"github.com/stretchr/testify/assert"
+)
+
+func ExampleNewSchemaFromStruct_primitives() {
+	type Schema struct {
+		Bool              bool
+		Int8              int8
+		Uint16            uint16
+		Int32             int32
+		Int64             int64
+		Int96             parquet.Int96
+		Float             float32
+		Double            float64
+		ByteArray         string
+		FixedLenByteArray [10]byte
+	}
+
+	sc, err := schema.NewSchemaFromStruct(Schema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 Schema {
+	//   required boolean field_id=-1 Bool;
+	//   required int32 field_id=-1 Int8 (Int(bitWidth=8, isSigned=true));
+	//   required int32 field_id=-1 Uint16 (Int(bitWidth=16, isSigned=false));
+	//   required int32 field_id=-1 Int32 (Int(bitWidth=32, isSigned=true));
+	//   required int64 field_id=-1 Int64 (Int(bitWidth=64, isSigned=true));
+	//   required int96 field_id=-1 Int96;
+	//   required float field_id=-1 Float;
+	//   required double field_id=-1 Double;
+	//   required byte_array field_id=-1 ByteArray;
+	//   required fixed_len_byte_array field_id=-1 FixedLenByteArray;
+	// }
+}
+
+func ExampleNewSchemaFromStruct_convertedtypes() {
+	type ConvertedSchema struct {
+		Utf8           string        `parquet:"name=utf8, converted=UTF8"`
+		Uint32         uint32        `parquet:"converted=INT_32"`
+		Date           int32         `parquet:"name=date, converted=date"`
+		TimeMilli      int32         `parquet:"name=timemilli, converted=TIME_MILLIS"`
+		TimeMicro      int64         `parquet:"name=timemicro, converted=time_micros"`
+		TimeStampMilli int64         `parquet:"converted=timestamp_millis"`
+		TimeStampMicro int64         `parquet:"converted=timestamp_micros"`
+		Interval       parquet.Int96 `parquet:"converted=INTERVAL"`
+		Decimal1       int32         `parquet:"converted=decimal, scale=2, precision=9"`
+		Decimal2       int64         `parquet:"converted=decimal, scale=2, precision=18"`
+		Decimal3       [12]byte      `parquet:"converted=decimal, scale=2, precision=10"`
+		Decimal4       string        `parquet:"converted=decimal, scale=2, precision=20"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(&ConvertedSchema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 ConvertedSchema {
+	//   required byte_array field_id=-1 utf8 (String);
+	//   required int32 field_id=-1 Uint32 (Int(bitWidth=32, isSigned=true));
+	//   required int32 field_id=-1 date (Date);
+	//   required int32 field_id=-1 timemilli (Time(isAdjustedToUTC=true, timeUnit=milliseconds));
+	//   required int64 field_id=-1 timemicro (Time(isAdjustedToUTC=true, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeStampMilli (Timestamp(isAdjustedToUTC=true, timeUnit=milliseconds, is_from_converted_type=true, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimeStampMicro (Timestamp(isAdjustedToUTC=true, timeUnit=microseconds, is_from_converted_type=true, force_set_converted_type=false));
+	//   required int96 field_id=-1 Interval;
+	//   required int32 field_id=-1 Decimal1 (Decimal(precision=9, scale=2));
+	//   required int64 field_id=-1 Decimal2 (Decimal(precision=18, scale=2));
+	//   required fixed_len_byte_array field_id=-1 Decimal3 (Decimal(precision=10, scale=2));
+	//   required byte_array field_id=-1 Decimal4 (Decimal(precision=20, scale=2));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_repetition() {
+	type RepetitionSchema struct {
+		List     []int64 `parquet:"fieldid=1"`
+		Repeated []int64 `parquet:"repetition=repeated, fieldid=2"`
+		Optional *int64  `parquet:"fieldid=3"`
+		Required *int64  `parquet:"repetition=REQUIRED, fieldid=4"`
+		Opt      int64   `parquet:"repetition=OPTIONAL, fieldid=5"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(RepetitionSchema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 RepetitionSchema {
+	//   required group field_id=1 List (List) {
+	//     repeated group field_id=-1 list {
+	//       required int64 field_id=-1 element (Int(bitWidth=64, isSigned=true));
+	//     }
+	//   }
+	//   repeated int64 field_id=2 Repeated (Int(bitWidth=64, isSigned=true));
+	//   optional int64 field_id=3 Optional (Int(bitWidth=64, isSigned=true));
+	//   required int64 field_id=4 Required (Int(bitWidth=64, isSigned=true));
+	//   optional int64 field_id=5 Opt (Int(bitWidth=64, isSigned=true));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_logicaltypes() {
+	type LogicalTypes struct {
+		String                []byte   `parquet:"logical=String"`
+		Enum                  string   `parquet:"logical=enum"`
+		Date                  int32    `parquet:"logical=date"`
+		Decimal1              int32    `parquet:"logical=decimal, precision=9, scale=2"`
+		Decimal2              int32    `parquet:"logical=decimal, logical.precision=9, scale=2"`
+		Decimal3              int32    `parquet:"logical=decimal, precision=5, logical.precision=9, scale=1, logical.scale=3"`
+		TimeMilliUTC          int32    `parquet:"logical=TIME, logical.unit=millis"`
+		TimeMilli             int32    `parquet:"logical=Time, logical.unit=millis, logical.isadjustedutc=false"`
+		TimeMicros            int64    `parquet:"logical=time, logical.unit=micros, logical.isadjustedutc=false"`
+		TimeMicrosUTC         int64    `parquet:"logical=time, logical.unit=micros, logical.isadjustedutc=true"`
+		TimeNanos             int64    `parquet:"logical=time, logical.unit=nanos"`
+		TimestampMilli        int64    `parquet:"logical=timestamp, logical.unit=millis"`
+		TimestampMicrosNotUTC int64    `parquet:"logical=timestamp, logical.unit=micros, logical.isadjustedutc=false"`
+		TimestampNanos        int64    `parquet:"logical=timestamp, logical.unit=nanos"`
+		JSON                  string   `parquet:"logical=json"`
+		BSON                  []byte   `parquet:"logical=BSON"`
+		UUID                  [16]byte `parquet:"logical=uuid"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(LogicalTypes{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 LogicalTypes {
+	//   required byte_array field_id=-1 String (String);
+	//   required byte_array field_id=-1 Enum (Enum);
+	//   required int32 field_id=-1 Date (Date);
+	//   required int32 field_id=-1 Decimal1 (Decimal(precision=9, scale=2));
+	//   required int32 field_id=-1 Decimal2 (Decimal(precision=9, scale=2));
+	//   required int32 field_id=-1 Decimal3 (Decimal(precision=9, scale=3));
+	//   required int32 field_id=-1 TimeMilliUTC (Time(isAdjustedToUTC=true, timeUnit=milliseconds));
+	//   required int32 field_id=-1 TimeMilli (Time(isAdjustedToUTC=false, timeUnit=milliseconds));
+	//   required int64 field_id=-1 TimeMicros (Time(isAdjustedToUTC=false, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeMicrosUTC (Time(isAdjustedToUTC=true, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeNanos (Time(isAdjustedToUTC=true, timeUnit=nanoseconds));
+	//   required int64 field_id=-1 TimestampMilli (Timestamp(isAdjustedToUTC=true, timeUnit=milliseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimestampMicrosNotUTC (Timestamp(isAdjustedToUTC=false, timeUnit=microseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimestampNanos (Timestamp(isAdjustedToUTC=true, timeUnit=nanoseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required byte_array field_id=-1 JSON (JSON);
+	//   required byte_array field_id=-1 BSON (BSON);
+	//   required fixed_len_byte_array field_id=-1 UUID (UUID);
+	// }
+}
+
+func ExampleNewSchemaFromStruct_physicaltype() {
+	type ChangeTypes struct {
+		Int32        int64  `parquet:"type=int32"`
+		FixedLen     string `parquet:"type=fixed_len_byte_array, length=10"`
+		SliceAsFixed []byte `parquet:"type=fixed_len_byte_array, length=12"`
+		Int          int    `parquet:"type=int32"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(ChangeTypes{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 ChangeTypes {
+	//   required int32 field_id=-1 Int32 (Int(bitWidth=32, isSigned=true));
+	//   required fixed_len_byte_array field_id=-1 FixedLen;
+	//   required fixed_len_byte_array field_id=-1 SliceAsFixed;
+	//   required int32 field_id=-1 Int (Int(bitWidth=32, isSigned=true));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_nestedtypes() {
+	type Other struct {
+		OptionalMap *map[string]*string `parquet:"valuerepetition=required, keylogical=String, valueconverted=BSON"`
+	}
+
+	type MyMap map[int32]string
+
+	type Nested struct {
+		SimpleMap     map[int32]string
+		FixedLenMap   map[string][]byte `parquet:"keytype=fixed_len_byte_array, keyfieldid=10, valuefieldid=11, keylength=10"`
+		DecimalMap    map[int32]string  `parquet:"logical=map, keyconverted=DECIMAL, keyscale=3, keyprecision=7, valuetype=fixed_len_byte_array, valuelength=4, valuelogical=decimal, valuelogical.precision=9, valuescale=2"`
+		OtherList     []*Other
+		OtherRepeated []Other  `parquet:"repetition=repeated"`
+		DateArray     [5]int32 `parquet:"valuelogical=date, logical=list"`
+		DateMap       MyMap    `parquet:"keylogical=TIME, keylogical.unit=MILLIS, keylogical.isadjustedutc=false, valuelogical=enum"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(Nested{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 Nested {

Review comment:
       did you want to make these assertion in this file?

##########
File path: go/parquet/schema/logical_types_test.go
##########
@@ -0,0 +1,551 @@
+// 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.
+
+package schema_test
+
+import (
+	"encoding/json"
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/schema"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestConvertedLogicalEquivalences(t *testing.T) {
+	tests := []struct {
+		name      string
+		converted schema.ConvertedType
+		logical   schema.LogicalType
+		expected  schema.LogicalType
+	}{
+		{"utf8", schema.ConvertedTypes.UTF8, schema.StringLogicalType{}, schema.StringLogicalType{}},
+		{"map", schema.ConvertedTypes.Map, schema.MapLogicalType{}, schema.MapLogicalType{}},
+		{"mapkeyval", schema.ConvertedTypes.MapKeyValue, schema.MapLogicalType{}, schema.MapLogicalType{}},
+		{"list", schema.ConvertedTypes.List, schema.NewListLogicalType(), schema.NewListLogicalType()},
+		{"enum", schema.ConvertedTypes.Enum, schema.EnumLogicalType{}, schema.EnumLogicalType{}},
+		{"date", schema.ConvertedTypes.Date, schema.DateLogicalType{}, schema.DateLogicalType{}},
+		{"timemilli", schema.ConvertedTypes.TimeMillis, schema.NewTimeLogicalType(true, schema.TimeUnitMillis), &schema.TimeLogicalType{}},
+		{"timemicro", schema.ConvertedTypes.TimeMicros, schema.NewTimeLogicalType(true, schema.TimeUnitMicros), &schema.TimeLogicalType{}},
+		{"timestampmilli", schema.ConvertedTypes.TimestampMillis, schema.NewTimestampLogicalType(true, schema.TimeUnitMillis), &schema.TimestampLogicalType{}},
+		{"timestampmicro", schema.ConvertedTypes.TimestampMicros, schema.NewTimestampLogicalType(true, schema.TimeUnitMicros), &schema.TimestampLogicalType{}},
+		{"uint8", schema.ConvertedTypes.Uint8, schema.NewIntLogicalType(8, false), &schema.IntLogicalType{}},
+		{"uint16", schema.ConvertedTypes.Uint16, schema.NewIntLogicalType(16, false), &schema.IntLogicalType{}},
+		{"uint32", schema.ConvertedTypes.Uint32, schema.NewIntLogicalType(32, false), &schema.IntLogicalType{}},
+		{"uint64", schema.ConvertedTypes.Uint64, schema.NewIntLogicalType(64, false), &schema.IntLogicalType{}},
+		{"int8", schema.ConvertedTypes.Int8, schema.NewIntLogicalType(8, true), &schema.IntLogicalType{}},

Review comment:
       literal comments would be helpful here as well.

##########
File path: go/parquet/schema/node.go
##########
@@ -0,0 +1,591 @@
+// 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.
+
+package schema
+
+import (
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/thrift/lib/go/thrift"
+	"golang.org/x/xerrors"
+)
+
+// NodeType describes whether the Node is a Primitive or Group node
+type NodeType int
+
+// the available constants for NodeType
+const (
+	Primitive NodeType = iota
+	Group
+)
+
+// Node is the interface for both Group and Primitive Nodes.
+// A logical schema type has a name, repetition level, and optionally
+// a logical type (ConvertedType in Parquet metadata parlance)

Review comment:
       nit: ConvertedType and LogicalType are diffent.  convertedType is the deprecated version of the concept that needs to be maintained for forward compatibility.

##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval

Review comment:
       why can't interval be serialized?  is this generated code?

##########
File path: go/parquet/schema/reflection.go
##########
@@ -0,0 +1,791 @@
+// 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.
+
+package schema
+
+import (
+	"reflect"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+type taggedInfo struct {
+	Name string
+
+	Type      parquet.Type
+	KeyType   parquet.Type
+	ValueType parquet.Type
+
+	Length      int32
+	KeyLength   int32
+	ValueLength int32
+
+	Scale      int32
+	KeyScale   int32
+	ValueScale int32
+
+	Precision      int32
+	KeyPrecision   int32
+	ValuePrecision int32
+
+	FieldID      int32
+	KeyFieldID   int32
+	ValueFieldID int32
+
+	RepetitionType  parquet.Repetition
+	ValueRepetition parquet.Repetition
+
+	Converted      ConvertedType
+	KeyConverted   ConvertedType
+	ValueConverted ConvertedType
+
+	LogicalFields      map[string]string
+	KeyLogicalFields   map[string]string
+	ValueLogicalFields map[string]string
+
+	LogicalType      LogicalType
+	KeyLogicalType   LogicalType
+	ValueLogicalType LogicalType
+}
+
+func (t *taggedInfo) CopyForKey() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.KeyType
+	ret.Length = t.KeyLength
+	ret.Scale = t.KeyScale
+	ret.Precision = t.KeyPrecision
+	ret.FieldID = t.KeyFieldID
+	ret.RepetitionType = parquet.Repetitions.Required
+	ret.Converted = t.KeyConverted
+	ret.LogicalType = t.KeyLogicalType
+	return
+}
+
+func (t *taggedInfo) CopyForValue() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.ValueType
+	ret.Length = t.ValueLength
+	ret.Scale = t.ValueScale
+	ret.Precision = t.ValuePrecision
+	ret.FieldID = t.ValueFieldID
+	ret.RepetitionType = t.ValueRepetition
+	ret.Converted = t.ValueConverted
+	ret.LogicalType = t.ValueLogicalType
+	return
+}
+
+func (t *taggedInfo) UpdateLogicalTypes() {
+	processLogicalType := func(fields map[string]string, precision, scale int32) LogicalType {
+		t, ok := fields["type"]
+		if !ok {
+			return NoLogicalType{}
+		}
+
+		switch strings.ToLower(t) {
+		case "string":
+			return StringLogicalType{}
+		case "map":
+			return MapLogicalType{}
+		case "list":
+			return ListLogicalType{}
+		case "enum":
+			return EnumLogicalType{}
+		case "decimal":
+			if v, ok := fields["precision"]; ok {
+				precision = int32FromType(v)
+			}
+			if v, ok := fields["scale"]; ok {
+				scale = int32FromType(v)
+			}
+			return NewDecimalLogicalType(precision, scale)
+		case "date":
+			return DateLogicalType{}
+		case "time":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimeLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(strings.ToLower(unit)))
+		case "timestamp":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimestampLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(unit))
+		case "integer":
+			width, ok := fields["bitwidth"]
+			if !ok {
+				panic("must specify bitwidth if explicitly setting integer logical type")
+			}
+			signed, ok := fields["signed"]
+			if !ok {
+				signed = "true"
+			}
+
+			return NewIntLogicalType(int8(int32FromType(width)), boolFromStr(signed))
+		case "null":
+			return NullLogicalType{}
+		case "json":
+			return JSONLogicalType{}
+		case "bson":
+			return BSONLogicalType{}
+		case "uuid":
+			return UUIDLogicalType{}
+		default:
+			panic(xerrors.Errorf("invalid logical type specified: %s", t))
+		}
+	}
+
+	t.LogicalType = processLogicalType(t.LogicalFields, t.Precision, t.Scale)
+	t.KeyLogicalType = processLogicalType(t.KeyLogicalFields, t.KeyPrecision, t.KeyScale)
+	t.ValueLogicalType = processLogicalType(t.ValueLogicalFields, t.ValuePrecision, t.ValueScale)
+}
+
+func newTaggedInfo() taggedInfo {
+	return taggedInfo{
+		Type:               parquet.Types.Undefined,
+		KeyType:            parquet.Types.Undefined,
+		ValueType:          parquet.Types.Undefined,
+		RepetitionType:     parquet.Repetitions.Undefined,
+		ValueRepetition:    parquet.Repetitions.Undefined,
+		Converted:          ConvertedTypes.NA,
+		KeyConverted:       ConvertedTypes.NA,
+		ValueConverted:     ConvertedTypes.NA,
+		FieldID:            -1,
+		KeyFieldID:         -1,
+		ValueFieldID:       -1,
+		LogicalFields:      make(map[string]string),
+		KeyLogicalFields:   make(map[string]string),
+		ValueLogicalFields: make(map[string]string),
+		LogicalType:        NoLogicalType{},
+		KeyLogicalType:     NoLogicalType{},
+		ValueLogicalType:   NoLogicalType{},
+	}
+}
+
+var int32FromType = func(v string) int32 {
+	val, err := strconv.Atoi(v)
+	if err != nil {
+		panic(err)
+	}
+	return int32(val)
+}
+
+var boolFromStr = func(v string) bool {
+	val, err := strconv.ParseBool(v)
+	if err != nil {
+		panic(err)
+	}
+	return val
+}
+
+func infoFromTags(f reflect.StructTag) *taggedInfo {
+	typeFromStr := func(v string) parquet.Type {
+		t, err := format.TypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(xerrors.Errorf("invalid type specified: %s", v))
+		}
+		return parquet.Type(t)
+	}
+
+	repFromStr := func(v string) parquet.Repetition {
+		r, err := format.FieldRepetitionTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return parquet.Repetition(r)
+	}
+
+	convertedFromStr := func(v string) ConvertedType {
+		c, err := format.ConvertedTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return ConvertedType(c)
+	}
+
+	if ptags, ok := f.Lookup("parquet"); ok {
+		info := newTaggedInfo()
+		for _, tag := range strings.Split(strings.Replace(ptags, "\t", "", -1), ",") {
+			tag = strings.TrimSpace(tag)
+			kv := strings.SplitN(tag, "=", 2)
+			key := strings.TrimSpace(strings.ToLower(kv[0]))
+			value := strings.TrimSpace(kv[1])
+
+			switch key {
+			case "name":
+				info.Name = value
+			case "type":
+				info.Type = typeFromStr(value)
+			case "keytype":
+				info.KeyType = typeFromStr(value)
+			case "valuetype":
+				info.ValueType = typeFromStr(value)
+			case "length":
+				info.Length = int32FromType(value)
+			case "keylength":
+				info.KeyLength = int32FromType(value)
+			case "valuelength":
+				info.ValueLength = int32FromType(value)
+			case "scale":
+				info.Scale = int32FromType(value)
+			case "keyscale":
+				info.KeyScale = int32FromType(value)
+			case "valuescale":
+				info.ValueScale = int32FromType(value)
+			case "precision":
+				info.Precision = int32FromType(value)
+			case "keyprecision":
+				info.KeyPrecision = int32FromType(value)
+			case "valueprecision":
+				info.ValuePrecision = int32FromType(value)
+			case "fieldid":
+				info.FieldID = int32FromType(value)
+			case "keyfieldid":
+				info.KeyFieldID = int32FromType(value)
+			case "valuefieldid":
+				info.ValueFieldID = int32FromType(value)
+			case "repetition":
+				info.RepetitionType = repFromStr(value)
+			case "valuerepetition":
+				info.ValueRepetition = repFromStr(value)
+			case "converted":
+				info.Converted = convertedFromStr(value)
+			case "keyconverted":
+				info.KeyConverted = convertedFromStr(value)
+			case "valueconverted":
+				info.ValueConverted = convertedFromStr(value)
+			case "logical":
+				info.LogicalFields["type"] = value
+			case "keylogical":
+				info.KeyLogicalFields["type"] = value
+			case "valuelogical":
+				info.ValueLogicalFields["type"] = value
+			default:
+				switch {
+				case strings.HasPrefix(key, "logical."):
+					info.LogicalFields[strings.TrimPrefix(key, "logical.")] = value
+				case strings.HasPrefix(key, "keylogical."):
+					info.KeyLogicalFields[strings.TrimPrefix(key, "keylogical.")] = value
+				case strings.HasPrefix(key, "valuelogical."):
+					info.ValueLogicalFields[strings.TrimPrefix(key, "valuelogical.")] = value
+				}
+			}
+		}
+		info.UpdateLogicalTypes()
+		return &info
+	}
+	return nil
+}
+
+func typeToNode(name string, typ reflect.Type, repType parquet.Repetition, info *taggedInfo) Node {

Review comment:
       even though this is private it looks like not trivial logic  some docs could be useful.

##########
File path: go/parquet/schema/node.go
##########
@@ -0,0 +1,591 @@
+// 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.
+
+package schema
+
+import (
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/thrift/lib/go/thrift"
+	"golang.org/x/xerrors"
+)
+
+// NodeType describes whether the Node is a Primitive or Group node
+type NodeType int
+
+// the available constants for NodeType
+const (
+	Primitive NodeType = iota
+	Group
+)
+
+// Node is the interface for both Group and Primitive Nodes.
+// A logical schema type has a name, repetition level, and optionally
+// a logical type (ConvertedType in Parquet metadata parlance)
+type Node interface {
+	Name() string
+	Type() NodeType
+	RepetitionType() parquet.Repetition
+	ConvertedType() ConvertedType
+	LogicalType() LogicalType
+	FieldID() int32
+	Parent() Node
+	SetParent(Node)
+	Path() string
+	Equals(Node) bool
+	Visit(v Visitor)
+	toThrift() *format.SchemaElement
+}
+
+// Visitor is an interface for creating functionality to walk the schema tree.
+//
+// A visitor can be passed to the Visit function of a Node in order to walk
+// the tree. VisitPre is called the first time a node is encountered. If
+// it is a group node, the return is checked and if it is false, the children
+// will be skipped.
+//
+// VisitPost is called after visiting any children
+type Visitor interface {
+	VisitPre(Node) bool
+	VisitPost(Node)
+}
+
+// ColumnPathFromNode walks the parents of the given node to construct it's
+// column path
+func ColumnPathFromNode(n Node) parquet.ColumnPath {
+	if n == nil {
+		return nil
+	}
+
+	c := make([]string, 0)
+
+	cursor := n
+	for cursor.Parent() != nil {
+		c = append(c, cursor.Name())
+		cursor = cursor.Parent()
+	}
+
+	for i := len(c)/2 - 1; i >= 0; i-- {

Review comment:
       this logic is a little opaque to me, could just be my limited experience with go.

##########
File path: go/parquet/schema/schema_element_test.go
##########
@@ -0,0 +1,432 @@
+// 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.
+
+package schema
+
+import (
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/suite"
+)
+
+type schemaElementConstruction struct {
+	node            Node
+	element         *format.SchemaElement
+	name            string
+	expectConverted bool
+	converted       ConvertedType
+	expectLogical   bool
+	checkLogical    func(*format.SchemaElement) bool
+}
+
+type decimalSchemaElementConstruction struct {
+	schemaElementConstruction
+	precision int
+	scale     int
+}
+
+type temporalSchemaElementConstruction struct {
+	schemaElementConstruction
+	adjusted bool
+	unit     TimeUnitType
+	getUnit  func(*format.SchemaElement) *format.TimeUnit
+}
+
+type intSchemaElementConstruction struct {
+	schemaElementConstruction
+	width  int8
+	signed bool
+}
+
+type legacySchemaElementConstructArgs struct {
+	name            string
+	physical        parquet.Type
+	len             int
+	expectConverted bool
+	converted       ConvertedType
+	expectLogical   bool
+	checkLogical    func(*format.SchemaElement) bool
+}
+
+type schemaElementConstructArgs struct {
+	name            string
+	logical         LogicalType
+	physical        parquet.Type
+	len             int
+	expectConverted bool
+	converted       ConvertedType
+	expectLogical   bool
+	checkLogical    func(*format.SchemaElement) bool
+}
+type SchemaElementConstructionSuite struct {
+	suite.Suite
+}
+
+func (s *SchemaElementConstructionSuite) reconstruct(c schemaElementConstructArgs) *schemaElementConstruction {
+	ret := &schemaElementConstruction{
+		node:            NewPrimitiveNodeLogical(c.name, parquet.Repetitions.Required, c.logical, c.physical, c.len, -1),
+		name:            c.name,
+		expectConverted: c.expectConverted,
+		converted:       c.converted,
+		expectLogical:   c.expectLogical,
+		checkLogical:    c.checkLogical,
+	}
+	ret.element = ret.node.toThrift()
+	return ret
+}
+
+func (s *SchemaElementConstructionSuite) legacyReconstruct(c legacySchemaElementConstructArgs) *schemaElementConstruction {
+	ret := &schemaElementConstruction{
+		node:            NewPrimitiveNodeConverted(c.name, parquet.Repetitions.Required, c.physical, c.converted, c.len, 0, 0, -1),
+		name:            c.name,
+		expectConverted: c.expectConverted,
+		converted:       c.converted,
+		expectLogical:   c.expectLogical,
+		checkLogical:    c.checkLogical,
+	}
+	ret.element = ret.node.toThrift()
+	return ret
+}
+
+func (s *SchemaElementConstructionSuite) inspect(c *schemaElementConstruction) {
+	if c.expectConverted {
+		s.True(c.element.IsSetConvertedType())
+		s.Equal(c.converted, ConvertedType(*c.element.ConvertedType))
+	} else {
+		s.False(c.element.IsSetConvertedType())
+	}
+	if c.expectLogical {
+		s.True(c.element.IsSetLogicalType())
+		s.True(c.checkLogical(c.element))
+	} else {
+		s.False(c.element.IsSetLogicalType())
+	}
+}
+
+func (s *SchemaElementConstructionSuite) TestSimple() {
+	checkNone := func(*format.SchemaElement) bool { return true }
+
+	tests := []struct {
+		name   string
+		args   *schemaElementConstructArgs
+		legacy *legacySchemaElementConstructArgs
+	}{
+		{"string", &schemaElementConstructArgs{
+			"string", StringLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.UTF8, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetSTRING() },
+		}, nil},
+		{"enum", &schemaElementConstructArgs{
+			"enum", EnumLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.Enum, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetENUM() },
+		}, nil},
+		{"date", &schemaElementConstructArgs{
+			"date", DateLogicalType{}, parquet.Types.Int32, -1, true, ConvertedTypes.Date, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetDATE() },
+		}, nil},
+		{"interval", &schemaElementConstructArgs{
+			"interval", IntervalLogicalType{}, parquet.Types.FixedLenByteArray, 12, true, ConvertedTypes.Interval, false,
+			checkNone,
+		}, nil},
+		{"null", &schemaElementConstructArgs{
+			"null", NullLogicalType{}, parquet.Types.Double, -1, false, ConvertedTypes.NA, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetUNKNOWN() },
+		}, nil},
+		{"json", &schemaElementConstructArgs{
+			"json", JSONLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.JSON, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetJSON() },
+		}, nil},
+		{"bson", &schemaElementConstructArgs{
+			"bson", BSONLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.BSON, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetBSON() },
+		}, nil},
+		{"uuid", &schemaElementConstructArgs{
+			"uuid", UUIDLogicalType{}, parquet.Types.FixedLenByteArray, 16, false, ConvertedTypes.NA, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetUUID() },
+		}, nil},
+		{"none", &schemaElementConstructArgs{
+			"none", NoLogicalType{}, parquet.Types.Int64, -1, false, ConvertedTypes.NA, false,
+			checkNone,
+		}, nil},
+		{"unknown", &schemaElementConstructArgs{
+			"unknown", UnknownLogicalType{}, parquet.Types.Int64, -1, true, ConvertedTypes.NA, false,
+			checkNone,
+		}, nil},
+		{"timestamp_ms", nil, &legacySchemaElementConstructArgs{
+			"timestamp_ms", parquet.Types.Int64, -1, true, ConvertedTypes.TimestampMillis, false, checkNone}},
+		{"timestamp_us", nil, &legacySchemaElementConstructArgs{
+			"timestamp_us", parquet.Types.Int64, -1, true, ConvertedTypes.TimestampMicros, false, checkNone}},
+	}
+	for _, tt := range tests {
+		s.Run(tt.name, func() {
+			var sc *schemaElementConstruction
+			if tt.args != nil {
+				sc = s.reconstruct(*tt.args)
+			} else {
+				sc = s.legacyReconstruct(*tt.legacy)
+			}
+			s.Equal(tt.name, sc.element.Name)
+			s.inspect(sc)
+		})
+	}
+}
+
+func (s *SchemaElementConstructionSuite) reconstructDecimal(c schemaElementConstructArgs) *decimalSchemaElementConstruction {
+	ret := s.reconstruct(c)
+	dec := c.logical.(*DecimalLogicalType)
+	return &decimalSchemaElementConstruction{*ret, int(dec.Precision()), int(dec.Scale())}
+}
+
+func (s *SchemaElementConstructionSuite) inspectDecimal(d *decimalSchemaElementConstruction) {
+	s.inspect(&d.schemaElementConstruction)
+	s.EqualValues(d.precision, d.element.GetPrecision())
+	s.EqualValues(d.scale, d.element.GetScale())
+	s.EqualValues(d.precision, d.element.LogicalType.DECIMAL.Precision)
+	s.EqualValues(d.scale, d.element.LogicalType.DECIMAL.Scale)
+}
+
+func (s *SchemaElementConstructionSuite) TestDecimal() {
+	checkDecimal := func(p *format.SchemaElement) bool { return p.LogicalType.IsSetDECIMAL() }
+
+	tests := []schemaElementConstructArgs{
+		{"decimal16_6", NewDecimalLogicalType(16, 6), parquet.Types.Int64, -1, true, ConvertedTypes.Decimal, true, checkDecimal},
+		{"decimal1_0", NewDecimalLogicalType(1, 0), parquet.Types.Int32, -1, true, ConvertedTypes.Decimal, true, checkDecimal},
+		{"decimal10", NewDecimalLogicalType(10, 0), parquet.Types.Int64, -1, true, ConvertedTypes.Decimal, true, checkDecimal},

Review comment:
       comment for -1 could make this easier to read.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield closed pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield closed pull request #10071:
URL: https://github.com/apache/arrow/pull/10071


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r617009627



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {
+			return true
+		}
+		n = n.Parent()
+	}
+	return false
+}
+
+// NewColumn returns a new column object for the given node with the provided
+// maximum definition and repetition levels.
+//
+// n MUST be a PrimitiveNode, otherwise this will panic.
+func NewColumn(n Node, maxDefinitionLvl, maxRepetitionLvl int16) *Column {
+	if n.Type() != Primitive {
+		panic("parquet: Column must be a primitive type")
+	}
+	return &Column{n.(*PrimitiveNode), maxDefinitionLvl, maxRepetitionLvl}
+}
+
+// Name is the column's name
+func (c *Column) Name() string { return c.pnode.Name() }
+
+// ColumnPath returns the full path to this column from the root of the schema
+func (c *Column) ColumnPath() parquet.ColumnPath { return c.pnode.columnPath() }
+
+// Path is equivalent to ColumnPath().String() returning the dot-string version of the path
+func (c *Column) Path() string { return c.pnode.Path() }
+
+// TypeLength is -1 if not a FixedLenByteArray, otherwise it is the length for the column

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619883110



##########
File path: go/parquet/schema/helpers.go
##########
@@ -0,0 +1,102 @@
+// 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.
+
+package schema
+
+import "github.com/apache/arrow/go/parquet"
+
+// ListOf is a convenience helper function to create a properly structured
+// list structure according to the Parquet Spec.
+//
+// <list-repetition> group <name> (LIST) {
+//   repeated group list {
+//     <element-repetition> <element-type> element;
+//   }
+// }
+//
+// <list-repetition> can only be optional or required. panics if repeated.
+// <element-repetition> can only be optional or required. panics if repeated.
+func ListOf(n Node, rep parquet.Repetition, fieldID int32) *GroupNode {
+	if rep == parquet.Repetitions.Repeated || n.RepetitionType() == parquet.Repetitions.Repeated {
+		panic("parquet: listof repetition and element repetition must not be repeated.")
+	}
+	listName := n.Name()
+
+	switch n := n.(type) {
+	case *PrimitiveNode:
+		n.name = "element"
+	case *GroupNode:
+		n.name = "element"
+	}
+
+	return NewGroupNodeLogical(listName, rep, FieldList{
+		NewGroupNode("list", parquet.Repetitions.Repeated, FieldList{n}, -1)},
+		ListLogicalType{}, fieldID)
+}
+
+// MapOf is a convenience helper function to create a properly structured
+// parquet map node setup according to the Parquet Spec.
+//
+// <map-repetition> group <name> (MAP) {
+// 	 repeated group key_value {
+// 	   required <key-type> key;
+//     <value-repetition> <value-type> value;
+//   }
+// }
+//
+// key node will be renamed to "key", value node if not nil will be renamed to "value"
+//
+// <map-repetition> must be only optional or required. panics if repeated is passed.

Review comment:
       Thanks for the comments.

##########
File path: go/parquet/schema/helpers.go
##########
@@ -0,0 +1,102 @@
+// 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.
+
+package schema
+
+import "github.com/apache/arrow/go/parquet"
+
+// ListOf is a convenience helper function to create a properly structured
+// list structure according to the Parquet Spec.
+//
+// <list-repetition> group <name> (LIST) {
+//   repeated group list {
+//     <element-repetition> <element-type> element;
+//   }
+// }
+//
+// <list-repetition> can only be optional or required. panics if repeated.
+// <element-repetition> can only be optional or required. panics if repeated.
+func ListOf(n Node, rep parquet.Repetition, fieldID int32) *GroupNode {
+	if rep == parquet.Repetitions.Repeated || n.RepetitionType() == parquet.Repetitions.Repeated {
+		panic("parquet: listof repetition and element repetition must not be repeated.")
+	}
+	listName := n.Name()
+
+	switch n := n.(type) {
+	case *PrimitiveNode:
+		n.name = "element"
+	case *GroupNode:
+		n.name = "element"
+	}
+
+	return NewGroupNodeLogical(listName, rep, FieldList{
+		NewGroupNode("list", parquet.Repetitions.Repeated, FieldList{n}, -1)},
+		ListLogicalType{}, fieldID)
+}
+
+// MapOf is a convenience helper function to create a properly structured
+// parquet map node setup according to the Parquet Spec.
+//
+// <map-repetition> group <name> (MAP) {
+// 	 repeated group key_value {
+// 	   required <key-type> key;
+//     <value-repetition> <value-type> value;
+//   }
+// }
+//
+// key node will be renamed to "key", value node if not nil will be renamed to "value"
+//
+// <map-repetition> must be only optional or required. panics if repeated is passed.
+//
+// the key node *must* be required repetition. panics if optional or repeated
+//
+// value node can be nil (omitted) or have a repetition of required or optional *only*.
+// panics if value node is not nil and has a repetition of repeated.
+func MapOf(name string, key Node, value Node, mapRep parquet.Repetition, fieldID int32) *GroupNode {
+	if mapRep == parquet.Repetitions.Repeated {
+		panic("parquet: map repetition cannot be Repeated")
+	}
+	if key.RepetitionType() != parquet.Repetitions.Required {
+		panic("parquet: map key repetition must be Required")
+	}
+	if value != nil {
+		if value.RepetitionType() == parquet.Repetitions.Repeated {
+			panic("parquet: map value cannot have repetition Repeated")
+		}
+		switch value := value.(type) {
+		case *PrimitiveNode:
+			value.name = "value"
+		case *GroupNode:
+			value.name = "value"
+		}
+	}
+
+	switch key := key.(type) {
+	case *PrimitiveNode:
+		key.name = "key"
+	case *GroupNode:
+		key.name = "key"
+	}
+
+	keyval := FieldList{key}
+	if value != nil {
+		keyval = append(keyval, value)
+	}
+
+	return NewGroupNodeLogical(name, mapRep, FieldList{
+		NewGroupNode("key_value", parquet.Repetitions.Repeated, keyval, -1),

Review comment:
       literal comment.

##########
File path: go/parquet/schema/schema_flatten_test.go
##########
@@ -0,0 +1,150 @@
+// 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.
+
+package schema
+
+import (
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/suite"
+)
+
+func NewPrimitive(name string, repetition format.FieldRepetitionType, typ format.Type, fieldID int32) *format.SchemaElement {
+	ret := &format.SchemaElement{
+		Name:           name,
+		RepetitionType: format.FieldRepetitionTypePtr(repetition),
+		Type:           format.TypePtr(typ),
+	}
+	if fieldID >= 0 {
+		ret.FieldID = &fieldID
+	}
+	return ret
+}
+
+func NewGroup(name string, repetition format.FieldRepetitionType, numChildren, fieldID int32) *format.SchemaElement {
+	ret := &format.SchemaElement{
+		Name:           name,
+		RepetitionType: format.FieldRepetitionTypePtr(repetition),
+		NumChildren:    &numChildren,
+	}
+	if fieldID >= 0 {
+		ret.FieldID = &fieldID
+	}
+	return ret
+}
+
+type SchemaFlattenSuite struct {
+	suite.Suite
+
+	name string
+}
+
+func (s *SchemaFlattenSuite) SetupSuite() {
+	s.name = "parquet_schema"
+}
+
+func (s *SchemaFlattenSuite) TestDecimalMetadata() {
+	group := NewGroupNodeConverted("group", parquet.Repetitions.Repeated, FieldList{
+		NewPrimitiveNodeConverted("decimal", parquet.Repetitions.Required, parquet.Types.Int64, ConvertedTypes.Decimal, 0, 8, 4, -1),

Review comment:
       literal comments here would definitely help readability.

##########
File path: go/parquet/schema/reflection_test.go
##########
@@ -0,0 +1,397 @@
+// 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.
+
+package schema_test
+
+import (
+	"log"
+	"os"
+	"reflect"
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/schema"
+	"github.com/stretchr/testify/assert"
+)
+
+func ExampleNewSchemaFromStruct_primitives() {
+	type Schema struct {
+		Bool              bool
+		Int8              int8
+		Uint16            uint16
+		Int32             int32
+		Int64             int64
+		Int96             parquet.Int96
+		Float             float32
+		Double            float64
+		ByteArray         string
+		FixedLenByteArray [10]byte
+	}
+
+	sc, err := schema.NewSchemaFromStruct(Schema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 Schema {
+	//   required boolean field_id=-1 Bool;
+	//   required int32 field_id=-1 Int8 (Int(bitWidth=8, isSigned=true));
+	//   required int32 field_id=-1 Uint16 (Int(bitWidth=16, isSigned=false));
+	//   required int32 field_id=-1 Int32 (Int(bitWidth=32, isSigned=true));
+	//   required int64 field_id=-1 Int64 (Int(bitWidth=64, isSigned=true));
+	//   required int96 field_id=-1 Int96;
+	//   required float field_id=-1 Float;
+	//   required double field_id=-1 Double;
+	//   required byte_array field_id=-1 ByteArray;
+	//   required fixed_len_byte_array field_id=-1 FixedLenByteArray;
+	// }
+}
+
+func ExampleNewSchemaFromStruct_convertedtypes() {
+	type ConvertedSchema struct {
+		Utf8           string        `parquet:"name=utf8, converted=UTF8"`
+		Uint32         uint32        `parquet:"converted=INT_32"`
+		Date           int32         `parquet:"name=date, converted=date"`
+		TimeMilli      int32         `parquet:"name=timemilli, converted=TIME_MILLIS"`
+		TimeMicro      int64         `parquet:"name=timemicro, converted=time_micros"`
+		TimeStampMilli int64         `parquet:"converted=timestamp_millis"`
+		TimeStampMicro int64         `parquet:"converted=timestamp_micros"`
+		Interval       parquet.Int96 `parquet:"converted=INTERVAL"`
+		Decimal1       int32         `parquet:"converted=decimal, scale=2, precision=9"`
+		Decimal2       int64         `parquet:"converted=decimal, scale=2, precision=18"`
+		Decimal3       [12]byte      `parquet:"converted=decimal, scale=2, precision=10"`
+		Decimal4       string        `parquet:"converted=decimal, scale=2, precision=20"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(&ConvertedSchema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 ConvertedSchema {
+	//   required byte_array field_id=-1 utf8 (String);
+	//   required int32 field_id=-1 Uint32 (Int(bitWidth=32, isSigned=true));
+	//   required int32 field_id=-1 date (Date);
+	//   required int32 field_id=-1 timemilli (Time(isAdjustedToUTC=true, timeUnit=milliseconds));
+	//   required int64 field_id=-1 timemicro (Time(isAdjustedToUTC=true, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeStampMilli (Timestamp(isAdjustedToUTC=true, timeUnit=milliseconds, is_from_converted_type=true, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimeStampMicro (Timestamp(isAdjustedToUTC=true, timeUnit=microseconds, is_from_converted_type=true, force_set_converted_type=false));
+	//   required int96 field_id=-1 Interval;
+	//   required int32 field_id=-1 Decimal1 (Decimal(precision=9, scale=2));
+	//   required int64 field_id=-1 Decimal2 (Decimal(precision=18, scale=2));
+	//   required fixed_len_byte_array field_id=-1 Decimal3 (Decimal(precision=10, scale=2));
+	//   required byte_array field_id=-1 Decimal4 (Decimal(precision=20, scale=2));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_repetition() {
+	type RepetitionSchema struct {
+		List     []int64 `parquet:"fieldid=1"`
+		Repeated []int64 `parquet:"repetition=repeated, fieldid=2"`
+		Optional *int64  `parquet:"fieldid=3"`
+		Required *int64  `parquet:"repetition=REQUIRED, fieldid=4"`
+		Opt      int64   `parquet:"repetition=OPTIONAL, fieldid=5"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(RepetitionSchema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 RepetitionSchema {
+	//   required group field_id=1 List (List) {
+	//     repeated group field_id=-1 list {
+	//       required int64 field_id=-1 element (Int(bitWidth=64, isSigned=true));
+	//     }
+	//   }
+	//   repeated int64 field_id=2 Repeated (Int(bitWidth=64, isSigned=true));
+	//   optional int64 field_id=3 Optional (Int(bitWidth=64, isSigned=true));
+	//   required int64 field_id=4 Required (Int(bitWidth=64, isSigned=true));
+	//   optional int64 field_id=5 Opt (Int(bitWidth=64, isSigned=true));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_logicaltypes() {
+	type LogicalTypes struct {
+		String                []byte   `parquet:"logical=String"`
+		Enum                  string   `parquet:"logical=enum"`
+		Date                  int32    `parquet:"logical=date"`
+		Decimal1              int32    `parquet:"logical=decimal, precision=9, scale=2"`
+		Decimal2              int32    `parquet:"logical=decimal, logical.precision=9, scale=2"`
+		Decimal3              int32    `parquet:"logical=decimal, precision=5, logical.precision=9, scale=1, logical.scale=3"`
+		TimeMilliUTC          int32    `parquet:"logical=TIME, logical.unit=millis"`
+		TimeMilli             int32    `parquet:"logical=Time, logical.unit=millis, logical.isadjustedutc=false"`
+		TimeMicros            int64    `parquet:"logical=time, logical.unit=micros, logical.isadjustedutc=false"`
+		TimeMicrosUTC         int64    `parquet:"logical=time, logical.unit=micros, logical.isadjustedutc=true"`
+		TimeNanos             int64    `parquet:"logical=time, logical.unit=nanos"`
+		TimestampMilli        int64    `parquet:"logical=timestamp, logical.unit=millis"`
+		TimestampMicrosNotUTC int64    `parquet:"logical=timestamp, logical.unit=micros, logical.isadjustedutc=false"`
+		TimestampNanos        int64    `parquet:"logical=timestamp, logical.unit=nanos"`
+		JSON                  string   `parquet:"logical=json"`
+		BSON                  []byte   `parquet:"logical=BSON"`
+		UUID                  [16]byte `parquet:"logical=uuid"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(LogicalTypes{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 LogicalTypes {
+	//   required byte_array field_id=-1 String (String);
+	//   required byte_array field_id=-1 Enum (Enum);
+	//   required int32 field_id=-1 Date (Date);
+	//   required int32 field_id=-1 Decimal1 (Decimal(precision=9, scale=2));
+	//   required int32 field_id=-1 Decimal2 (Decimal(precision=9, scale=2));
+	//   required int32 field_id=-1 Decimal3 (Decimal(precision=9, scale=3));
+	//   required int32 field_id=-1 TimeMilliUTC (Time(isAdjustedToUTC=true, timeUnit=milliseconds));
+	//   required int32 field_id=-1 TimeMilli (Time(isAdjustedToUTC=false, timeUnit=milliseconds));
+	//   required int64 field_id=-1 TimeMicros (Time(isAdjustedToUTC=false, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeMicrosUTC (Time(isAdjustedToUTC=true, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeNanos (Time(isAdjustedToUTC=true, timeUnit=nanoseconds));
+	//   required int64 field_id=-1 TimestampMilli (Timestamp(isAdjustedToUTC=true, timeUnit=milliseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimestampMicrosNotUTC (Timestamp(isAdjustedToUTC=false, timeUnit=microseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimestampNanos (Timestamp(isAdjustedToUTC=true, timeUnit=nanoseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required byte_array field_id=-1 JSON (JSON);
+	//   required byte_array field_id=-1 BSON (BSON);
+	//   required fixed_len_byte_array field_id=-1 UUID (UUID);
+	// }
+}
+
+func ExampleNewSchemaFromStruct_physicaltype() {
+	type ChangeTypes struct {
+		Int32        int64  `parquet:"type=int32"`
+		FixedLen     string `parquet:"type=fixed_len_byte_array, length=10"`
+		SliceAsFixed []byte `parquet:"type=fixed_len_byte_array, length=12"`
+		Int          int    `parquet:"type=int32"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(ChangeTypes{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 ChangeTypes {
+	//   required int32 field_id=-1 Int32 (Int(bitWidth=32, isSigned=true));
+	//   required fixed_len_byte_array field_id=-1 FixedLen;
+	//   required fixed_len_byte_array field_id=-1 SliceAsFixed;
+	//   required int32 field_id=-1 Int (Int(bitWidth=32, isSigned=true));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_nestedtypes() {
+	type Other struct {
+		OptionalMap *map[string]*string `parquet:"valuerepetition=required, keylogical=String, valueconverted=BSON"`
+	}
+
+	type MyMap map[int32]string
+
+	type Nested struct {
+		SimpleMap     map[int32]string
+		FixedLenMap   map[string][]byte `parquet:"keytype=fixed_len_byte_array, keyfieldid=10, valuefieldid=11, keylength=10"`
+		DecimalMap    map[int32]string  `parquet:"logical=map, keyconverted=DECIMAL, keyscale=3, keyprecision=7, valuetype=fixed_len_byte_array, valuelength=4, valuelogical=decimal, valuelogical.precision=9, valuescale=2"`
+		OtherList     []*Other
+		OtherRepeated []Other  `parquet:"repetition=repeated"`
+		DateArray     [5]int32 `parquet:"valuelogical=date, logical=list"`
+		DateMap       MyMap    `parquet:"keylogical=TIME, keylogical.unit=MILLIS, keylogical.isadjustedutc=false, valuelogical=enum"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(Nested{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 Nested {

Review comment:
       did you want to make these assertion in this file?

##########
File path: go/parquet/schema/logical_types_test.go
##########
@@ -0,0 +1,551 @@
+// 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.
+
+package schema_test
+
+import (
+	"encoding/json"
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/schema"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestConvertedLogicalEquivalences(t *testing.T) {
+	tests := []struct {
+		name      string
+		converted schema.ConvertedType
+		logical   schema.LogicalType
+		expected  schema.LogicalType
+	}{
+		{"utf8", schema.ConvertedTypes.UTF8, schema.StringLogicalType{}, schema.StringLogicalType{}},
+		{"map", schema.ConvertedTypes.Map, schema.MapLogicalType{}, schema.MapLogicalType{}},
+		{"mapkeyval", schema.ConvertedTypes.MapKeyValue, schema.MapLogicalType{}, schema.MapLogicalType{}},
+		{"list", schema.ConvertedTypes.List, schema.NewListLogicalType(), schema.NewListLogicalType()},
+		{"enum", schema.ConvertedTypes.Enum, schema.EnumLogicalType{}, schema.EnumLogicalType{}},
+		{"date", schema.ConvertedTypes.Date, schema.DateLogicalType{}, schema.DateLogicalType{}},
+		{"timemilli", schema.ConvertedTypes.TimeMillis, schema.NewTimeLogicalType(true, schema.TimeUnitMillis), &schema.TimeLogicalType{}},
+		{"timemicro", schema.ConvertedTypes.TimeMicros, schema.NewTimeLogicalType(true, schema.TimeUnitMicros), &schema.TimeLogicalType{}},
+		{"timestampmilli", schema.ConvertedTypes.TimestampMillis, schema.NewTimestampLogicalType(true, schema.TimeUnitMillis), &schema.TimestampLogicalType{}},
+		{"timestampmicro", schema.ConvertedTypes.TimestampMicros, schema.NewTimestampLogicalType(true, schema.TimeUnitMicros), &schema.TimestampLogicalType{}},
+		{"uint8", schema.ConvertedTypes.Uint8, schema.NewIntLogicalType(8, false), &schema.IntLogicalType{}},
+		{"uint16", schema.ConvertedTypes.Uint16, schema.NewIntLogicalType(16, false), &schema.IntLogicalType{}},
+		{"uint32", schema.ConvertedTypes.Uint32, schema.NewIntLogicalType(32, false), &schema.IntLogicalType{}},
+		{"uint64", schema.ConvertedTypes.Uint64, schema.NewIntLogicalType(64, false), &schema.IntLogicalType{}},
+		{"int8", schema.ConvertedTypes.Int8, schema.NewIntLogicalType(8, true), &schema.IntLogicalType{}},

Review comment:
       literal comments would be helpful here as well.

##########
File path: go/parquet/schema/node.go
##########
@@ -0,0 +1,591 @@
+// 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.
+
+package schema
+
+import (
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/thrift/lib/go/thrift"
+	"golang.org/x/xerrors"
+)
+
+// NodeType describes whether the Node is a Primitive or Group node
+type NodeType int
+
+// the available constants for NodeType
+const (
+	Primitive NodeType = iota
+	Group
+)
+
+// Node is the interface for both Group and Primitive Nodes.
+// A logical schema type has a name, repetition level, and optionally
+// a logical type (ConvertedType in Parquet metadata parlance)

Review comment:
       nit: ConvertedType and LogicalType are diffent.  convertedType is the deprecated version of the concept that needs to be maintained for forward compatibility.

##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval

Review comment:
       why can't interval be serialized?  is this generated code?

##########
File path: go/parquet/schema/reflection.go
##########
@@ -0,0 +1,791 @@
+// 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.
+
+package schema
+
+import (
+	"reflect"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+type taggedInfo struct {
+	Name string
+
+	Type      parquet.Type
+	KeyType   parquet.Type
+	ValueType parquet.Type
+
+	Length      int32
+	KeyLength   int32
+	ValueLength int32
+
+	Scale      int32
+	KeyScale   int32
+	ValueScale int32
+
+	Precision      int32
+	KeyPrecision   int32
+	ValuePrecision int32
+
+	FieldID      int32
+	KeyFieldID   int32
+	ValueFieldID int32
+
+	RepetitionType  parquet.Repetition
+	ValueRepetition parquet.Repetition
+
+	Converted      ConvertedType
+	KeyConverted   ConvertedType
+	ValueConverted ConvertedType
+
+	LogicalFields      map[string]string
+	KeyLogicalFields   map[string]string
+	ValueLogicalFields map[string]string
+
+	LogicalType      LogicalType
+	KeyLogicalType   LogicalType
+	ValueLogicalType LogicalType
+}
+
+func (t *taggedInfo) CopyForKey() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.KeyType
+	ret.Length = t.KeyLength
+	ret.Scale = t.KeyScale
+	ret.Precision = t.KeyPrecision
+	ret.FieldID = t.KeyFieldID
+	ret.RepetitionType = parquet.Repetitions.Required
+	ret.Converted = t.KeyConverted
+	ret.LogicalType = t.KeyLogicalType
+	return
+}
+
+func (t *taggedInfo) CopyForValue() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.ValueType
+	ret.Length = t.ValueLength
+	ret.Scale = t.ValueScale
+	ret.Precision = t.ValuePrecision
+	ret.FieldID = t.ValueFieldID
+	ret.RepetitionType = t.ValueRepetition
+	ret.Converted = t.ValueConverted
+	ret.LogicalType = t.ValueLogicalType
+	return
+}
+
+func (t *taggedInfo) UpdateLogicalTypes() {
+	processLogicalType := func(fields map[string]string, precision, scale int32) LogicalType {
+		t, ok := fields["type"]
+		if !ok {
+			return NoLogicalType{}
+		}
+
+		switch strings.ToLower(t) {
+		case "string":
+			return StringLogicalType{}
+		case "map":
+			return MapLogicalType{}
+		case "list":
+			return ListLogicalType{}
+		case "enum":
+			return EnumLogicalType{}
+		case "decimal":
+			if v, ok := fields["precision"]; ok {
+				precision = int32FromType(v)
+			}
+			if v, ok := fields["scale"]; ok {
+				scale = int32FromType(v)
+			}
+			return NewDecimalLogicalType(precision, scale)
+		case "date":
+			return DateLogicalType{}
+		case "time":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimeLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(strings.ToLower(unit)))
+		case "timestamp":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimestampLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(unit))
+		case "integer":
+			width, ok := fields["bitwidth"]
+			if !ok {
+				panic("must specify bitwidth if explicitly setting integer logical type")
+			}
+			signed, ok := fields["signed"]
+			if !ok {
+				signed = "true"
+			}
+
+			return NewIntLogicalType(int8(int32FromType(width)), boolFromStr(signed))
+		case "null":
+			return NullLogicalType{}
+		case "json":
+			return JSONLogicalType{}
+		case "bson":
+			return BSONLogicalType{}
+		case "uuid":
+			return UUIDLogicalType{}
+		default:
+			panic(xerrors.Errorf("invalid logical type specified: %s", t))
+		}
+	}
+
+	t.LogicalType = processLogicalType(t.LogicalFields, t.Precision, t.Scale)
+	t.KeyLogicalType = processLogicalType(t.KeyLogicalFields, t.KeyPrecision, t.KeyScale)
+	t.ValueLogicalType = processLogicalType(t.ValueLogicalFields, t.ValuePrecision, t.ValueScale)
+}
+
+func newTaggedInfo() taggedInfo {
+	return taggedInfo{
+		Type:               parquet.Types.Undefined,
+		KeyType:            parquet.Types.Undefined,
+		ValueType:          parquet.Types.Undefined,
+		RepetitionType:     parquet.Repetitions.Undefined,
+		ValueRepetition:    parquet.Repetitions.Undefined,
+		Converted:          ConvertedTypes.NA,
+		KeyConverted:       ConvertedTypes.NA,
+		ValueConverted:     ConvertedTypes.NA,
+		FieldID:            -1,
+		KeyFieldID:         -1,
+		ValueFieldID:       -1,
+		LogicalFields:      make(map[string]string),
+		KeyLogicalFields:   make(map[string]string),
+		ValueLogicalFields: make(map[string]string),
+		LogicalType:        NoLogicalType{},
+		KeyLogicalType:     NoLogicalType{},
+		ValueLogicalType:   NoLogicalType{},
+	}
+}
+
+var int32FromType = func(v string) int32 {
+	val, err := strconv.Atoi(v)
+	if err != nil {
+		panic(err)
+	}
+	return int32(val)
+}
+
+var boolFromStr = func(v string) bool {
+	val, err := strconv.ParseBool(v)
+	if err != nil {
+		panic(err)
+	}
+	return val
+}
+
+func infoFromTags(f reflect.StructTag) *taggedInfo {
+	typeFromStr := func(v string) parquet.Type {
+		t, err := format.TypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(xerrors.Errorf("invalid type specified: %s", v))
+		}
+		return parquet.Type(t)
+	}
+
+	repFromStr := func(v string) parquet.Repetition {
+		r, err := format.FieldRepetitionTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return parquet.Repetition(r)
+	}
+
+	convertedFromStr := func(v string) ConvertedType {
+		c, err := format.ConvertedTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return ConvertedType(c)
+	}
+
+	if ptags, ok := f.Lookup("parquet"); ok {
+		info := newTaggedInfo()
+		for _, tag := range strings.Split(strings.Replace(ptags, "\t", "", -1), ",") {
+			tag = strings.TrimSpace(tag)
+			kv := strings.SplitN(tag, "=", 2)
+			key := strings.TrimSpace(strings.ToLower(kv[0]))
+			value := strings.TrimSpace(kv[1])
+
+			switch key {
+			case "name":
+				info.Name = value
+			case "type":
+				info.Type = typeFromStr(value)
+			case "keytype":
+				info.KeyType = typeFromStr(value)
+			case "valuetype":
+				info.ValueType = typeFromStr(value)
+			case "length":
+				info.Length = int32FromType(value)
+			case "keylength":
+				info.KeyLength = int32FromType(value)
+			case "valuelength":
+				info.ValueLength = int32FromType(value)
+			case "scale":
+				info.Scale = int32FromType(value)
+			case "keyscale":
+				info.KeyScale = int32FromType(value)
+			case "valuescale":
+				info.ValueScale = int32FromType(value)
+			case "precision":
+				info.Precision = int32FromType(value)
+			case "keyprecision":
+				info.KeyPrecision = int32FromType(value)
+			case "valueprecision":
+				info.ValuePrecision = int32FromType(value)
+			case "fieldid":
+				info.FieldID = int32FromType(value)
+			case "keyfieldid":
+				info.KeyFieldID = int32FromType(value)
+			case "valuefieldid":
+				info.ValueFieldID = int32FromType(value)
+			case "repetition":
+				info.RepetitionType = repFromStr(value)
+			case "valuerepetition":
+				info.ValueRepetition = repFromStr(value)
+			case "converted":
+				info.Converted = convertedFromStr(value)
+			case "keyconverted":
+				info.KeyConverted = convertedFromStr(value)
+			case "valueconverted":
+				info.ValueConverted = convertedFromStr(value)
+			case "logical":
+				info.LogicalFields["type"] = value
+			case "keylogical":
+				info.KeyLogicalFields["type"] = value
+			case "valuelogical":
+				info.ValueLogicalFields["type"] = value
+			default:
+				switch {
+				case strings.HasPrefix(key, "logical."):
+					info.LogicalFields[strings.TrimPrefix(key, "logical.")] = value
+				case strings.HasPrefix(key, "keylogical."):
+					info.KeyLogicalFields[strings.TrimPrefix(key, "keylogical.")] = value
+				case strings.HasPrefix(key, "valuelogical."):
+					info.ValueLogicalFields[strings.TrimPrefix(key, "valuelogical.")] = value
+				}
+			}
+		}
+		info.UpdateLogicalTypes()
+		return &info
+	}
+	return nil
+}
+
+func typeToNode(name string, typ reflect.Type, repType parquet.Repetition, info *taggedInfo) Node {

Review comment:
       even though this is private it looks like not trivial logic  some docs could be useful.

##########
File path: go/parquet/schema/node.go
##########
@@ -0,0 +1,591 @@
+// 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.
+
+package schema
+
+import (
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/thrift/lib/go/thrift"
+	"golang.org/x/xerrors"
+)
+
+// NodeType describes whether the Node is a Primitive or Group node
+type NodeType int
+
+// the available constants for NodeType
+const (
+	Primitive NodeType = iota
+	Group
+)
+
+// Node is the interface for both Group and Primitive Nodes.
+// A logical schema type has a name, repetition level, and optionally
+// a logical type (ConvertedType in Parquet metadata parlance)
+type Node interface {
+	Name() string
+	Type() NodeType
+	RepetitionType() parquet.Repetition
+	ConvertedType() ConvertedType
+	LogicalType() LogicalType
+	FieldID() int32
+	Parent() Node
+	SetParent(Node)
+	Path() string
+	Equals(Node) bool
+	Visit(v Visitor)
+	toThrift() *format.SchemaElement
+}
+
+// Visitor is an interface for creating functionality to walk the schema tree.
+//
+// A visitor can be passed to the Visit function of a Node in order to walk
+// the tree. VisitPre is called the first time a node is encountered. If
+// it is a group node, the return is checked and if it is false, the children
+// will be skipped.
+//
+// VisitPost is called after visiting any children
+type Visitor interface {
+	VisitPre(Node) bool
+	VisitPost(Node)
+}
+
+// ColumnPathFromNode walks the parents of the given node to construct it's
+// column path
+func ColumnPathFromNode(n Node) parquet.ColumnPath {
+	if n == nil {
+		return nil
+	}
+
+	c := make([]string, 0)
+
+	cursor := n
+	for cursor.Parent() != nil {
+		c = append(c, cursor.Name())
+		cursor = cursor.Parent()
+	}
+
+	for i := len(c)/2 - 1; i >= 0; i-- {

Review comment:
       this logic is a little opaque to me, could just be my limited experience with go.

##########
File path: go/parquet/schema/schema_element_test.go
##########
@@ -0,0 +1,432 @@
+// 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.
+
+package schema
+
+import (
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/suite"
+)
+
+type schemaElementConstruction struct {
+	node            Node
+	element         *format.SchemaElement
+	name            string
+	expectConverted bool
+	converted       ConvertedType
+	expectLogical   bool
+	checkLogical    func(*format.SchemaElement) bool
+}
+
+type decimalSchemaElementConstruction struct {
+	schemaElementConstruction
+	precision int
+	scale     int
+}
+
+type temporalSchemaElementConstruction struct {
+	schemaElementConstruction
+	adjusted bool
+	unit     TimeUnitType
+	getUnit  func(*format.SchemaElement) *format.TimeUnit
+}
+
+type intSchemaElementConstruction struct {
+	schemaElementConstruction
+	width  int8
+	signed bool
+}
+
+type legacySchemaElementConstructArgs struct {
+	name            string
+	physical        parquet.Type
+	len             int
+	expectConverted bool
+	converted       ConvertedType
+	expectLogical   bool
+	checkLogical    func(*format.SchemaElement) bool
+}
+
+type schemaElementConstructArgs struct {
+	name            string
+	logical         LogicalType
+	physical        parquet.Type
+	len             int
+	expectConverted bool
+	converted       ConvertedType
+	expectLogical   bool
+	checkLogical    func(*format.SchemaElement) bool
+}
+type SchemaElementConstructionSuite struct {
+	suite.Suite
+}
+
+func (s *SchemaElementConstructionSuite) reconstruct(c schemaElementConstructArgs) *schemaElementConstruction {
+	ret := &schemaElementConstruction{
+		node:            NewPrimitiveNodeLogical(c.name, parquet.Repetitions.Required, c.logical, c.physical, c.len, -1),
+		name:            c.name,
+		expectConverted: c.expectConverted,
+		converted:       c.converted,
+		expectLogical:   c.expectLogical,
+		checkLogical:    c.checkLogical,
+	}
+	ret.element = ret.node.toThrift()
+	return ret
+}
+
+func (s *SchemaElementConstructionSuite) legacyReconstruct(c legacySchemaElementConstructArgs) *schemaElementConstruction {
+	ret := &schemaElementConstruction{
+		node:            NewPrimitiveNodeConverted(c.name, parquet.Repetitions.Required, c.physical, c.converted, c.len, 0, 0, -1),
+		name:            c.name,
+		expectConverted: c.expectConverted,
+		converted:       c.converted,
+		expectLogical:   c.expectLogical,
+		checkLogical:    c.checkLogical,
+	}
+	ret.element = ret.node.toThrift()
+	return ret
+}
+
+func (s *SchemaElementConstructionSuite) inspect(c *schemaElementConstruction) {
+	if c.expectConverted {
+		s.True(c.element.IsSetConvertedType())
+		s.Equal(c.converted, ConvertedType(*c.element.ConvertedType))
+	} else {
+		s.False(c.element.IsSetConvertedType())
+	}
+	if c.expectLogical {
+		s.True(c.element.IsSetLogicalType())
+		s.True(c.checkLogical(c.element))
+	} else {
+		s.False(c.element.IsSetLogicalType())
+	}
+}
+
+func (s *SchemaElementConstructionSuite) TestSimple() {
+	checkNone := func(*format.SchemaElement) bool { return true }
+
+	tests := []struct {
+		name   string
+		args   *schemaElementConstructArgs
+		legacy *legacySchemaElementConstructArgs
+	}{
+		{"string", &schemaElementConstructArgs{
+			"string", StringLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.UTF8, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetSTRING() },
+		}, nil},
+		{"enum", &schemaElementConstructArgs{
+			"enum", EnumLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.Enum, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetENUM() },
+		}, nil},
+		{"date", &schemaElementConstructArgs{
+			"date", DateLogicalType{}, parquet.Types.Int32, -1, true, ConvertedTypes.Date, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetDATE() },
+		}, nil},
+		{"interval", &schemaElementConstructArgs{
+			"interval", IntervalLogicalType{}, parquet.Types.FixedLenByteArray, 12, true, ConvertedTypes.Interval, false,
+			checkNone,
+		}, nil},
+		{"null", &schemaElementConstructArgs{
+			"null", NullLogicalType{}, parquet.Types.Double, -1, false, ConvertedTypes.NA, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetUNKNOWN() },
+		}, nil},
+		{"json", &schemaElementConstructArgs{
+			"json", JSONLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.JSON, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetJSON() },
+		}, nil},
+		{"bson", &schemaElementConstructArgs{
+			"bson", BSONLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.BSON, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetBSON() },
+		}, nil},
+		{"uuid", &schemaElementConstructArgs{
+			"uuid", UUIDLogicalType{}, parquet.Types.FixedLenByteArray, 16, false, ConvertedTypes.NA, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetUUID() },
+		}, nil},
+		{"none", &schemaElementConstructArgs{
+			"none", NoLogicalType{}, parquet.Types.Int64, -1, false, ConvertedTypes.NA, false,
+			checkNone,
+		}, nil},
+		{"unknown", &schemaElementConstructArgs{
+			"unknown", UnknownLogicalType{}, parquet.Types.Int64, -1, true, ConvertedTypes.NA, false,
+			checkNone,
+		}, nil},
+		{"timestamp_ms", nil, &legacySchemaElementConstructArgs{
+			"timestamp_ms", parquet.Types.Int64, -1, true, ConvertedTypes.TimestampMillis, false, checkNone}},
+		{"timestamp_us", nil, &legacySchemaElementConstructArgs{
+			"timestamp_us", parquet.Types.Int64, -1, true, ConvertedTypes.TimestampMicros, false, checkNone}},
+	}
+	for _, tt := range tests {
+		s.Run(tt.name, func() {
+			var sc *schemaElementConstruction
+			if tt.args != nil {
+				sc = s.reconstruct(*tt.args)
+			} else {
+				sc = s.legacyReconstruct(*tt.legacy)
+			}
+			s.Equal(tt.name, sc.element.Name)
+			s.inspect(sc)
+		})
+	}
+}
+
+func (s *SchemaElementConstructionSuite) reconstructDecimal(c schemaElementConstructArgs) *decimalSchemaElementConstruction {
+	ret := s.reconstruct(c)
+	dec := c.logical.(*DecimalLogicalType)
+	return &decimalSchemaElementConstruction{*ret, int(dec.Precision()), int(dec.Scale())}
+}
+
+func (s *SchemaElementConstructionSuite) inspectDecimal(d *decimalSchemaElementConstruction) {
+	s.inspect(&d.schemaElementConstruction)
+	s.EqualValues(d.precision, d.element.GetPrecision())
+	s.EqualValues(d.scale, d.element.GetScale())
+	s.EqualValues(d.precision, d.element.LogicalType.DECIMAL.Precision)
+	s.EqualValues(d.scale, d.element.LogicalType.DECIMAL.Scale)
+}
+
+func (s *SchemaElementConstructionSuite) TestDecimal() {
+	checkDecimal := func(p *format.SchemaElement) bool { return p.LogicalType.IsSetDECIMAL() }
+
+	tests := []schemaElementConstructArgs{
+		{"decimal16_6", NewDecimalLogicalType(16, 6), parquet.Types.Int64, -1, true, ConvertedTypes.Decimal, true, checkDecimal},
+		{"decimal1_0", NewDecimalLogicalType(1, 0), parquet.Types.Int32, -1, true, ConvertedTypes.Decimal, true, checkDecimal},
+		{"decimal10", NewDecimalLogicalType(10, 0), parquet.Types.Int64, -1, true, ConvertedTypes.Decimal, true, checkDecimal},

Review comment:
       comment for -1 could make this easier to read.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] github-actions[bot] commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-821270463


   https://issues.apache.org/jira/browse/ARROW-12424


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616858908



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {

Review comment:
       Looking some more at the C++ code, it appears HasSpacedValues is only used in ReadBatchSpaced which is deprecated precisely because a lot of this logic is broken, and outside of Arrow (which no longer uses this code path) doesn't serve much value.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619923902



##########
File path: go/parquet/schema/reflection.go
##########
@@ -0,0 +1,791 @@
+// 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.
+
+package schema
+
+import (
+	"reflect"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+type taggedInfo struct {
+	Name string
+
+	Type      parquet.Type
+	KeyType   parquet.Type
+	ValueType parquet.Type
+
+	Length      int32
+	KeyLength   int32
+	ValueLength int32
+
+	Scale      int32
+	KeyScale   int32
+	ValueScale int32
+
+	Precision      int32
+	KeyPrecision   int32
+	ValuePrecision int32
+
+	FieldID      int32
+	KeyFieldID   int32
+	ValueFieldID int32
+
+	RepetitionType  parquet.Repetition
+	ValueRepetition parquet.Repetition
+
+	Converted      ConvertedType
+	KeyConverted   ConvertedType
+	ValueConverted ConvertedType
+
+	LogicalFields      map[string]string
+	KeyLogicalFields   map[string]string
+	ValueLogicalFields map[string]string
+
+	LogicalType      LogicalType
+	KeyLogicalType   LogicalType
+	ValueLogicalType LogicalType
+}
+
+func (t *taggedInfo) CopyForKey() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.KeyType
+	ret.Length = t.KeyLength
+	ret.Scale = t.KeyScale
+	ret.Precision = t.KeyPrecision
+	ret.FieldID = t.KeyFieldID
+	ret.RepetitionType = parquet.Repetitions.Required
+	ret.Converted = t.KeyConverted
+	ret.LogicalType = t.KeyLogicalType
+	return
+}
+
+func (t *taggedInfo) CopyForValue() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.ValueType
+	ret.Length = t.ValueLength
+	ret.Scale = t.ValueScale
+	ret.Precision = t.ValuePrecision
+	ret.FieldID = t.ValueFieldID
+	ret.RepetitionType = t.ValueRepetition
+	ret.Converted = t.ValueConverted
+	ret.LogicalType = t.ValueLogicalType
+	return
+}
+
+func (t *taggedInfo) UpdateLogicalTypes() {
+	processLogicalType := func(fields map[string]string, precision, scale int32) LogicalType {
+		t, ok := fields["type"]
+		if !ok {
+			return NoLogicalType{}
+		}
+
+		switch strings.ToLower(t) {
+		case "string":
+			return StringLogicalType{}
+		case "map":
+			return MapLogicalType{}
+		case "list":
+			return ListLogicalType{}
+		case "enum":
+			return EnumLogicalType{}
+		case "decimal":
+			if v, ok := fields["precision"]; ok {
+				precision = int32FromType(v)
+			}
+			if v, ok := fields["scale"]; ok {
+				scale = int32FromType(v)
+			}
+			return NewDecimalLogicalType(precision, scale)
+		case "date":
+			return DateLogicalType{}
+		case "time":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimeLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(strings.ToLower(unit)))
+		case "timestamp":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimestampLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(unit))
+		case "integer":
+			width, ok := fields["bitwidth"]
+			if !ok {
+				panic("must specify bitwidth if explicitly setting integer logical type")
+			}
+			signed, ok := fields["signed"]
+			if !ok {
+				signed = "true"
+			}
+
+			return NewIntLogicalType(int8(int32FromType(width)), boolFromStr(signed))
+		case "null":
+			return NullLogicalType{}
+		case "json":
+			return JSONLogicalType{}
+		case "bson":
+			return BSONLogicalType{}
+		case "uuid":
+			return UUIDLogicalType{}
+		default:
+			panic(xerrors.Errorf("invalid logical type specified: %s", t))
+		}
+	}
+
+	t.LogicalType = processLogicalType(t.LogicalFields, t.Precision, t.Scale)
+	t.KeyLogicalType = processLogicalType(t.KeyLogicalFields, t.KeyPrecision, t.KeyScale)
+	t.ValueLogicalType = processLogicalType(t.ValueLogicalFields, t.ValuePrecision, t.ValueScale)
+}
+
+func newTaggedInfo() taggedInfo {
+	return taggedInfo{
+		Type:               parquet.Types.Undefined,
+		KeyType:            parquet.Types.Undefined,
+		ValueType:          parquet.Types.Undefined,
+		RepetitionType:     parquet.Repetitions.Undefined,
+		ValueRepetition:    parquet.Repetitions.Undefined,
+		Converted:          ConvertedTypes.NA,
+		KeyConverted:       ConvertedTypes.NA,
+		ValueConverted:     ConvertedTypes.NA,
+		FieldID:            -1,
+		KeyFieldID:         -1,
+		ValueFieldID:       -1,
+		LogicalFields:      make(map[string]string),
+		KeyLogicalFields:   make(map[string]string),
+		ValueLogicalFields: make(map[string]string),
+		LogicalType:        NoLogicalType{},
+		KeyLogicalType:     NoLogicalType{},
+		ValueLogicalType:   NoLogicalType{},
+	}
+}
+
+var int32FromType = func(v string) int32 {
+	val, err := strconv.Atoi(v)
+	if err != nil {
+		panic(err)
+	}
+	return int32(val)
+}
+
+var boolFromStr = func(v string) bool {
+	val, err := strconv.ParseBool(v)
+	if err != nil {
+		panic(err)
+	}
+	return val
+}
+
+func infoFromTags(f reflect.StructTag) *taggedInfo {
+	typeFromStr := func(v string) parquet.Type {
+		t, err := format.TypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(xerrors.Errorf("invalid type specified: %s", v))
+		}
+		return parquet.Type(t)
+	}
+
+	repFromStr := func(v string) parquet.Repetition {
+		r, err := format.FieldRepetitionTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return parquet.Repetition(r)
+	}
+
+	convertedFromStr := func(v string) ConvertedType {
+		c, err := format.ConvertedTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return ConvertedType(c)
+	}
+
+	if ptags, ok := f.Lookup("parquet"); ok {
+		info := newTaggedInfo()
+		for _, tag := range strings.Split(strings.Replace(ptags, "\t", "", -1), ",") {
+			tag = strings.TrimSpace(tag)
+			kv := strings.SplitN(tag, "=", 2)
+			key := strings.TrimSpace(strings.ToLower(kv[0]))
+			value := strings.TrimSpace(kv[1])
+
+			switch key {
+			case "name":
+				info.Name = value
+			case "type":
+				info.Type = typeFromStr(value)
+			case "keytype":
+				info.KeyType = typeFromStr(value)
+			case "valuetype":
+				info.ValueType = typeFromStr(value)
+			case "length":
+				info.Length = int32FromType(value)
+			case "keylength":
+				info.KeyLength = int32FromType(value)
+			case "valuelength":
+				info.ValueLength = int32FromType(value)
+			case "scale":
+				info.Scale = int32FromType(value)
+			case "keyscale":
+				info.KeyScale = int32FromType(value)
+			case "valuescale":
+				info.ValueScale = int32FromType(value)
+			case "precision":
+				info.Precision = int32FromType(value)
+			case "keyprecision":
+				info.KeyPrecision = int32FromType(value)
+			case "valueprecision":
+				info.ValuePrecision = int32FromType(value)
+			case "fieldid":
+				info.FieldID = int32FromType(value)
+			case "keyfieldid":
+				info.KeyFieldID = int32FromType(value)
+			case "valuefieldid":
+				info.ValueFieldID = int32FromType(value)
+			case "repetition":
+				info.RepetitionType = repFromStr(value)
+			case "valuerepetition":
+				info.ValueRepetition = repFromStr(value)
+			case "converted":
+				info.Converted = convertedFromStr(value)
+			case "keyconverted":
+				info.KeyConverted = convertedFromStr(value)
+			case "valueconverted":
+				info.ValueConverted = convertedFromStr(value)
+			case "logical":
+				info.LogicalFields["type"] = value
+			case "keylogical":
+				info.KeyLogicalFields["type"] = value
+			case "valuelogical":
+				info.ValueLogicalFields["type"] = value
+			default:
+				switch {
+				case strings.HasPrefix(key, "logical."):
+					info.LogicalFields[strings.TrimPrefix(key, "logical.")] = value
+				case strings.HasPrefix(key, "keylogical."):
+					info.KeyLogicalFields[strings.TrimPrefix(key, "keylogical.")] = value
+				case strings.HasPrefix(key, "valuelogical."):
+					info.ValueLogicalFields[strings.TrimPrefix(key, "valuelogical.")] = value
+				}
+			}
+		}
+		info.UpdateLogicalTypes()
+		return &info
+	}
+	return nil
+}
+
+func typeToNode(name string, typ reflect.Type, repType parquet.Repetition, info *taggedInfo) Node {

Review comment:
       added comments to the function and the rest of the function body




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619882824



##########
File path: go/parquet/schema/converted_types.go
##########
@@ -0,0 +1,191 @@
+// 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.
+
+package schema
+
+import (
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// ConvertedType corresponds to the ConvertedType in the parquet.Thrift,
+// with added values of None and NA for handling when these values are not
+// set in the metadata
+type ConvertedType format.ConvertedType
+
+var (
+	// ConvertedTypes is a struct containing the constants for the types
+	// to make it easy to reference them while making it clear what they are
+	ConvertedTypes = struct {
+		None            ConvertedType
+		UTF8            ConvertedType
+		Map             ConvertedType
+		MapKeyValue     ConvertedType
+		List            ConvertedType
+		Enum            ConvertedType
+		Decimal         ConvertedType
+		Date            ConvertedType
+		TimeMillis      ConvertedType
+		TimeMicros      ConvertedType
+		TimestampMillis ConvertedType
+		TimestampMicros ConvertedType
+		Uint8           ConvertedType
+		Uint16          ConvertedType
+		Uint32          ConvertedType
+		Uint64          ConvertedType
+		Int8            ConvertedType
+		Int16           ConvertedType
+		Int32           ConvertedType
+		Int64           ConvertedType
+		JSON            ConvertedType
+		BSON            ConvertedType
+		Interval        ConvertedType
+		NA              ConvertedType
+	}{
+		None:            -1, // thrift enum starts at 0, so we know this will not be used
+		UTF8:            ConvertedType(format.ConvertedType_UTF8),
+		Map:             ConvertedType(format.ConvertedType_MAP),
+		MapKeyValue:     ConvertedType(format.ConvertedType_MAP_KEY_VALUE),
+		List:            ConvertedType(format.ConvertedType_LIST),
+		Enum:            ConvertedType(format.ConvertedType_ENUM),
+		Decimal:         ConvertedType(format.ConvertedType_DECIMAL),
+		Date:            ConvertedType(format.ConvertedType_DATE),
+		TimeMillis:      ConvertedType(format.ConvertedType_TIME_MILLIS),
+		TimeMicros:      ConvertedType(format.ConvertedType_TIME_MICROS),
+		TimestampMillis: ConvertedType(format.ConvertedType_TIMESTAMP_MILLIS),
+		TimestampMicros: ConvertedType(format.ConvertedType_TIMESTAMP_MICROS),
+		Uint8:           ConvertedType(format.ConvertedType_UINT_8),
+		Uint16:          ConvertedType(format.ConvertedType_UINT_16),
+		Uint32:          ConvertedType(format.ConvertedType_UINT_32),
+		Uint64:          ConvertedType(format.ConvertedType_UINT_64),
+		Int8:            ConvertedType(format.ConvertedType_INT_8),
+		Int16:           ConvertedType(format.ConvertedType_INT_16),
+		Int32:           ConvertedType(format.ConvertedType_INT_32),
+		Int64:           ConvertedType(format.ConvertedType_INT_64),
+		JSON:            ConvertedType(format.ConvertedType_JSON),
+		BSON:            ConvertedType(format.ConvertedType_BSON),
+		Interval:        ConvertedType(format.ConvertedType_INTERVAL),
+		NA:              24, // should always be the last values after Interval
+	}
+)
+
+func (p ConvertedType) String() string {
+	switch p {
+	case ConvertedTypes.None:
+		return "NONE"
+	case ConvertedTypes.NA:
+		return "UNKNOWN"
+	default:
+		return format.ConvertedType(p).String()
+	}
+}
+
+// ToLogicalType returns the correct LogicalType for the given ConvertedType, using the decimal
+// metadata provided to define the precision/scale if necessary
+func (p ConvertedType) ToLogicalType(convertedDecimal DecimalMetadata) LogicalType {
+	switch p {
+	case ConvertedTypes.UTF8:
+		return StringLogicalType{}
+	case ConvertedTypes.Map, ConvertedTypes.MapKeyValue:
+		return MapLogicalType{}
+	case ConvertedTypes.List:
+		return ListLogicalType{}
+	case ConvertedTypes.Enum:
+		return EnumLogicalType{}
+	case ConvertedTypes.Decimal:
+		return NewDecimalLogicalType(convertedDecimal.Precision, convertedDecimal.Scale)
+	case ConvertedTypes.Date:
+		return DateLogicalType{}
+	case ConvertedTypes.TimeMillis:
+		return NewTimeLogicalType(true, TimeUnitMillis)
+	case ConvertedTypes.TimeMicros:
+		return NewTimeLogicalType(true, TimeUnitMicros)
+	case ConvertedTypes.TimestampMillis:
+		t := NewTimestampLogicalType(true, TimeUnitMillis)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.TimestampMicros:
+		t := NewTimestampLogicalType(true, TimeUnitMicros)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.Interval:
+		return IntervalLogicalType{}
+	case ConvertedTypes.Int8:
+		return NewIntLogicalType(8, true)
+	case ConvertedTypes.Int16:
+		return NewIntLogicalType(16, true)
+	case ConvertedTypes.Int32:
+		return NewIntLogicalType(32, true)
+	case ConvertedTypes.Int64:
+		return NewIntLogicalType(64, true)
+	case ConvertedTypes.Uint8:
+		return NewIntLogicalType(8, false)
+	case ConvertedTypes.Uint16:
+		return NewIntLogicalType(16, false)
+	case ConvertedTypes.Uint32:
+		return NewIntLogicalType(32, false)
+	case ConvertedTypes.Uint64:
+		return NewIntLogicalType(64, false)
+	case ConvertedTypes.JSON:
+		return JSONLogicalType{}
+	case ConvertedTypes.BSON:
+		return BSONLogicalType{}
+	case ConvertedTypes.None:
+		return NoLogicalType{}
+	case ConvertedTypes.NA:
+		fallthrough
+	default:
+		return UnknownLogicalType{}
+	}
+}
+
+// GetSortOrder defaults to the sort order based on the physical type if convert
+// is ConvertedTypes.None, otherwise determines the sort order by the converted type.
+func GetSortOrder(convert ConvertedType, primitive format.Type) SortOrder {
+	if convert == ConvertedTypes.None {
+		return DefaultSortOrder(primitive)
+	}
+	switch convert {
+	case ConvertedTypes.Int8,
+		ConvertedTypes.Int16,
+		ConvertedTypes.Int32,
+		ConvertedTypes.Int64,
+		ConvertedTypes.Date,
+		ConvertedTypes.TimeMicros,
+		ConvertedTypes.TimeMillis,
+		ConvertedTypes.TimestampMicros,
+		ConvertedTypes.TimestampMillis:
+		return SortSIGNED
+	case ConvertedTypes.Uint8,
+		ConvertedTypes.Uint16,
+		ConvertedTypes.Uint32,
+		ConvertedTypes.Uint64,
+		ConvertedTypes.Enum,
+		ConvertedTypes.UTF8,
+		ConvertedTypes.BSON,
+		ConvertedTypes.JSON:
+		return SortUNSIGNED
+	case ConvertedTypes.Decimal,

Review comment:
       this doesn't look right for decimal.  I believe it should be signed?  Or might vary based on underlying type.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r631957018



##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval
+	IsSerialized() bool
+	// Returns true if not NoLogicalType
+	IsValid() bool
+	// Returns true if it is NoType
+	IsNone() bool
+	// returns a string representation of the Logical Type
+	String() string
+	toThrift() *format.LogicalType
+	// Return the equivalent ConvertedType for legacy Parquet systems
+	ToConvertedType() (ConvertedType, DecimalMetadata)
+	// Returns true if the specified ConvertedType is compatible with this
+	// logical type
+	IsCompatible(ConvertedType, DecimalMetadata) bool
+	// Returns true if this logical type can be used with the provided physical type
+	IsApplicable(t parquet.Type, tlen int32) bool
+	// Returns true if the logical types are the same
+	Equals(LogicalType) bool
+	// Returns the default stat sort order for this logical type
+	SortOrder() SortOrder
+}
+
+// TemporalLogicalType is a smaller interface for Time based logical types
+// like Time / Timestamp
+type TemporalLogicalType interface {
+	LogicalType
+	IsAdjustedToUTC() bool
+	TimeUnit() TimeUnitType
+}
+
+// SortOrder mirrors the parquet.thrift sort order type
+type SortOrder int8
+
+// Constants for the Stat sort order definitions
+const (
+	SortSIGNED SortOrder = iota
+	SortUNSIGNED
+	SortUNKNOWN
+)
+
+// DefaultSortOrder returns the default stat sort order for the given physical type
+func DefaultSortOrder(primitive format.Type) SortOrder {
+	switch primitive {
+	case format.Type_BOOLEAN, format.Type_INT32, format.Type_INT64, format.Type_FLOAT, format.Type_DOUBLE:
+		return SortSIGNED
+	case format.Type_BYTE_ARRAY, format.Type_FIXED_LEN_BYTE_ARRAY:
+		return SortUNSIGNED
+	case format.Type_INT96:
+		fallthrough
+	default:
+		return SortUNKNOWN
+	}
+}
+
+// GetLogicalSortOrder returns the default sort order for this logical type
+// or falls back to the default sort order for the physical type if not valid
+func GetLogicalSortOrder(logical LogicalType, primitive format.Type) SortOrder {
+	switch {
+	case logical == nil || !logical.IsValid():
+		return SortUNKNOWN
+	case logical.Equals(NoLogicalType{}):
+		return DefaultSortOrder(primitive)
+	default:
+		return logical.SortOrder()
+	}
+}
+
+type baseLogicalType struct{}
+
+func (baseLogicalType) IsSerialized() bool {
+	return true
+}
+
+func (baseLogicalType) IsValid() bool {
+	return true
+}
+
+func (baseLogicalType) IsNested() bool {
+	return false
+}
+
+func (baseLogicalType) IsNone() bool { return false }
+
+// StringLogicalType is a UTF8 string, only usable with ByteArray and FixedLenByteArray
+type StringLogicalType struct{ baseLogicalType }
+
+func (StringLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (StringLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": StringLogicalType{}.String()})
+}
+
+func (StringLogicalType) String() string {
+	return "String"
+}
+
+func (StringLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.UTF8, DecimalMetadata{}
+}
+
+func (StringLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.UTF8 && !dec.IsSet
+}
+
+func (StringLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (StringLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{STRING: format.NewStringType()}
+}
+
+func (StringLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(StringLogicalType)
+	return ok
+}
+
+// MapLogicalType represents a mapped type
+type MapLogicalType struct{ baseLogicalType }
+
+func (MapLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (MapLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": MapLogicalType{}.String()})
+}
+
+func (MapLogicalType) String() string {
+	return "Map"
+}
+
+func (MapLogicalType) IsNested() bool {
+	return true
+}
+
+func (MapLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Map, DecimalMetadata{}
+}
+
+func (MapLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return (t == ConvertedTypes.Map || t == ConvertedTypes.MapKeyValue) && !dec.IsSet
+}
+
+func (MapLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (MapLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{MAP: format.NewMapType()}
+}
+
+func (MapLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(MapLogicalType)
+	return ok
+}
+
+func NewListLogicalType() LogicalType {
+	return ListLogicalType{}
+}
+
+// ListLogicalType is used for columns which are themselves nested lists
+type ListLogicalType struct{ baseLogicalType }
+
+func (ListLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (ListLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": ListLogicalType{}.String()})
+}
+
+func (ListLogicalType) String() string {
+	return "List"
+}
+
+func (ListLogicalType) IsNested() bool {
+	return true
+}
+
+func (ListLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.List, DecimalMetadata{}
+}
+
+func (ListLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.List && !dec.IsSet
+}
+
+func (ListLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (ListLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{LIST: format.NewListType()}
+}
+
+func (ListLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(ListLogicalType)
+	return ok
+}
+
+// EnumLogicalType is for representing an enum, which should be a byte array type
+type EnumLogicalType struct{ baseLogicalType }
+
+func (EnumLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (EnumLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": EnumLogicalType{}.String()})
+}
+
+func (EnumLogicalType) String() string {
+	return "Enum"
+}
+
+func (EnumLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Enum, DecimalMetadata{}
+}
+
+func (EnumLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Enum && !dec.IsSet
+}
+
+func (EnumLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (EnumLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{ENUM: format.NewEnumType()}
+}
+
+func (EnumLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(EnumLogicalType)
+	return ok
+}
+
+// NewDecimalLogicalType returns a Decimal logical type with the given
+// precision and scale.
+//
+// Panics if precision < 1 or scale is not in the range (0, precision)
+func NewDecimalLogicalType(precision int32, scale int32) LogicalType {
+	if precision < 1 {
+		panic("parquet: precision must be greater than or equal to 1 for decimal logical type")
+	}
+	if scale < 0 || scale > precision {
+		panic("parquet: scale must be a non-negative integer that does not exceed precision for decimal logical type")
+	}
+	return &DecimalLogicalType{typ: &format.DecimalType{Precision: precision, Scale: scale}}
+}
+
+// DecimalLogicalType is used to represent a decimal value of a given
+// precision and scale
+type DecimalLogicalType struct {
+	baseLogicalType
+	typ *format.DecimalType
+}
+
+func (t DecimalLogicalType) Precision() int32 {
+	return t.typ.Precision
+}
+
+func (t DecimalLogicalType) Scale() int32 {
+	return t.typ.Scale
+}
+
+func (DecimalLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t DecimalLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{"Type": "Decimal", "precision": t.typ.Precision, "scale": t.typ.Scale})
+}
+
+func (t DecimalLogicalType) String() string {
+	return fmt.Sprintf("Decimal(precision=%d, scale=%d)", t.typ.Precision, t.typ.Scale)
+}
+
+func (t DecimalLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Decimal, DecimalMetadata{IsSet: true, Scale: t.typ.GetScale(), Precision: t.typ.GetPrecision()}
+}
+
+func (t DecimalLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.Decimal &&
+		dec.IsSet && dec.Scale == t.typ.Scale && dec.Precision == t.typ.Precision
+}
+
+func (t DecimalLogicalType) IsApplicable(typ parquet.Type, tlen int32) bool {
+	switch typ {
+	case parquet.Types.Int32:
+		return 1 <= t.typ.Precision && t.typ.Precision <= 9
+	case parquet.Types.Int64:
+		if t.typ.Precision < 10 {
+			debug.Log("int64 used for decimal logical, precision is small enough to use int32")
+		}
+		return 1 <= t.typ.Precision && t.typ.Precision <= 18
+	case parquet.Types.FixedLenByteArray:
+		return t.typ.Precision <= int32(math.Floor(math.Log10(math.Pow(2.0, (8.0*float64(tlen)-1.0)))))
+	case parquet.Types.ByteArray:
+		return true
+	}
+	return false
+}
+
+func (t DecimalLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DECIMAL: t.typ}
+}
+
+func (t DecimalLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*DecimalLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.Precision == other.typ.Precision && t.typ.Scale == other.typ.Scale
+}
+
+// DateLogicalType is an int32 representing the number of days since the Unix Epoch
+// 1 January 1970
+type DateLogicalType struct{ baseLogicalType }
+
+func (DateLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (DateLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": DateLogicalType{}.String()})
+}
+
+func (DateLogicalType) String() string {
+	return "Date"
+}
+
+func (DateLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Date, DecimalMetadata{}
+}
+
+func (DateLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Date && !dec.IsSet
+}
+
+func (DateLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int32
+}
+
+func (DateLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DATE: format.NewDateType()}
+}
+
+func (DateLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(DateLogicalType)
+	return ok
+}
+
+func timeUnitFromThrift(unit *format.TimeUnit) TimeUnitType {
+	switch {
+	case unit == nil:
+		return TimeUnitUnknown
+	case unit.IsSetMILLIS():
+		return TimeUnitMillis
+	case unit.IsSetMICROS():
+		return TimeUnitMicros
+	case unit.IsSetNANOS():
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func timeUnitToString(unit *format.TimeUnit) string {
+	switch {
+	case unit == nil:
+		return "unknown"
+	case unit.IsSetMILLIS():
+		return "milliseconds"
+	case unit.IsSetMICROS():
+		return "microseconds"
+	case unit.IsSetNANOS():
+		return "nanoseconds"
+	default:
+		return "unknown"
+	}
+}
+
+func timeUnitFromString(v string) TimeUnitType {
+	switch v {
+	case "millis":
+		return TimeUnitMillis
+	case "micros":
+		return TimeUnitMicros
+	case "nanos":
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func createTimeUnit(unit TimeUnitType) *format.TimeUnit {
+	tunit := format.NewTimeUnit()
+	switch unit {
+	case TimeUnitMicros:
+		tunit.MICROS = format.NewMicroSeconds()
+	case TimeUnitMillis:
+		tunit.MILLIS = format.NewMilliSeconds()
+	case TimeUnitNanos:
+		tunit.NANOS = format.NewNanoSeconds()
+	default:
+		panic("parquet: time unit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+	}
+	return tunit
+}
+
+// NewTimeLogicalType returns a time type of the given unit.
+func NewTimeLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimeLogicalType{typ: &format.TimeType{
+		IsAdjustedToUTC: isAdjustedToUTC,
+		Unit:            createTimeUnit(unit),
+	}}
+}
+
+// TimeLogicalType is a time type without a date and must be an
+// int32 for milliseconds, or an int64 for micro or nano seconds.
+type TimeLogicalType struct {
+	baseLogicalType
+	typ *format.TimeType
+}
+
+func (t TimeLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimeLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimeLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimeLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Time", "isAdjustedToUTC": t.typ.IsAdjustedToUTC, "timeUnit": timeUnitToString(t.typ.GetUnit())})
+}
+
+func (t TimeLogicalType) String() string {
+	return fmt.Sprintf("Time(isAdjustedToUTC=%t, timeUnit=%s)", t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()))
+}
+
+func (t TimeLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimeMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimeMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimeLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return c == ConvertedTypes.TimeMillis
+		case TimeUnitMicros:
+			return c == ConvertedTypes.TimeMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (t TimeLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetUnit().IsSetMILLIS()) ||
+		(typ == parquet.Types.Int64 &&
+			(t.typ.GetUnit().IsSetMICROS() || t.typ.GetUnit().IsSetNANOS()))
+}
+
+func (t TimeLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIME: t.typ}
+}
+
+func (t TimeLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimeLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewTimestampLogicalType returns a logical timestamp type with "forceConverted"
+// set to false
+func NewTimestampLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: false,
+		fromConverted:  false,
+	}
+}
+
+// NewTimestampLogicalTypeForce returns a timestamp logical type with
+// "forceConverted" set to true
+func NewTimestampLogicalTypeForce(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: true,
+		fromConverted:  false,
+	}
+}
+
+// TimestampLogicalType represents an int64 number that can be decoded
+// into a year, month, day, hour, minute, second, and subsecond
+type TimestampLogicalType struct {
+	baseLogicalType
+	typ            *format.TimestampType
+	forceConverted bool
+	fromConverted  bool
+}
+
+func (t TimestampLogicalType) IsFromConvertedType() bool {
+	return t.fromConverted
+}
+
+func (t TimestampLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimestampLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimestampLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimestampLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type":                     "Timestamp",
+		"isAdjustedToUTC":          t.typ.IsAdjustedToUTC,
+		"timeUnit":                 timeUnitToString(t.typ.GetUnit()),
+		"is_from_converted_type":   t.fromConverted,
+		"force_set_converted_type": t.forceConverted,
+	})
+}
+
+func (t TimestampLogicalType) IsSerialized() bool {
+	return !t.fromConverted
+}
+
+func (t TimestampLogicalType) String() string {
+	return fmt.Sprintf("Timestamp(isAdjustedToUTC=%t, timeUnit=%s, is_from_converted_type=%t, force_set_converted_type=%t)",
+		t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()), t.fromConverted, t.forceConverted)
+}
+
+func (t TimestampLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC || t.forceConverted {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimestampMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimestampMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimestampLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+
+	switch timeUnitFromThrift(t.typ.Unit) {
+	case TimeUnitMillis:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMillis
+		}
+	case TimeUnitMicros:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (TimestampLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int64
+}
+
+func (t TimestampLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIMESTAMP: t.typ}
+}
+
+func (t TimestampLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimestampLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewIntLogicalType creates an integer logical type of the desired bitwidth
+// and whether it is signed or not.
+//
+// Bit width must be exactly 8, 16, 32 or 64 for an integer logical type
+func NewIntLogicalType(bitWidth int8, signed bool) LogicalType {
+	switch bitWidth {
+	case 8, 16, 32, 64:
+	default:
+		panic("parquet: bit width must be exactly 8, 16, 32, or 64 for Int logical type")
+	}
+	return &IntLogicalType{
+		typ: &format.IntType{
+			BitWidth: bitWidth,
+			IsSigned: signed,
+		},
+	}
+}
+
+// IntLogicalType represents an integer type of a specific bit width and
+// is either signed or unsigned.
+type IntLogicalType struct {
+	baseLogicalType
+	typ *format.IntType
+}
+
+func (t IntLogicalType) BitWidth() int8 {
+	return t.typ.BitWidth
+}
+
+func (t IntLogicalType) IsSigned() bool {
+	return t.typ.IsSigned
+}
+
+func (t IntLogicalType) SortOrder() SortOrder {
+	if t.typ.IsSigned {
+		return SortSIGNED
+	}
+	return SortUNSIGNED
+}
+
+func (t IntLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Int", "bitWidth": t.typ.BitWidth, "isSigned": t.typ.IsSigned,
+	})
+}
+
+func (t IntLogicalType) String() string {
+	return fmt.Sprintf("Int(bitWidth=%d, isSigned=%t)", t.typ.GetBitWidth(), t.typ.GetIsSigned())
+}
+
+func (t IntLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	var d DecimalMetadata
+	if t.typ.IsSigned {
+		switch t.typ.BitWidth {
+		case 8:
+			return ConvertedTypes.Int8, d
+		case 16:
+			return ConvertedTypes.Int16, d
+		case 32:
+			return ConvertedTypes.Int32, d
+		case 64:
+			return ConvertedTypes.Int64, d
+		}
+	} else {
+		switch t.typ.BitWidth {
+		case 8:
+			return ConvertedTypes.Uint8, d
+		case 16:
+			return ConvertedTypes.Uint16, d
+		case 32:
+			return ConvertedTypes.Uint32, d
+		case 64:
+			return ConvertedTypes.Uint64, d
+		}
+	}
+	return ConvertedTypes.None, d
+}
+
+func (t IntLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	v, _ := t.ToConvertedType()
+	return c == v
+}
+
+func (t IntLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetBitWidth() <= 32) ||
+		(typ == parquet.Types.Int64 && t.typ.GetBitWidth() == 64)
+}
+
+func (t IntLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{INTEGER: t.typ}
+}
+
+func (t IntLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*IntLogicalType)
+	if !ok {
+		return false
+	}
+
+	return t.typ.GetIsSigned() == other.typ.GetIsSigned() &&
+		t.typ.GetBitWidth() == other.typ.GetBitWidth()
+}
+
+// UnknownLogicalType is a type that is essentially a placeholder for when
+// we don't know the type.
+type UnknownLogicalType struct{ baseLogicalType }
+
+func (UnknownLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (UnknownLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": UnknownLogicalType{}.String()})
+}
+
+func (UnknownLogicalType) IsValid() bool { return false }
+
+func (UnknownLogicalType) IsSerialized() bool { return false }
+
+func (UnknownLogicalType) String() string {
+	return "Unknown"
+}
+
+func (UnknownLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.NA, DecimalMetadata{}
+}
+
+func (UnknownLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.NA && !dec.IsSet
+}
+
+func (UnknownLogicalType) IsApplicable(parquet.Type, int32) bool { return true }
+
+func (UnknownLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{UNKNOWN: format.NewNullType()}
+}
+
+func (UnknownLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(UnknownLogicalType)
+	return ok
+}
+
+// JSONLogicalType represents a byte array column which is to be interpreted
+// as a JSON string.
+type JSONLogicalType struct{ baseLogicalType }
+
+func (JSONLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (JSONLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": JSONLogicalType{}.String()})
+}
+
+func (JSONLogicalType) String() string {
+	return "JSON"
+}
+
+func (JSONLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.JSON, DecimalMetadata{}
+}
+
+func (JSONLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.JSON && !dec.IsSet
+}
+
+func (JSONLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (JSONLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{JSON: format.NewJsonType()}
+}
+
+func (JSONLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(JSONLogicalType)
+	return ok
+}
+
+// BSONLogicalType represents a binary JSON string in the byte array
+type BSONLogicalType struct{ baseLogicalType }
+
+func (BSONLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (BSONLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": BSONLogicalType{}.String()})
+}
+
+func (BSONLogicalType) String() string {
+	return "BSON"
+}
+
+func (BSONLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.BSON, DecimalMetadata{}
+}
+
+func (BSONLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.BSON && !dec.IsSet
+}
+
+func (BSONLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (BSONLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{BSON: format.NewBsonType()}
+}
+
+func (BSONLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(BSONLogicalType)
+	return ok
+}
+
+// UUIDLogicalType can only be used with a FixedLength byte array column
+// that is exactly 16 bytes long
+type UUIDLogicalType struct{ baseLogicalType }
+
+func (UUIDLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (UUIDLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": UUIDLogicalType{}.String()})
+}
+
+func (UUIDLogicalType) String() string {
+	return "UUID"
+}
+
+func (UUIDLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (UUIDLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	switch c {
+	case ConvertedTypes.None, ConvertedTypes.NA:
+		return true
+	}
+	return false
+}
+
+func (UUIDLogicalType) IsApplicable(t parquet.Type, tlen int32) bool {
+	return t == parquet.Types.FixedLenByteArray && tlen == 16
+}
+
+func (UUIDLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{UUID: format.NewUUIDType()}
+}
+
+func (UUIDLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(UUIDLogicalType)
+	return ok
+}
+
+// IntervalLogicalType is not yet in the thrift spec, but represents
+// an interval time and needs to be a fixed length byte array of 12 bytes
+type IntervalLogicalType struct{ baseLogicalType }
+
+func (IntervalLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN

Review comment:
       I agree with the docs you linked to, so I think parquet.thrift is wrong in this case.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619902201



##########
File path: go/parquet/schema/converted_types.go
##########
@@ -0,0 +1,191 @@
+// 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.
+
+package schema
+
+import (
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// ConvertedType corresponds to the ConvertedType in the parquet.Thrift,
+// with added values of None and NA for handling when these values are not
+// set in the metadata
+type ConvertedType format.ConvertedType
+
+var (
+	// ConvertedTypes is a struct containing the constants for the types
+	// to make it easy to reference them while making it clear what they are
+	ConvertedTypes = struct {
+		None            ConvertedType
+		UTF8            ConvertedType
+		Map             ConvertedType
+		MapKeyValue     ConvertedType
+		List            ConvertedType
+		Enum            ConvertedType
+		Decimal         ConvertedType
+		Date            ConvertedType
+		TimeMillis      ConvertedType
+		TimeMicros      ConvertedType
+		TimestampMillis ConvertedType
+		TimestampMicros ConvertedType
+		Uint8           ConvertedType
+		Uint16          ConvertedType
+		Uint32          ConvertedType
+		Uint64          ConvertedType
+		Int8            ConvertedType
+		Int16           ConvertedType
+		Int32           ConvertedType
+		Int64           ConvertedType
+		JSON            ConvertedType
+		BSON            ConvertedType
+		Interval        ConvertedType
+		NA              ConvertedType
+	}{
+		None:            -1, // thrift enum starts at 0, so we know this will not be used
+		UTF8:            ConvertedType(format.ConvertedType_UTF8),
+		Map:             ConvertedType(format.ConvertedType_MAP),
+		MapKeyValue:     ConvertedType(format.ConvertedType_MAP_KEY_VALUE),
+		List:            ConvertedType(format.ConvertedType_LIST),
+		Enum:            ConvertedType(format.ConvertedType_ENUM),
+		Decimal:         ConvertedType(format.ConvertedType_DECIMAL),
+		Date:            ConvertedType(format.ConvertedType_DATE),
+		TimeMillis:      ConvertedType(format.ConvertedType_TIME_MILLIS),
+		TimeMicros:      ConvertedType(format.ConvertedType_TIME_MICROS),
+		TimestampMillis: ConvertedType(format.ConvertedType_TIMESTAMP_MILLIS),
+		TimestampMicros: ConvertedType(format.ConvertedType_TIMESTAMP_MICROS),
+		Uint8:           ConvertedType(format.ConvertedType_UINT_8),
+		Uint16:          ConvertedType(format.ConvertedType_UINT_16),
+		Uint32:          ConvertedType(format.ConvertedType_UINT_32),
+		Uint64:          ConvertedType(format.ConvertedType_UINT_64),
+		Int8:            ConvertedType(format.ConvertedType_INT_8),
+		Int16:           ConvertedType(format.ConvertedType_INT_16),
+		Int32:           ConvertedType(format.ConvertedType_INT_32),
+		Int64:           ConvertedType(format.ConvertedType_INT_64),
+		JSON:            ConvertedType(format.ConvertedType_JSON),
+		BSON:            ConvertedType(format.ConvertedType_BSON),
+		Interval:        ConvertedType(format.ConvertedType_INTERVAL),
+		NA:              24, // should always be the last values after Interval
+	}
+)
+
+func (p ConvertedType) String() string {
+	switch p {
+	case ConvertedTypes.None:
+		return "NONE"
+	case ConvertedTypes.NA:
+		return "UNKNOWN"
+	default:
+		return format.ConvertedType(p).String()
+	}
+}
+
+// ToLogicalType returns the correct LogicalType for the given ConvertedType, using the decimal
+// metadata provided to define the precision/scale if necessary
+func (p ConvertedType) ToLogicalType(convertedDecimal DecimalMetadata) LogicalType {
+	switch p {
+	case ConvertedTypes.UTF8:
+		return StringLogicalType{}
+	case ConvertedTypes.Map, ConvertedTypes.MapKeyValue:
+		return MapLogicalType{}
+	case ConvertedTypes.List:
+		return ListLogicalType{}
+	case ConvertedTypes.Enum:
+		return EnumLogicalType{}
+	case ConvertedTypes.Decimal:
+		return NewDecimalLogicalType(convertedDecimal.Precision, convertedDecimal.Scale)
+	case ConvertedTypes.Date:
+		return DateLogicalType{}
+	case ConvertedTypes.TimeMillis:
+		return NewTimeLogicalType(true, TimeUnitMillis)
+	case ConvertedTypes.TimeMicros:
+		return NewTimeLogicalType(true, TimeUnitMicros)
+	case ConvertedTypes.TimestampMillis:
+		t := NewTimestampLogicalType(true, TimeUnitMillis)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.TimestampMicros:
+		t := NewTimestampLogicalType(true, TimeUnitMicros)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.Interval:
+		return IntervalLogicalType{}
+	case ConvertedTypes.Int8:
+		return NewIntLogicalType(8, true)
+	case ConvertedTypes.Int16:
+		return NewIntLogicalType(16, true)
+	case ConvertedTypes.Int32:
+		return NewIntLogicalType(32, true)
+	case ConvertedTypes.Int64:
+		return NewIntLogicalType(64, true)
+	case ConvertedTypes.Uint8:
+		return NewIntLogicalType(8, false)
+	case ConvertedTypes.Uint16:
+		return NewIntLogicalType(16, false)
+	case ConvertedTypes.Uint32:
+		return NewIntLogicalType(32, false)
+	case ConvertedTypes.Uint64:
+		return NewIntLogicalType(64, false)
+	case ConvertedTypes.JSON:
+		return JSONLogicalType{}
+	case ConvertedTypes.BSON:
+		return BSONLogicalType{}
+	case ConvertedTypes.None:
+		return NoLogicalType{}
+	case ConvertedTypes.NA:
+		fallthrough
+	default:
+		return UnknownLogicalType{}
+	}
+}
+
+// GetSortOrder defaults to the sort order based on the physical type if convert
+// is ConvertedTypes.None, otherwise determines the sort order by the converted type.
+func GetSortOrder(convert ConvertedType, primitive format.Type) SortOrder {
+	if convert == ConvertedTypes.None {
+		return DefaultSortOrder(primitive)
+	}
+	switch convert {
+	case ConvertedTypes.Int8,
+		ConvertedTypes.Int16,
+		ConvertedTypes.Int32,
+		ConvertedTypes.Int64,
+		ConvertedTypes.Date,
+		ConvertedTypes.TimeMicros,
+		ConvertedTypes.TimeMillis,
+		ConvertedTypes.TimestampMicros,
+		ConvertedTypes.TimestampMillis:
+		return SortSIGNED
+	case ConvertedTypes.Uint8,
+		ConvertedTypes.Uint16,
+		ConvertedTypes.Uint32,
+		ConvertedTypes.Uint64,
+		ConvertedTypes.Enum,
+		ConvertedTypes.UTF8,
+		ConvertedTypes.BSON,
+		ConvertedTypes.JSON:
+		return SortUNSIGNED
+	case ConvertedTypes.Decimal,

Review comment:
       so i definitely copied this from the C++ implementation, i believe it's "sortUNKNOWN" because it can differ based on the underlying type, that said it means my current implementation of statistic sorting for decimals is incorrect now that i read the spec :) so i'll fix that in the rest of my code so it's correct before I put that part of the chunks up haha. 
   
   That said, given that int32 and int64 as the underling types for the decimal are signed comparison, i think i'll shift this to being marked as SortSIGNED since the spec defines decimal as sort by signed comparison.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619923604



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16

Review comment:
       comment added. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-837508041


   Yes, sorry, last week was busy with other things, I'll take a look at this and other open PRs tomorrow and Wednesday.  f you don't hear anything by wednesday evening please try to ping me again.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616819029



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16

Review comment:
       useful to comment maxDevLvl and maxRepLvl I might be misremembering but I think we might have renamed these simply to defLvl and RepLvl in the C++ code.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616823330



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {

Review comment:
       ah NM there can be no repeated fields here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r631936133



##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval
+	IsSerialized() bool
+	// Returns true if not NoLogicalType
+	IsValid() bool
+	// Returns true if it is NoType
+	IsNone() bool
+	// returns a string representation of the Logical Type
+	String() string
+	toThrift() *format.LogicalType
+	// Return the equivalent ConvertedType for legacy Parquet systems
+	ToConvertedType() (ConvertedType, DecimalMetadata)
+	// Returns true if the specified ConvertedType is compatible with this
+	// logical type
+	IsCompatible(ConvertedType, DecimalMetadata) bool
+	// Returns true if this logical type can be used with the provided physical type
+	IsApplicable(t parquet.Type, tlen int32) bool
+	// Returns true if the logical types are the same
+	Equals(LogicalType) bool
+	// Returns the default stat sort order for this logical type
+	SortOrder() SortOrder
+}
+
+// TemporalLogicalType is a smaller interface for Time based logical types
+// like Time / Timestamp
+type TemporalLogicalType interface {
+	LogicalType
+	IsAdjustedToUTC() bool
+	TimeUnit() TimeUnitType
+}
+
+// SortOrder mirrors the parquet.thrift sort order type
+type SortOrder int8
+
+// Constants for the Stat sort order definitions
+const (
+	SortSIGNED SortOrder = iota
+	SortUNSIGNED
+	SortUNKNOWN
+)
+
+// DefaultSortOrder returns the default stat sort order for the given physical type
+func DefaultSortOrder(primitive format.Type) SortOrder {
+	switch primitive {
+	case format.Type_BOOLEAN, format.Type_INT32, format.Type_INT64, format.Type_FLOAT, format.Type_DOUBLE:
+		return SortSIGNED
+	case format.Type_BYTE_ARRAY, format.Type_FIXED_LEN_BYTE_ARRAY:
+		return SortUNSIGNED
+	case format.Type_INT96:
+		fallthrough
+	default:
+		return SortUNKNOWN
+	}
+}
+
+// GetLogicalSortOrder returns the default sort order for this logical type
+// or falls back to the default sort order for the physical type if not valid
+func GetLogicalSortOrder(logical LogicalType, primitive format.Type) SortOrder {
+	switch {
+	case logical == nil || !logical.IsValid():
+		return SortUNKNOWN
+	case logical.Equals(NoLogicalType{}):
+		return DefaultSortOrder(primitive)
+	default:
+		return logical.SortOrder()
+	}
+}
+
+type baseLogicalType struct{}
+
+func (baseLogicalType) IsSerialized() bool {
+	return true
+}
+
+func (baseLogicalType) IsValid() bool {
+	return true
+}
+
+func (baseLogicalType) IsNested() bool {
+	return false
+}
+
+func (baseLogicalType) IsNone() bool { return false }
+
+// StringLogicalType is a UTF8 string, only usable with ByteArray and FixedLenByteArray
+type StringLogicalType struct{ baseLogicalType }
+
+func (StringLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (StringLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": StringLogicalType{}.String()})
+}
+
+func (StringLogicalType) String() string {
+	return "String"
+}
+
+func (StringLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.UTF8, DecimalMetadata{}
+}
+
+func (StringLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.UTF8 && !dec.IsSet
+}
+
+func (StringLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (StringLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{STRING: format.NewStringType()}
+}
+
+func (StringLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(StringLogicalType)
+	return ok
+}
+
+// MapLogicalType represents a mapped type
+type MapLogicalType struct{ baseLogicalType }
+
+func (MapLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (MapLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": MapLogicalType{}.String()})
+}
+
+func (MapLogicalType) String() string {
+	return "Map"
+}
+
+func (MapLogicalType) IsNested() bool {
+	return true
+}
+
+func (MapLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Map, DecimalMetadata{}
+}
+
+func (MapLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return (t == ConvertedTypes.Map || t == ConvertedTypes.MapKeyValue) && !dec.IsSet
+}
+
+func (MapLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (MapLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{MAP: format.NewMapType()}
+}
+
+func (MapLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(MapLogicalType)
+	return ok
+}
+
+func NewListLogicalType() LogicalType {
+	return ListLogicalType{}
+}
+
+// ListLogicalType is used for columns which are themselves nested lists
+type ListLogicalType struct{ baseLogicalType }
+
+func (ListLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (ListLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": ListLogicalType{}.String()})
+}
+
+func (ListLogicalType) String() string {
+	return "List"
+}
+
+func (ListLogicalType) IsNested() bool {
+	return true
+}
+
+func (ListLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.List, DecimalMetadata{}
+}
+
+func (ListLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.List && !dec.IsSet
+}
+
+func (ListLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (ListLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{LIST: format.NewListType()}
+}
+
+func (ListLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(ListLogicalType)
+	return ok
+}
+
+// EnumLogicalType is for representing an enum, which should be a byte array type
+type EnumLogicalType struct{ baseLogicalType }
+
+func (EnumLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (EnumLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": EnumLogicalType{}.String()})
+}
+
+func (EnumLogicalType) String() string {
+	return "Enum"
+}
+
+func (EnumLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Enum, DecimalMetadata{}
+}
+
+func (EnumLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Enum && !dec.IsSet
+}
+
+func (EnumLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (EnumLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{ENUM: format.NewEnumType()}
+}
+
+func (EnumLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(EnumLogicalType)
+	return ok
+}
+
+// NewDecimalLogicalType returns a Decimal logical type with the given
+// precision and scale.
+//
+// Panics if precision < 1 or scale is not in the range (0, precision)
+func NewDecimalLogicalType(precision int32, scale int32) LogicalType {
+	if precision < 1 {
+		panic("parquet: precision must be greater than or equal to 1 for decimal logical type")
+	}
+	if scale < 0 || scale > precision {
+		panic("parquet: scale must be a non-negative integer that does not exceed precision for decimal logical type")
+	}
+	return &DecimalLogicalType{typ: &format.DecimalType{Precision: precision, Scale: scale}}
+}
+
+// DecimalLogicalType is used to represent a decimal value of a given
+// precision and scale
+type DecimalLogicalType struct {
+	baseLogicalType
+	typ *format.DecimalType
+}
+
+func (t DecimalLogicalType) Precision() int32 {
+	return t.typ.Precision
+}
+
+func (t DecimalLogicalType) Scale() int32 {
+	return t.typ.Scale
+}
+
+func (DecimalLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t DecimalLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{"Type": "Decimal", "precision": t.typ.Precision, "scale": t.typ.Scale})
+}
+
+func (t DecimalLogicalType) String() string {
+	return fmt.Sprintf("Decimal(precision=%d, scale=%d)", t.typ.Precision, t.typ.Scale)
+}
+
+func (t DecimalLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Decimal, DecimalMetadata{IsSet: true, Scale: t.typ.GetScale(), Precision: t.typ.GetPrecision()}
+}
+
+func (t DecimalLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.Decimal &&
+		dec.IsSet && dec.Scale == t.typ.Scale && dec.Precision == t.typ.Precision
+}
+
+func (t DecimalLogicalType) IsApplicable(typ parquet.Type, tlen int32) bool {
+	switch typ {
+	case parquet.Types.Int32:
+		return 1 <= t.typ.Precision && t.typ.Precision <= 9
+	case parquet.Types.Int64:
+		if t.typ.Precision < 10 {
+			debug.Log("int64 used for decimal logical, precision is small enough to use int32")
+		}
+		return 1 <= t.typ.Precision && t.typ.Precision <= 18
+	case parquet.Types.FixedLenByteArray:
+		return t.typ.Precision <= int32(math.Floor(math.Log10(math.Pow(2.0, (8.0*float64(tlen)-1.0)))))
+	case parquet.Types.ByteArray:
+		return true
+	}
+	return false
+}
+
+func (t DecimalLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DECIMAL: t.typ}
+}
+
+func (t DecimalLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*DecimalLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.Precision == other.typ.Precision && t.typ.Scale == other.typ.Scale
+}
+
+// DateLogicalType is an int32 representing the number of days since the Unix Epoch
+// 1 January 1970
+type DateLogicalType struct{ baseLogicalType }
+
+func (DateLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (DateLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": DateLogicalType{}.String()})
+}
+
+func (DateLogicalType) String() string {
+	return "Date"
+}
+
+func (DateLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Date, DecimalMetadata{}
+}
+
+func (DateLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Date && !dec.IsSet
+}
+
+func (DateLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int32
+}
+
+func (DateLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DATE: format.NewDateType()}
+}
+
+func (DateLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(DateLogicalType)
+	return ok
+}
+
+func timeUnitFromThrift(unit *format.TimeUnit) TimeUnitType {
+	switch {
+	case unit == nil:
+		return TimeUnitUnknown
+	case unit.IsSetMILLIS():
+		return TimeUnitMillis
+	case unit.IsSetMICROS():
+		return TimeUnitMicros
+	case unit.IsSetNANOS():
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func timeUnitToString(unit *format.TimeUnit) string {
+	switch {
+	case unit == nil:
+		return "unknown"
+	case unit.IsSetMILLIS():
+		return "milliseconds"
+	case unit.IsSetMICROS():
+		return "microseconds"
+	case unit.IsSetNANOS():
+		return "nanoseconds"
+	default:
+		return "unknown"
+	}
+}
+
+func timeUnitFromString(v string) TimeUnitType {
+	switch v {
+	case "millis":
+		return TimeUnitMillis
+	case "micros":
+		return TimeUnitMicros
+	case "nanos":
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func createTimeUnit(unit TimeUnitType) *format.TimeUnit {
+	tunit := format.NewTimeUnit()
+	switch unit {
+	case TimeUnitMicros:
+		tunit.MICROS = format.NewMicroSeconds()
+	case TimeUnitMillis:
+		tunit.MILLIS = format.NewMilliSeconds()
+	case TimeUnitNanos:
+		tunit.NANOS = format.NewNanoSeconds()
+	default:
+		panic("parquet: time unit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+	}
+	return tunit
+}
+
+// NewTimeLogicalType returns a time type of the given unit.
+func NewTimeLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimeLogicalType{typ: &format.TimeType{
+		IsAdjustedToUTC: isAdjustedToUTC,
+		Unit:            createTimeUnit(unit),
+	}}
+}
+
+// TimeLogicalType is a time type without a date and must be an
+// int32 for milliseconds, or an int64 for micro or nano seconds.
+type TimeLogicalType struct {
+	baseLogicalType
+	typ *format.TimeType
+}
+
+func (t TimeLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimeLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimeLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimeLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Time", "isAdjustedToUTC": t.typ.IsAdjustedToUTC, "timeUnit": timeUnitToString(t.typ.GetUnit())})
+}
+
+func (t TimeLogicalType) String() string {
+	return fmt.Sprintf("Time(isAdjustedToUTC=%t, timeUnit=%s)", t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()))
+}
+
+func (t TimeLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimeMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimeMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimeLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return c == ConvertedTypes.TimeMillis
+		case TimeUnitMicros:
+			return c == ConvertedTypes.TimeMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (t TimeLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetUnit().IsSetMILLIS()) ||
+		(typ == parquet.Types.Int64 &&
+			(t.typ.GetUnit().IsSetMICROS() || t.typ.GetUnit().IsSetNANOS()))
+}
+
+func (t TimeLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIME: t.typ}
+}
+
+func (t TimeLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimeLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewTimestampLogicalType returns a logical timestamp type with "forceConverted"
+// set to false
+func NewTimestampLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: false,
+		fromConverted:  false,
+	}
+}
+
+// NewTimestampLogicalTypeForce returns a timestamp logical type with
+// "forceConverted" set to true
+func NewTimestampLogicalTypeForce(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: true,
+		fromConverted:  false,
+	}
+}
+
+// TimestampLogicalType represents an int64 number that can be decoded
+// into a year, month, day, hour, minute, second, and subsecond
+type TimestampLogicalType struct {
+	baseLogicalType
+	typ            *format.TimestampType
+	forceConverted bool
+	fromConverted  bool
+}
+
+func (t TimestampLogicalType) IsFromConvertedType() bool {
+	return t.fromConverted
+}
+
+func (t TimestampLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimestampLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimestampLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimestampLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type":                     "Timestamp",
+		"isAdjustedToUTC":          t.typ.IsAdjustedToUTC,
+		"timeUnit":                 timeUnitToString(t.typ.GetUnit()),
+		"is_from_converted_type":   t.fromConverted,
+		"force_set_converted_type": t.forceConverted,
+	})
+}
+
+func (t TimestampLogicalType) IsSerialized() bool {
+	return !t.fromConverted
+}
+
+func (t TimestampLogicalType) String() string {
+	return fmt.Sprintf("Timestamp(isAdjustedToUTC=%t, timeUnit=%s, is_from_converted_type=%t, force_set_converted_type=%t)",
+		t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()), t.fromConverted, t.forceConverted)
+}
+
+func (t TimestampLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC || t.forceConverted {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimestampMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimestampMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimestampLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+
+	switch timeUnitFromThrift(t.typ.Unit) {
+	case TimeUnitMillis:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMillis
+		}
+	case TimeUnitMicros:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (TimestampLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int64
+}
+
+func (t TimestampLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIMESTAMP: t.typ}
+}
+
+func (t TimestampLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimestampLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewIntLogicalType creates an integer logical type of the desired bitwidth
+// and whether it is signed or not.
+//
+// Bit width must be exactly 8, 16, 32 or 64 for an integer logical type
+func NewIntLogicalType(bitWidth int8, signed bool) LogicalType {
+	switch bitWidth {
+	case 8, 16, 32, 64:
+	default:
+		panic("parquet: bit width must be exactly 8, 16, 32, or 64 for Int logical type")
+	}
+	return &IntLogicalType{
+		typ: &format.IntType{
+			BitWidth: bitWidth,
+			IsSigned: signed,
+		},
+	}
+}
+
+// IntLogicalType represents an integer type of a specific bit width and
+// is either signed or unsigned.
+type IntLogicalType struct {
+	baseLogicalType
+	typ *format.IntType
+}
+
+func (t IntLogicalType) BitWidth() int8 {
+	return t.typ.BitWidth
+}
+
+func (t IntLogicalType) IsSigned() bool {
+	return t.typ.IsSigned
+}
+
+func (t IntLogicalType) SortOrder() SortOrder {
+	if t.typ.IsSigned {
+		return SortSIGNED
+	}
+	return SortUNSIGNED
+}
+
+func (t IntLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Int", "bitWidth": t.typ.BitWidth, "isSigned": t.typ.IsSigned,
+	})
+}
+
+func (t IntLogicalType) String() string {
+	return fmt.Sprintf("Int(bitWidth=%d, isSigned=%t)", t.typ.GetBitWidth(), t.typ.GetIsSigned())
+}
+
+func (t IntLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	var d DecimalMetadata
+	if t.typ.IsSigned {
+		switch t.typ.BitWidth {
+		case 8:
+			return ConvertedTypes.Int8, d
+		case 16:
+			return ConvertedTypes.Int16, d
+		case 32:
+			return ConvertedTypes.Int32, d
+		case 64:
+			return ConvertedTypes.Int64, d
+		}
+	} else {
+		switch t.typ.BitWidth {
+		case 8:
+			return ConvertedTypes.Uint8, d
+		case 16:
+			return ConvertedTypes.Uint16, d
+		case 32:
+			return ConvertedTypes.Uint32, d
+		case 64:
+			return ConvertedTypes.Uint64, d
+		}
+	}
+	return ConvertedTypes.None, d
+}
+
+func (t IntLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	v, _ := t.ToConvertedType()
+	return c == v
+}
+
+func (t IntLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetBitWidth() <= 32) ||
+		(typ == parquet.Types.Int64 && t.typ.GetBitWidth() == 64)
+}
+
+func (t IntLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{INTEGER: t.typ}
+}
+
+func (t IntLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*IntLogicalType)
+	if !ok {
+		return false
+	}
+
+	return t.typ.GetIsSigned() == other.typ.GetIsSigned() &&
+		t.typ.GetBitWidth() == other.typ.GetBitWidth()
+}
+
+// UnknownLogicalType is a type that is essentially a placeholder for when
+// we don't know the type.
+type UnknownLogicalType struct{ baseLogicalType }
+
+func (UnknownLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (UnknownLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": UnknownLogicalType{}.String()})
+}
+
+func (UnknownLogicalType) IsValid() bool { return false }
+
+func (UnknownLogicalType) IsSerialized() bool { return false }
+
+func (UnknownLogicalType) String() string {
+	return "Unknown"
+}
+
+func (UnknownLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.NA, DecimalMetadata{}
+}
+
+func (UnknownLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.NA && !dec.IsSet
+}
+
+func (UnknownLogicalType) IsApplicable(parquet.Type, int32) bool { return true }
+
+func (UnknownLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{UNKNOWN: format.NewNullType()}
+}
+
+func (UnknownLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(UnknownLogicalType)
+	return ok
+}
+
+// JSONLogicalType represents a byte array column which is to be interpreted
+// as a JSON string.
+type JSONLogicalType struct{ baseLogicalType }
+
+func (JSONLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (JSONLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": JSONLogicalType{}.String()})
+}
+
+func (JSONLogicalType) String() string {
+	return "JSON"
+}
+
+func (JSONLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.JSON, DecimalMetadata{}
+}
+
+func (JSONLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.JSON && !dec.IsSet
+}
+
+func (JSONLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (JSONLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{JSON: format.NewJsonType()}
+}
+
+func (JSONLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(JSONLogicalType)
+	return ok
+}
+
+// BSONLogicalType represents a binary JSON string in the byte array
+type BSONLogicalType struct{ baseLogicalType }
+
+func (BSONLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (BSONLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": BSONLogicalType{}.String()})
+}
+
+func (BSONLogicalType) String() string {
+	return "BSON"
+}
+
+func (BSONLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.BSON, DecimalMetadata{}
+}
+
+func (BSONLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.BSON && !dec.IsSet
+}
+
+func (BSONLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (BSONLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{BSON: format.NewBsonType()}
+}
+
+func (BSONLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(BSONLogicalType)
+	return ok
+}
+
+// UUIDLogicalType can only be used with a FixedLength byte array column
+// that is exactly 16 bytes long
+type UUIDLogicalType struct{ baseLogicalType }
+
+func (UUIDLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (UUIDLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": UUIDLogicalType{}.String()})
+}
+
+func (UUIDLogicalType) String() string {
+	return "UUID"
+}
+
+func (UUIDLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (UUIDLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	switch c {
+	case ConvertedTypes.None, ConvertedTypes.NA:
+		return true
+	}
+	return false
+}
+
+func (UUIDLogicalType) IsApplicable(t parquet.Type, tlen int32) bool {
+	return t == parquet.Types.FixedLenByteArray && tlen == 16
+}
+
+func (UUIDLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{UUID: format.NewUUIDType()}
+}
+
+func (UUIDLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(UUIDLogicalType)
+	return ok
+}
+
+// IntervalLogicalType is not yet in the thrift spec, but represents
+// an interval time and needs to be a fixed length byte array of 12 bytes
+type IntervalLogicalType struct{ baseLogicalType }
+
+func (IntervalLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN

Review comment:
       https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#interval
   
   > The sort order used for INTERVAL is undefined. When writing data, no min/max statistics should be saved for this type and if such non-compliant statistics are found during reading, they must be ignored.
   
   😸 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619903362



##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval

Review comment:
       https://github.com/apache/arrow/blob/master/cpp/src/parquet/parquet.thrift#L340
   
   The current parquet.thrift file has the logical type for INTERVAL commented out so the generated code doesn't contain a LogicalType for Interval and thus it's unable to be serialized. This also comes from the C++ implementation too https://github.com/apache/arrow/blob/master/cpp/src/parquet/schema.cc#L520




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616828634



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {

Review comment:
       I'll take a look, I wrote this back in january so it's possible that the C++ has been improved since then and i missed it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619883019



##########
File path: go/parquet/schema/helpers.go
##########
@@ -0,0 +1,102 @@
+// 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.
+
+package schema
+
+import "github.com/apache/arrow/go/parquet"
+
+// ListOf is a convenience helper function to create a properly structured
+// list structure according to the Parquet Spec.
+//
+// <list-repetition> group <name> (LIST) {
+//   repeated group list {
+//     <element-repetition> <element-type> element;
+//   }
+// }
+//
+// <list-repetition> can only be optional or required. panics if repeated.
+// <element-repetition> can only be optional or required. panics if repeated.
+func ListOf(n Node, rep parquet.Repetition, fieldID int32) *GroupNode {
+	if rep == parquet.Repetitions.Repeated || n.RepetitionType() == parquet.Repetitions.Repeated {
+		panic("parquet: listof repetition and element repetition must not be repeated.")
+	}
+	listName := n.Name()
+
+	switch n := n.(type) {
+	case *PrimitiveNode:
+		n.name = "element"
+	case *GroupNode:
+		n.name = "element"
+	}
+
+	return NewGroupNodeLogical(listName, rep, FieldList{
+		NewGroupNode("list", parquet.Repetitions.Repeated, FieldList{n}, -1)},

Review comment:
       literal comment please.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r617009700



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {
+			return true
+		}
+		n = n.Parent()
+	}
+	return false
+}
+
+// NewColumn returns a new column object for the given node with the provided
+// maximum definition and repetition levels.
+//
+// n MUST be a PrimitiveNode, otherwise this will panic.
+func NewColumn(n Node, maxDefinitionLvl, maxRepetitionLvl int16) *Column {
+	if n.Type() != Primitive {
+		panic("parquet: Column must be a primitive type")

Review comment:
       added




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r633676933



##########
File path: go/parquet/schema/reflection.go
##########
@@ -0,0 +1,827 @@
+// 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.
+
+package schema
+
+import (
+	"reflect"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+type taggedInfo struct {
+	Name string
+
+	Type      parquet.Type
+	KeyType   parquet.Type
+	ValueType parquet.Type
+
+	Length      int32
+	KeyLength   int32
+	ValueLength int32
+
+	Scale      int32
+	KeyScale   int32
+	ValueScale int32
+
+	Precision      int32
+	KeyPrecision   int32
+	ValuePrecision int32
+
+	FieldID      int32
+	KeyFieldID   int32
+	ValueFieldID int32
+
+	RepetitionType  parquet.Repetition
+	ValueRepetition parquet.Repetition
+
+	Converted      ConvertedType
+	KeyConverted   ConvertedType
+	ValueConverted ConvertedType
+
+	LogicalFields      map[string]string
+	KeyLogicalFields   map[string]string
+	ValueLogicalFields map[string]string
+
+	LogicalType      LogicalType
+	KeyLogicalType   LogicalType
+	ValueLogicalType LogicalType
+}
+
+func (t *taggedInfo) CopyForKey() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.KeyType
+	ret.Length = t.KeyLength
+	ret.Scale = t.KeyScale
+	ret.Precision = t.KeyPrecision
+	ret.FieldID = t.KeyFieldID
+	ret.RepetitionType = parquet.Repetitions.Required
+	ret.Converted = t.KeyConverted
+	ret.LogicalType = t.KeyLogicalType
+	return
+}
+
+func (t *taggedInfo) CopyForValue() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.ValueType
+	ret.Length = t.ValueLength
+	ret.Scale = t.ValueScale
+	ret.Precision = t.ValuePrecision
+	ret.FieldID = t.ValueFieldID
+	ret.RepetitionType = t.ValueRepetition
+	ret.Converted = t.ValueConverted
+	ret.LogicalType = t.ValueLogicalType
+	return
+}
+
+func (t *taggedInfo) UpdateLogicalTypes() {
+	processLogicalType := func(fields map[string]string, precision, scale int32) LogicalType {
+		t, ok := fields["type"]
+		if !ok {
+			return NoLogicalType{}
+		}
+
+		switch strings.ToLower(t) {
+		case "string":
+			return StringLogicalType{}
+		case "map":
+			return MapLogicalType{}
+		case "list":
+			return ListLogicalType{}
+		case "enum":
+			return EnumLogicalType{}
+		case "decimal":
+			if v, ok := fields["precision"]; ok {
+				precision = int32FromType(v)
+			}
+			if v, ok := fields["scale"]; ok {
+				scale = int32FromType(v)
+			}
+			return NewDecimalLogicalType(precision, scale)
+		case "date":
+			return DateLogicalType{}
+		case "time":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimeLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(strings.ToLower(unit)))
+		case "timestamp":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimestampLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(unit))
+		case "integer":
+			width, ok := fields["bitwidth"]
+			if !ok {
+				panic("must specify bitwidth if explicitly setting integer logical type")
+			}
+			signed, ok := fields["signed"]
+			if !ok {
+				signed = "true"
+			}
+
+			return NewIntLogicalType(int8(int32FromType(width)), boolFromStr(signed))
+		case "null":
+			return NullLogicalType{}
+		case "json":
+			return JSONLogicalType{}
+		case "bson":
+			return BSONLogicalType{}
+		case "uuid":
+			return UUIDLogicalType{}
+		default:
+			panic(xerrors.Errorf("invalid logical type specified: %s", t))
+		}
+	}
+
+	t.LogicalType = processLogicalType(t.LogicalFields, t.Precision, t.Scale)
+	t.KeyLogicalType = processLogicalType(t.KeyLogicalFields, t.KeyPrecision, t.KeyScale)
+	t.ValueLogicalType = processLogicalType(t.ValueLogicalFields, t.ValuePrecision, t.ValueScale)
+}
+
+func newTaggedInfo() taggedInfo {
+	return taggedInfo{
+		Type:               parquet.Types.Undefined,
+		KeyType:            parquet.Types.Undefined,
+		ValueType:          parquet.Types.Undefined,
+		RepetitionType:     parquet.Repetitions.Undefined,
+		ValueRepetition:    parquet.Repetitions.Undefined,
+		Converted:          ConvertedTypes.NA,
+		KeyConverted:       ConvertedTypes.NA,
+		ValueConverted:     ConvertedTypes.NA,
+		FieldID:            -1,
+		KeyFieldID:         -1,
+		ValueFieldID:       -1,
+		LogicalFields:      make(map[string]string),
+		KeyLogicalFields:   make(map[string]string),
+		ValueLogicalFields: make(map[string]string),
+		LogicalType:        NoLogicalType{},
+		KeyLogicalType:     NoLogicalType{},
+		ValueLogicalType:   NoLogicalType{},
+	}
+}
+
+var int32FromType = func(v string) int32 {
+	val, err := strconv.Atoi(v)
+	if err != nil {
+		panic(err)
+	}
+	return int32(val)
+}
+
+var boolFromStr = func(v string) bool {
+	val, err := strconv.ParseBool(v)
+	if err != nil {
+		panic(err)
+	}
+	return val
+}
+
+func infoFromTags(f reflect.StructTag) *taggedInfo {
+	typeFromStr := func(v string) parquet.Type {
+		t, err := format.TypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(xerrors.Errorf("invalid type specified: %s", v))
+		}
+		return parquet.Type(t)
+	}
+
+	repFromStr := func(v string) parquet.Repetition {
+		r, err := format.FieldRepetitionTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return parquet.Repetition(r)
+	}
+
+	convertedFromStr := func(v string) ConvertedType {
+		c, err := format.ConvertedTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return ConvertedType(c)
+	}
+
+	if ptags, ok := f.Lookup("parquet"); ok {
+		info := newTaggedInfo()
+		for _, tag := range strings.Split(strings.Replace(ptags, "\t", "", -1), ",") {
+			tag = strings.TrimSpace(tag)
+			kv := strings.SplitN(tag, "=", 2)
+			key := strings.TrimSpace(strings.ToLower(kv[0]))
+			value := strings.TrimSpace(kv[1])
+
+			switch key {
+			case "name":
+				info.Name = value
+			case "type":
+				info.Type = typeFromStr(value)
+			case "keytype":
+				info.KeyType = typeFromStr(value)
+			case "valuetype":
+				info.ValueType = typeFromStr(value)
+			case "length":
+				info.Length = int32FromType(value)
+			case "keylength":
+				info.KeyLength = int32FromType(value)
+			case "valuelength":
+				info.ValueLength = int32FromType(value)
+			case "scale":
+				info.Scale = int32FromType(value)
+			case "keyscale":
+				info.KeyScale = int32FromType(value)
+			case "valuescale":
+				info.ValueScale = int32FromType(value)
+			case "precision":
+				info.Precision = int32FromType(value)
+			case "keyprecision":
+				info.KeyPrecision = int32FromType(value)
+			case "valueprecision":
+				info.ValuePrecision = int32FromType(value)
+			case "fieldid":
+				info.FieldID = int32FromType(value)
+			case "keyfieldid":
+				info.KeyFieldID = int32FromType(value)
+			case "valuefieldid":
+				info.ValueFieldID = int32FromType(value)
+			case "repetition":
+				info.RepetitionType = repFromStr(value)
+			case "valuerepetition":
+				info.ValueRepetition = repFromStr(value)
+			case "converted":
+				info.Converted = convertedFromStr(value)
+			case "keyconverted":
+				info.KeyConverted = convertedFromStr(value)
+			case "valueconverted":
+				info.ValueConverted = convertedFromStr(value)
+			case "logical":
+				info.LogicalFields["type"] = value
+			case "keylogical":
+				info.KeyLogicalFields["type"] = value
+			case "valuelogical":
+				info.ValueLogicalFields["type"] = value
+			default:
+				switch {
+				case strings.HasPrefix(key, "logical."):
+					info.LogicalFields[strings.TrimPrefix(key, "logical.")] = value
+				case strings.HasPrefix(key, "keylogical."):
+					info.KeyLogicalFields[strings.TrimPrefix(key, "keylogical.")] = value
+				case strings.HasPrefix(key, "valuelogical."):
+					info.ValueLogicalFields[strings.TrimPrefix(key, "valuelogical.")] = value
+				}
+			}
+		}
+		info.UpdateLogicalTypes()
+		return &info
+	}
+	return nil
+}
+
+// typeToNode recurseively converts a physical type and the tag info into parquet Nodes
+//
+// to avoid having to propagate errors up potentially high numbers of recursive calls
+// we use panics and then recover in the public function NewSchemaFromStruct so that a
+// failure very far down the stack quickly unwinds.
+func typeToNode(name string, typ reflect.Type, repType parquet.Repetition, info *taggedInfo) Node {
+	// set up our default values for everything
+	var (
+		converted             = ConvertedTypes.None
+		logical   LogicalType = NoLogicalType{}
+		fieldID               = int32(-1)
+		physical              = parquet.Types.Undefined
+		typeLen               = 0
+		precision             = 0
+		scale                 = 0
+	)
+	if info != nil { // we have struct tag info to process
+		fieldID = info.FieldID

Review comment:
       Should I modify this to no longer auto-generate the fieldIDs? is there a specific reason why the C++ code won't generate them anymore?

##########
File path: go/parquet/schema/node.go
##########
@@ -0,0 +1,625 @@
+// 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.
+
+package schema
+
+import (
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/thrift/lib/go/thrift"
+	"golang.org/x/xerrors"
+)
+
+// NodeType describes whether the Node is a Primitive or Group node
+type NodeType int
+
+// the available constants for NodeType
+const (
+	Primitive NodeType = iota
+	Group
+)
+
+// Node is the interface for both Group and Primitive Nodes.
+// A logical schema type has a name, repetition level, and optionally
+// a logical type (converted type is the deprecated version of the logical
+// type concept, which is maintained for forward compatibility)
+type Node interface {
+	Name() string
+	Type() NodeType
+	RepetitionType() parquet.Repetition
+	ConvertedType() ConvertedType
+	LogicalType() LogicalType
+	FieldID() int32
+	Parent() Node
+	SetParent(Node)
+	Path() string
+	Equals(Node) bool
+	Visit(v Visitor)
+	toThrift() *format.SchemaElement
+}
+
+// Visitor is an interface for creating functionality to walk the schema tree.
+//
+// A visitor can be passed to the Visit function of a Node in order to walk
+// the tree. VisitPre is called the first time a node is encountered. If
+// it is a group node, the return is checked and if it is false, the children
+// will be skipped.
+//
+// VisitPost is called after visiting any children
+type Visitor interface {
+	VisitPre(Node) bool
+	VisitPost(Node)
+}
+
+// ColumnPathFromNode walks the parents of the given node to construct it's
+// column path
+func ColumnPathFromNode(n Node) parquet.ColumnPath {
+	if n == nil {
+		return nil
+	}
+
+	c := make([]string, 0)
+
+	// build the path in reverse order as we traverse nodes to the top
+	cursor := n
+	for cursor.Parent() != nil {
+		c = append(c, cursor.Name())
+		cursor = cursor.Parent()
+	}
+
+	// reverse the order of the list in place so that our result
+	// is in the proper, correct order.
+	for i := len(c)/2 - 1; i >= 0; i-- {

Review comment:
       Nope, there's no built-in reverse for slices. There's a few simple algorithmic things like that which exist in the C++ stdlib but Go doesn't have simply because of the philosophy Go has in trying not to hide complexity where it can. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r631940375



##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval
+	IsSerialized() bool
+	// Returns true if not NoLogicalType
+	IsValid() bool
+	// Returns true if it is NoType
+	IsNone() bool
+	// returns a string representation of the Logical Type
+	String() string
+	toThrift() *format.LogicalType
+	// Return the equivalent ConvertedType for legacy Parquet systems
+	ToConvertedType() (ConvertedType, DecimalMetadata)
+	// Returns true if the specified ConvertedType is compatible with this
+	// logical type
+	IsCompatible(ConvertedType, DecimalMetadata) bool
+	// Returns true if this logical type can be used with the provided physical type
+	IsApplicable(t parquet.Type, tlen int32) bool
+	// Returns true if the logical types are the same
+	Equals(LogicalType) bool
+	// Returns the default stat sort order for this logical type
+	SortOrder() SortOrder
+}
+
+// TemporalLogicalType is a smaller interface for Time based logical types
+// like Time / Timestamp
+type TemporalLogicalType interface {
+	LogicalType
+	IsAdjustedToUTC() bool
+	TimeUnit() TimeUnitType
+}
+
+// SortOrder mirrors the parquet.thrift sort order type
+type SortOrder int8
+
+// Constants for the Stat sort order definitions
+const (
+	SortSIGNED SortOrder = iota
+	SortUNSIGNED
+	SortUNKNOWN
+)
+
+// DefaultSortOrder returns the default stat sort order for the given physical type
+func DefaultSortOrder(primitive format.Type) SortOrder {
+	switch primitive {
+	case format.Type_BOOLEAN, format.Type_INT32, format.Type_INT64, format.Type_FLOAT, format.Type_DOUBLE:
+		return SortSIGNED
+	case format.Type_BYTE_ARRAY, format.Type_FIXED_LEN_BYTE_ARRAY:
+		return SortUNSIGNED
+	case format.Type_INT96:
+		fallthrough
+	default:
+		return SortUNKNOWN
+	}
+}
+
+// GetLogicalSortOrder returns the default sort order for this logical type
+// or falls back to the default sort order for the physical type if not valid
+func GetLogicalSortOrder(logical LogicalType, primitive format.Type) SortOrder {
+	switch {
+	case logical == nil || !logical.IsValid():
+		return SortUNKNOWN
+	case logical.Equals(NoLogicalType{}):
+		return DefaultSortOrder(primitive)
+	default:
+		return logical.SortOrder()
+	}
+}
+
+type baseLogicalType struct{}
+
+func (baseLogicalType) IsSerialized() bool {
+	return true
+}
+
+func (baseLogicalType) IsValid() bool {
+	return true
+}
+
+func (baseLogicalType) IsNested() bool {
+	return false
+}
+
+func (baseLogicalType) IsNone() bool { return false }
+
+// StringLogicalType is a UTF8 string, only usable with ByteArray and FixedLenByteArray
+type StringLogicalType struct{ baseLogicalType }
+
+func (StringLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (StringLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": StringLogicalType{}.String()})
+}
+
+func (StringLogicalType) String() string {
+	return "String"
+}
+
+func (StringLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.UTF8, DecimalMetadata{}
+}
+
+func (StringLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.UTF8 && !dec.IsSet
+}
+
+func (StringLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (StringLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{STRING: format.NewStringType()}
+}
+
+func (StringLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(StringLogicalType)
+	return ok
+}
+
+// MapLogicalType represents a mapped type
+type MapLogicalType struct{ baseLogicalType }
+
+func (MapLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (MapLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": MapLogicalType{}.String()})
+}
+
+func (MapLogicalType) String() string {
+	return "Map"
+}
+
+func (MapLogicalType) IsNested() bool {
+	return true
+}
+
+func (MapLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Map, DecimalMetadata{}
+}
+
+func (MapLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return (t == ConvertedTypes.Map || t == ConvertedTypes.MapKeyValue) && !dec.IsSet
+}
+
+func (MapLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (MapLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{MAP: format.NewMapType()}
+}
+
+func (MapLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(MapLogicalType)
+	return ok
+}
+
+func NewListLogicalType() LogicalType {
+	return ListLogicalType{}
+}
+
+// ListLogicalType is used for columns which are themselves nested lists
+type ListLogicalType struct{ baseLogicalType }
+
+func (ListLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (ListLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": ListLogicalType{}.String()})
+}
+
+func (ListLogicalType) String() string {
+	return "List"
+}
+
+func (ListLogicalType) IsNested() bool {
+	return true
+}
+
+func (ListLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.List, DecimalMetadata{}
+}
+
+func (ListLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.List && !dec.IsSet
+}
+
+func (ListLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (ListLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{LIST: format.NewListType()}
+}
+
+func (ListLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(ListLogicalType)
+	return ok
+}
+
+// EnumLogicalType is for representing an enum, which should be a byte array type
+type EnumLogicalType struct{ baseLogicalType }
+
+func (EnumLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (EnumLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": EnumLogicalType{}.String()})
+}
+
+func (EnumLogicalType) String() string {
+	return "Enum"
+}
+
+func (EnumLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Enum, DecimalMetadata{}
+}
+
+func (EnumLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Enum && !dec.IsSet
+}
+
+func (EnumLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (EnumLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{ENUM: format.NewEnumType()}
+}
+
+func (EnumLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(EnumLogicalType)
+	return ok
+}
+
+// NewDecimalLogicalType returns a Decimal logical type with the given
+// precision and scale.
+//
+// Panics if precision < 1 or scale is not in the range (0, precision)
+func NewDecimalLogicalType(precision int32, scale int32) LogicalType {
+	if precision < 1 {
+		panic("parquet: precision must be greater than or equal to 1 for decimal logical type")
+	}
+	if scale < 0 || scale > precision {
+		panic("parquet: scale must be a non-negative integer that does not exceed precision for decimal logical type")
+	}
+	return &DecimalLogicalType{typ: &format.DecimalType{Precision: precision, Scale: scale}}
+}
+
+// DecimalLogicalType is used to represent a decimal value of a given
+// precision and scale
+type DecimalLogicalType struct {
+	baseLogicalType
+	typ *format.DecimalType
+}
+
+func (t DecimalLogicalType) Precision() int32 {
+	return t.typ.Precision
+}
+
+func (t DecimalLogicalType) Scale() int32 {
+	return t.typ.Scale
+}
+
+func (DecimalLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t DecimalLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{"Type": "Decimal", "precision": t.typ.Precision, "scale": t.typ.Scale})
+}
+
+func (t DecimalLogicalType) String() string {
+	return fmt.Sprintf("Decimal(precision=%d, scale=%d)", t.typ.Precision, t.typ.Scale)
+}
+
+func (t DecimalLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Decimal, DecimalMetadata{IsSet: true, Scale: t.typ.GetScale(), Precision: t.typ.GetPrecision()}
+}
+
+func (t DecimalLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.Decimal &&
+		dec.IsSet && dec.Scale == t.typ.Scale && dec.Precision == t.typ.Precision
+}
+
+func (t DecimalLogicalType) IsApplicable(typ parquet.Type, tlen int32) bool {
+	switch typ {
+	case parquet.Types.Int32:
+		return 1 <= t.typ.Precision && t.typ.Precision <= 9
+	case parquet.Types.Int64:
+		if t.typ.Precision < 10 {
+			debug.Log("int64 used for decimal logical, precision is small enough to use int32")
+		}
+		return 1 <= t.typ.Precision && t.typ.Precision <= 18
+	case parquet.Types.FixedLenByteArray:
+		return t.typ.Precision <= int32(math.Floor(math.Log10(math.Pow(2.0, (8.0*float64(tlen)-1.0)))))
+	case parquet.Types.ByteArray:
+		return true
+	}
+	return false
+}
+
+func (t DecimalLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DECIMAL: t.typ}
+}
+
+func (t DecimalLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*DecimalLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.Precision == other.typ.Precision && t.typ.Scale == other.typ.Scale
+}
+
+// DateLogicalType is an int32 representing the number of days since the Unix Epoch
+// 1 January 1970
+type DateLogicalType struct{ baseLogicalType }
+
+func (DateLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (DateLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": DateLogicalType{}.String()})
+}
+
+func (DateLogicalType) String() string {
+	return "Date"
+}
+
+func (DateLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Date, DecimalMetadata{}
+}
+
+func (DateLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Date && !dec.IsSet
+}
+
+func (DateLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int32
+}
+
+func (DateLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DATE: format.NewDateType()}
+}
+
+func (DateLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(DateLogicalType)
+	return ok
+}
+
+func timeUnitFromThrift(unit *format.TimeUnit) TimeUnitType {
+	switch {
+	case unit == nil:
+		return TimeUnitUnknown
+	case unit.IsSetMILLIS():
+		return TimeUnitMillis
+	case unit.IsSetMICROS():
+		return TimeUnitMicros
+	case unit.IsSetNANOS():
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func timeUnitToString(unit *format.TimeUnit) string {
+	switch {
+	case unit == nil:
+		return "unknown"
+	case unit.IsSetMILLIS():
+		return "milliseconds"
+	case unit.IsSetMICROS():
+		return "microseconds"
+	case unit.IsSetNANOS():
+		return "nanoseconds"
+	default:
+		return "unknown"
+	}
+}
+
+func timeUnitFromString(v string) TimeUnitType {
+	switch v {
+	case "millis":
+		return TimeUnitMillis
+	case "micros":
+		return TimeUnitMicros
+	case "nanos":
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func createTimeUnit(unit TimeUnitType) *format.TimeUnit {
+	tunit := format.NewTimeUnit()
+	switch unit {
+	case TimeUnitMicros:
+		tunit.MICROS = format.NewMicroSeconds()
+	case TimeUnitMillis:
+		tunit.MILLIS = format.NewMilliSeconds()
+	case TimeUnitNanos:
+		tunit.NANOS = format.NewNanoSeconds()
+	default:
+		panic("parquet: time unit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+	}
+	return tunit
+}
+
+// NewTimeLogicalType returns a time type of the given unit.
+func NewTimeLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimeLogicalType{typ: &format.TimeType{
+		IsAdjustedToUTC: isAdjustedToUTC,
+		Unit:            createTimeUnit(unit),
+	}}
+}
+
+// TimeLogicalType is a time type without a date and must be an
+// int32 for milliseconds, or an int64 for micro or nano seconds.
+type TimeLogicalType struct {
+	baseLogicalType
+	typ *format.TimeType
+}
+
+func (t TimeLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimeLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimeLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimeLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Time", "isAdjustedToUTC": t.typ.IsAdjustedToUTC, "timeUnit": timeUnitToString(t.typ.GetUnit())})
+}
+
+func (t TimeLogicalType) String() string {
+	return fmt.Sprintf("Time(isAdjustedToUTC=%t, timeUnit=%s)", t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()))
+}
+
+func (t TimeLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimeMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimeMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimeLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return c == ConvertedTypes.TimeMillis
+		case TimeUnitMicros:
+			return c == ConvertedTypes.TimeMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (t TimeLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetUnit().IsSetMILLIS()) ||
+		(typ == parquet.Types.Int64 &&
+			(t.typ.GetUnit().IsSetMICROS() || t.typ.GetUnit().IsSetNANOS()))
+}
+
+func (t TimeLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIME: t.typ}
+}
+
+func (t TimeLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimeLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewTimestampLogicalType returns a logical timestamp type with "forceConverted"
+// set to false
+func NewTimestampLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: false,
+		fromConverted:  false,
+	}
+}
+
+// NewTimestampLogicalTypeForce returns a timestamp logical type with
+// "forceConverted" set to true
+func NewTimestampLogicalTypeForce(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: true,
+		fromConverted:  false,
+	}
+}
+
+// TimestampLogicalType represents an int64 number that can be decoded
+// into a year, month, day, hour, minute, second, and subsecond
+type TimestampLogicalType struct {
+	baseLogicalType
+	typ            *format.TimestampType
+	forceConverted bool
+	fromConverted  bool
+}
+
+func (t TimestampLogicalType) IsFromConvertedType() bool {
+	return t.fromConverted
+}
+
+func (t TimestampLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimestampLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimestampLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimestampLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type":                     "Timestamp",
+		"isAdjustedToUTC":          t.typ.IsAdjustedToUTC,
+		"timeUnit":                 timeUnitToString(t.typ.GetUnit()),
+		"is_from_converted_type":   t.fromConverted,
+		"force_set_converted_type": t.forceConverted,
+	})
+}
+
+func (t TimestampLogicalType) IsSerialized() bool {
+	return !t.fromConverted
+}
+
+func (t TimestampLogicalType) String() string {
+	return fmt.Sprintf("Timestamp(isAdjustedToUTC=%t, timeUnit=%s, is_from_converted_type=%t, force_set_converted_type=%t)",
+		t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()), t.fromConverted, t.forceConverted)
+}
+
+func (t TimestampLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC || t.forceConverted {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimestampMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimestampMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimestampLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+
+	switch timeUnitFromThrift(t.typ.Unit) {
+	case TimeUnitMillis:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMillis
+		}
+	case TimeUnitMicros:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (TimestampLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int64
+}
+
+func (t TimestampLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIMESTAMP: t.typ}
+}
+
+func (t TimestampLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimestampLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewIntLogicalType creates an integer logical type of the desired bitwidth
+// and whether it is signed or not.
+//
+// Bit width must be exactly 8, 16, 32 or 64 for an integer logical type
+func NewIntLogicalType(bitWidth int8, signed bool) LogicalType {
+	switch bitWidth {
+	case 8, 16, 32, 64:
+	default:
+		panic("parquet: bit width must be exactly 8, 16, 32, or 64 for Int logical type")
+	}
+	return &IntLogicalType{
+		typ: &format.IntType{
+			BitWidth: bitWidth,
+			IsSigned: signed,
+		},
+	}
+}
+
+// IntLogicalType represents an integer type of a specific bit width and
+// is either signed or unsigned.
+type IntLogicalType struct {
+	baseLogicalType
+	typ *format.IntType
+}
+
+func (t IntLogicalType) BitWidth() int8 {
+	return t.typ.BitWidth
+}
+
+func (t IntLogicalType) IsSigned() bool {
+	return t.typ.IsSigned
+}
+
+func (t IntLogicalType) SortOrder() SortOrder {
+	if t.typ.IsSigned {
+		return SortSIGNED
+	}
+	return SortUNSIGNED
+}
+
+func (t IntLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Int", "bitWidth": t.typ.BitWidth, "isSigned": t.typ.IsSigned,
+	})
+}
+
+func (t IntLogicalType) String() string {
+	return fmt.Sprintf("Int(bitWidth=%d, isSigned=%t)", t.typ.GetBitWidth(), t.typ.GetIsSigned())
+}
+
+func (t IntLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	var d DecimalMetadata
+	if t.typ.IsSigned {
+		switch t.typ.BitWidth {
+		case 8:
+			return ConvertedTypes.Int8, d
+		case 16:
+			return ConvertedTypes.Int16, d
+		case 32:
+			return ConvertedTypes.Int32, d
+		case 64:
+			return ConvertedTypes.Int64, d
+		}
+	} else {
+		switch t.typ.BitWidth {
+		case 8:
+			return ConvertedTypes.Uint8, d
+		case 16:
+			return ConvertedTypes.Uint16, d
+		case 32:
+			return ConvertedTypes.Uint32, d
+		case 64:
+			return ConvertedTypes.Uint64, d
+		}
+	}
+	return ConvertedTypes.None, d
+}
+
+func (t IntLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	v, _ := t.ToConvertedType()
+	return c == v
+}
+
+func (t IntLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetBitWidth() <= 32) ||
+		(typ == parquet.Types.Int64 && t.typ.GetBitWidth() == 64)
+}
+
+func (t IntLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{INTEGER: t.typ}
+}
+
+func (t IntLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*IntLogicalType)
+	if !ok {
+		return false
+	}
+
+	return t.typ.GetIsSigned() == other.typ.GetIsSigned() &&
+		t.typ.GetBitWidth() == other.typ.GetBitWidth()
+}
+
+// UnknownLogicalType is a type that is essentially a placeholder for when
+// we don't know the type.
+type UnknownLogicalType struct{ baseLogicalType }
+
+func (UnknownLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (UnknownLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": UnknownLogicalType{}.String()})
+}
+
+func (UnknownLogicalType) IsValid() bool { return false }
+
+func (UnknownLogicalType) IsSerialized() bool { return false }
+
+func (UnknownLogicalType) String() string {
+	return "Unknown"
+}
+
+func (UnknownLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.NA, DecimalMetadata{}
+}
+
+func (UnknownLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.NA && !dec.IsSet
+}
+
+func (UnknownLogicalType) IsApplicable(parquet.Type, int32) bool { return true }
+
+func (UnknownLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{UNKNOWN: format.NewNullType()}
+}
+
+func (UnknownLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(UnknownLogicalType)
+	return ok
+}
+
+// JSONLogicalType represents a byte array column which is to be interpreted
+// as a JSON string.
+type JSONLogicalType struct{ baseLogicalType }
+
+func (JSONLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (JSONLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": JSONLogicalType{}.String()})
+}
+
+func (JSONLogicalType) String() string {
+	return "JSON"
+}
+
+func (JSONLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.JSON, DecimalMetadata{}
+}
+
+func (JSONLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.JSON && !dec.IsSet
+}
+
+func (JSONLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (JSONLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{JSON: format.NewJsonType()}
+}
+
+func (JSONLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(JSONLogicalType)
+	return ok
+}
+
+// BSONLogicalType represents a binary JSON string in the byte array
+type BSONLogicalType struct{ baseLogicalType }
+
+func (BSONLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (BSONLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": BSONLogicalType{}.String()})
+}
+
+func (BSONLogicalType) String() string {
+	return "BSON"
+}
+
+func (BSONLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.BSON, DecimalMetadata{}
+}
+
+func (BSONLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.BSON && !dec.IsSet
+}
+
+func (BSONLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (BSONLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{BSON: format.NewBsonType()}
+}
+
+func (BSONLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(BSONLogicalType)
+	return ok
+}
+
+// UUIDLogicalType can only be used with a FixedLength byte array column
+// that is exactly 16 bytes long
+type UUIDLogicalType struct{ baseLogicalType }
+
+func (UUIDLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (UUIDLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": UUIDLogicalType{}.String()})
+}
+
+func (UUIDLogicalType) String() string {
+	return "UUID"
+}
+
+func (UUIDLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (UUIDLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	switch c {
+	case ConvertedTypes.None, ConvertedTypes.NA:
+		return true
+	}
+	return false
+}
+
+func (UUIDLogicalType) IsApplicable(t parquet.Type, tlen int32) bool {
+	return t == parquet.Types.FixedLenByteArray && tlen == 16
+}
+
+func (UUIDLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{UUID: format.NewUUIDType()}
+}
+
+func (UUIDLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(UUIDLogicalType)
+	return ok
+}
+
+// IntervalLogicalType is not yet in the thrift spec, but represents
+// an interval time and needs to be a fixed length byte array of 12 bytes
+type IntervalLogicalType struct{ baseLogicalType }
+
+func (IntervalLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN

Review comment:
       Looks like a doc update is in order [parquet.thrift](https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L875) seems to state otherwise.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616817803



##########
File path: go/parquet/internal/debug/log_off.go
##########
@@ -0,0 +1,21 @@
+// 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.
+
+// +build !debug
+
+package debug
+
+func Log(interface{}) {}

Review comment:
       comment? Is this a logging interface?  does Go have a common library for this?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r631930157



##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval
+	IsSerialized() bool
+	// Returns true if not NoLogicalType
+	IsValid() bool
+	// Returns true if it is NoType
+	IsNone() bool
+	// returns a string representation of the Logical Type
+	String() string
+	toThrift() *format.LogicalType
+	// Return the equivalent ConvertedType for legacy Parquet systems
+	ToConvertedType() (ConvertedType, DecimalMetadata)
+	// Returns true if the specified ConvertedType is compatible with this
+	// logical type
+	IsCompatible(ConvertedType, DecimalMetadata) bool
+	// Returns true if this logical type can be used with the provided physical type
+	IsApplicable(t parquet.Type, tlen int32) bool
+	// Returns true if the logical types are the same
+	Equals(LogicalType) bool
+	// Returns the default stat sort order for this logical type
+	SortOrder() SortOrder
+}
+
+// TemporalLogicalType is a smaller interface for Time based logical types
+// like Time / Timestamp
+type TemporalLogicalType interface {
+	LogicalType
+	IsAdjustedToUTC() bool
+	TimeUnit() TimeUnitType
+}
+
+// SortOrder mirrors the parquet.thrift sort order type
+type SortOrder int8
+
+// Constants for the Stat sort order definitions
+const (
+	SortSIGNED SortOrder = iota
+	SortUNSIGNED
+	SortUNKNOWN
+)
+
+// DefaultSortOrder returns the default stat sort order for the given physical type
+func DefaultSortOrder(primitive format.Type) SortOrder {
+	switch primitive {
+	case format.Type_BOOLEAN, format.Type_INT32, format.Type_INT64, format.Type_FLOAT, format.Type_DOUBLE:
+		return SortSIGNED
+	case format.Type_BYTE_ARRAY, format.Type_FIXED_LEN_BYTE_ARRAY:
+		return SortUNSIGNED
+	case format.Type_INT96:
+		fallthrough
+	default:
+		return SortUNKNOWN
+	}
+}
+
+// GetLogicalSortOrder returns the default sort order for this logical type
+// or falls back to the default sort order for the physical type if not valid
+func GetLogicalSortOrder(logical LogicalType, primitive format.Type) SortOrder {
+	switch {
+	case logical == nil || !logical.IsValid():
+		return SortUNKNOWN
+	case logical.Equals(NoLogicalType{}):
+		return DefaultSortOrder(primitive)
+	default:
+		return logical.SortOrder()
+	}
+}
+
+type baseLogicalType struct{}
+
+func (baseLogicalType) IsSerialized() bool {
+	return true
+}
+
+func (baseLogicalType) IsValid() bool {
+	return true
+}
+
+func (baseLogicalType) IsNested() bool {
+	return false
+}
+
+func (baseLogicalType) IsNone() bool { return false }
+
+// StringLogicalType is a UTF8 string, only usable with ByteArray and FixedLenByteArray
+type StringLogicalType struct{ baseLogicalType }
+
+func (StringLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (StringLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": StringLogicalType{}.String()})
+}
+
+func (StringLogicalType) String() string {
+	return "String"
+}
+
+func (StringLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.UTF8, DecimalMetadata{}
+}
+
+func (StringLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.UTF8 && !dec.IsSet
+}
+
+func (StringLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (StringLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{STRING: format.NewStringType()}
+}
+
+func (StringLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(StringLogicalType)
+	return ok
+}
+
+// MapLogicalType represents a mapped type
+type MapLogicalType struct{ baseLogicalType }
+
+func (MapLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (MapLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": MapLogicalType{}.String()})
+}
+
+func (MapLogicalType) String() string {
+	return "Map"
+}
+
+func (MapLogicalType) IsNested() bool {
+	return true
+}
+
+func (MapLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Map, DecimalMetadata{}
+}
+
+func (MapLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return (t == ConvertedTypes.Map || t == ConvertedTypes.MapKeyValue) && !dec.IsSet
+}
+
+func (MapLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (MapLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{MAP: format.NewMapType()}
+}
+
+func (MapLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(MapLogicalType)
+	return ok
+}
+
+func NewListLogicalType() LogicalType {
+	return ListLogicalType{}
+}
+
+// ListLogicalType is used for columns which are themselves nested lists
+type ListLogicalType struct{ baseLogicalType }
+
+func (ListLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (ListLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": ListLogicalType{}.String()})
+}
+
+func (ListLogicalType) String() string {
+	return "List"
+}
+
+func (ListLogicalType) IsNested() bool {
+	return true
+}
+
+func (ListLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.List, DecimalMetadata{}
+}
+
+func (ListLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.List && !dec.IsSet
+}
+
+func (ListLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (ListLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{LIST: format.NewListType()}
+}
+
+func (ListLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(ListLogicalType)
+	return ok
+}
+
+// EnumLogicalType is for representing an enum, which should be a byte array type
+type EnumLogicalType struct{ baseLogicalType }
+
+func (EnumLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (EnumLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": EnumLogicalType{}.String()})
+}
+
+func (EnumLogicalType) String() string {
+	return "Enum"
+}
+
+func (EnumLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Enum, DecimalMetadata{}
+}
+
+func (EnumLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Enum && !dec.IsSet
+}
+
+func (EnumLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (EnumLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{ENUM: format.NewEnumType()}
+}
+
+func (EnumLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(EnumLogicalType)
+	return ok
+}
+
+// NewDecimalLogicalType returns a Decimal logical type with the given
+// precision and scale.
+//
+// Panics if precision < 1 or scale is not in the range (0, precision)
+func NewDecimalLogicalType(precision int32, scale int32) LogicalType {
+	if precision < 1 {
+		panic("parquet: precision must be greater than or equal to 1 for decimal logical type")
+	}
+	if scale < 0 || scale > precision {
+		panic("parquet: scale must be a non-negative integer that does not exceed precision for decimal logical type")
+	}
+	return &DecimalLogicalType{typ: &format.DecimalType{Precision: precision, Scale: scale}}
+}
+
+// DecimalLogicalType is used to represent a decimal value of a given
+// precision and scale
+type DecimalLogicalType struct {
+	baseLogicalType
+	typ *format.DecimalType
+}
+
+func (t DecimalLogicalType) Precision() int32 {
+	return t.typ.Precision
+}
+
+func (t DecimalLogicalType) Scale() int32 {
+	return t.typ.Scale
+}
+
+func (DecimalLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t DecimalLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{"Type": "Decimal", "precision": t.typ.Precision, "scale": t.typ.Scale})
+}
+
+func (t DecimalLogicalType) String() string {
+	return fmt.Sprintf("Decimal(precision=%d, scale=%d)", t.typ.Precision, t.typ.Scale)
+}
+
+func (t DecimalLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Decimal, DecimalMetadata{IsSet: true, Scale: t.typ.GetScale(), Precision: t.typ.GetPrecision()}
+}
+
+func (t DecimalLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.Decimal &&
+		dec.IsSet && dec.Scale == t.typ.Scale && dec.Precision == t.typ.Precision
+}
+
+func (t DecimalLogicalType) IsApplicable(typ parquet.Type, tlen int32) bool {
+	switch typ {
+	case parquet.Types.Int32:
+		return 1 <= t.typ.Precision && t.typ.Precision <= 9
+	case parquet.Types.Int64:
+		if t.typ.Precision < 10 {
+			debug.Log("int64 used for decimal logical, precision is small enough to use int32")
+		}
+		return 1 <= t.typ.Precision && t.typ.Precision <= 18
+	case parquet.Types.FixedLenByteArray:
+		return t.typ.Precision <= int32(math.Floor(math.Log10(math.Pow(2.0, (8.0*float64(tlen)-1.0)))))
+	case parquet.Types.ByteArray:
+		return true
+	}
+	return false
+}
+
+func (t DecimalLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DECIMAL: t.typ}
+}
+
+func (t DecimalLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*DecimalLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.Precision == other.typ.Precision && t.typ.Scale == other.typ.Scale
+}
+
+// DateLogicalType is an int32 representing the number of days since the Unix Epoch
+// 1 January 1970
+type DateLogicalType struct{ baseLogicalType }
+
+func (DateLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (DateLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": DateLogicalType{}.String()})
+}
+
+func (DateLogicalType) String() string {
+	return "Date"
+}
+
+func (DateLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Date, DecimalMetadata{}
+}
+
+func (DateLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Date && !dec.IsSet
+}
+
+func (DateLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int32
+}
+
+func (DateLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DATE: format.NewDateType()}
+}
+
+func (DateLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(DateLogicalType)
+	return ok
+}
+
+func timeUnitFromThrift(unit *format.TimeUnit) TimeUnitType {
+	switch {
+	case unit == nil:
+		return TimeUnitUnknown
+	case unit.IsSetMILLIS():
+		return TimeUnitMillis
+	case unit.IsSetMICROS():
+		return TimeUnitMicros
+	case unit.IsSetNANOS():
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func timeUnitToString(unit *format.TimeUnit) string {
+	switch {
+	case unit == nil:
+		return "unknown"
+	case unit.IsSetMILLIS():
+		return "milliseconds"
+	case unit.IsSetMICROS():
+		return "microseconds"
+	case unit.IsSetNANOS():
+		return "nanoseconds"
+	default:
+		return "unknown"
+	}
+}
+
+func timeUnitFromString(v string) TimeUnitType {
+	switch v {
+	case "millis":
+		return TimeUnitMillis
+	case "micros":
+		return TimeUnitMicros
+	case "nanos":
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func createTimeUnit(unit TimeUnitType) *format.TimeUnit {
+	tunit := format.NewTimeUnit()
+	switch unit {
+	case TimeUnitMicros:
+		tunit.MICROS = format.NewMicroSeconds()
+	case TimeUnitMillis:
+		tunit.MILLIS = format.NewMilliSeconds()
+	case TimeUnitNanos:
+		tunit.NANOS = format.NewNanoSeconds()
+	default:
+		panic("parquet: time unit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+	}
+	return tunit
+}
+
+// NewTimeLogicalType returns a time type of the given unit.
+func NewTimeLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimeLogicalType{typ: &format.TimeType{
+		IsAdjustedToUTC: isAdjustedToUTC,
+		Unit:            createTimeUnit(unit),
+	}}
+}
+
+// TimeLogicalType is a time type without a date and must be an
+// int32 for milliseconds, or an int64 for micro or nano seconds.
+type TimeLogicalType struct {
+	baseLogicalType
+	typ *format.TimeType
+}
+
+func (t TimeLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimeLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimeLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimeLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Time", "isAdjustedToUTC": t.typ.IsAdjustedToUTC, "timeUnit": timeUnitToString(t.typ.GetUnit())})
+}
+
+func (t TimeLogicalType) String() string {
+	return fmt.Sprintf("Time(isAdjustedToUTC=%t, timeUnit=%s)", t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()))
+}
+
+func (t TimeLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimeMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimeMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimeLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return c == ConvertedTypes.TimeMillis
+		case TimeUnitMicros:
+			return c == ConvertedTypes.TimeMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (t TimeLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetUnit().IsSetMILLIS()) ||
+		(typ == parquet.Types.Int64 &&
+			(t.typ.GetUnit().IsSetMICROS() || t.typ.GetUnit().IsSetNANOS()))
+}
+
+func (t TimeLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIME: t.typ}
+}
+
+func (t TimeLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimeLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewTimestampLogicalType returns a logical timestamp type with "forceConverted"
+// set to false
+func NewTimestampLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: false,
+		fromConverted:  false,
+	}
+}
+
+// NewTimestampLogicalTypeForce returns a timestamp logical type with
+// "forceConverted" set to true
+func NewTimestampLogicalTypeForce(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: true,
+		fromConverted:  false,
+	}
+}
+
+// TimestampLogicalType represents an int64 number that can be decoded
+// into a year, month, day, hour, minute, second, and subsecond
+type TimestampLogicalType struct {
+	baseLogicalType
+	typ            *format.TimestampType
+	forceConverted bool

Review comment:
       could you add comments on what forceConverted and fromConverted are?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616824254



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {
+			return true
+		}
+		n = n.Parent()
+	}
+	return false
+}
+
+// NewColumn returns a new column object for the given node with the provided
+// maximum definition and repetition levels.
+//
+// n MUST be a PrimitiveNode, otherwise this will panic.
+func NewColumn(n Node, maxDefinitionLvl, maxRepetitionLvl int16) *Column {
+	if n.Type() != Primitive {
+		panic("parquet: Column must be a primitive type")

Review comment:
       Is it worth including details of Node here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-830928131


   @nickpoorman  my latest update here significantly reduces the panics and replaces them with returning errors, but also provides convenience functions `Must`, `MustPrimitive`, and `MustGroup` which can make it easier to build schema's without having to check for errors constantly if the consumer is ok with panic'ing.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616822647



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {

Review comment:
       Does this mirror current C++ code.  I think once you hit a repeated field you can also stop.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r631950006



##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval
+	IsSerialized() bool
+	// Returns true if not NoLogicalType
+	IsValid() bool
+	// Returns true if it is NoType
+	IsNone() bool
+	// returns a string representation of the Logical Type
+	String() string
+	toThrift() *format.LogicalType
+	// Return the equivalent ConvertedType for legacy Parquet systems
+	ToConvertedType() (ConvertedType, DecimalMetadata)
+	// Returns true if the specified ConvertedType is compatible with this
+	// logical type
+	IsCompatible(ConvertedType, DecimalMetadata) bool
+	// Returns true if this logical type can be used with the provided physical type
+	IsApplicable(t parquet.Type, tlen int32) bool
+	// Returns true if the logical types are the same
+	Equals(LogicalType) bool
+	// Returns the default stat sort order for this logical type
+	SortOrder() SortOrder
+}
+
+// TemporalLogicalType is a smaller interface for Time based logical types
+// like Time / Timestamp
+type TemporalLogicalType interface {
+	LogicalType
+	IsAdjustedToUTC() bool
+	TimeUnit() TimeUnitType
+}
+
+// SortOrder mirrors the parquet.thrift sort order type
+type SortOrder int8
+
+// Constants for the Stat sort order definitions
+const (
+	SortSIGNED SortOrder = iota
+	SortUNSIGNED
+	SortUNKNOWN
+)
+
+// DefaultSortOrder returns the default stat sort order for the given physical type
+func DefaultSortOrder(primitive format.Type) SortOrder {
+	switch primitive {
+	case format.Type_BOOLEAN, format.Type_INT32, format.Type_INT64, format.Type_FLOAT, format.Type_DOUBLE:
+		return SortSIGNED
+	case format.Type_BYTE_ARRAY, format.Type_FIXED_LEN_BYTE_ARRAY:
+		return SortUNSIGNED
+	case format.Type_INT96:
+		fallthrough
+	default:
+		return SortUNKNOWN
+	}
+}
+
+// GetLogicalSortOrder returns the default sort order for this logical type
+// or falls back to the default sort order for the physical type if not valid
+func GetLogicalSortOrder(logical LogicalType, primitive format.Type) SortOrder {
+	switch {
+	case logical == nil || !logical.IsValid():
+		return SortUNKNOWN
+	case logical.Equals(NoLogicalType{}):
+		return DefaultSortOrder(primitive)
+	default:
+		return logical.SortOrder()
+	}
+}
+
+type baseLogicalType struct{}
+
+func (baseLogicalType) IsSerialized() bool {
+	return true
+}
+
+func (baseLogicalType) IsValid() bool {
+	return true
+}
+
+func (baseLogicalType) IsNested() bool {
+	return false
+}
+
+func (baseLogicalType) IsNone() bool { return false }
+
+// StringLogicalType is a UTF8 string, only usable with ByteArray and FixedLenByteArray
+type StringLogicalType struct{ baseLogicalType }
+
+func (StringLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (StringLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": StringLogicalType{}.String()})
+}
+
+func (StringLogicalType) String() string {
+	return "String"
+}
+
+func (StringLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.UTF8, DecimalMetadata{}
+}
+
+func (StringLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.UTF8 && !dec.IsSet
+}
+
+func (StringLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (StringLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{STRING: format.NewStringType()}
+}
+
+func (StringLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(StringLogicalType)
+	return ok
+}
+
+// MapLogicalType represents a mapped type
+type MapLogicalType struct{ baseLogicalType }
+
+func (MapLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (MapLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": MapLogicalType{}.String()})
+}
+
+func (MapLogicalType) String() string {
+	return "Map"
+}
+
+func (MapLogicalType) IsNested() bool {
+	return true
+}
+
+func (MapLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Map, DecimalMetadata{}
+}
+
+func (MapLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return (t == ConvertedTypes.Map || t == ConvertedTypes.MapKeyValue) && !dec.IsSet
+}
+
+func (MapLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (MapLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{MAP: format.NewMapType()}
+}
+
+func (MapLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(MapLogicalType)
+	return ok
+}
+
+func NewListLogicalType() LogicalType {
+	return ListLogicalType{}
+}
+
+// ListLogicalType is used for columns which are themselves nested lists
+type ListLogicalType struct{ baseLogicalType }
+
+func (ListLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (ListLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": ListLogicalType{}.String()})
+}
+
+func (ListLogicalType) String() string {
+	return "List"
+}
+
+func (ListLogicalType) IsNested() bool {
+	return true
+}
+
+func (ListLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.List, DecimalMetadata{}
+}
+
+func (ListLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.List && !dec.IsSet
+}
+
+func (ListLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (ListLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{LIST: format.NewListType()}
+}
+
+func (ListLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(ListLogicalType)
+	return ok
+}
+
+// EnumLogicalType is for representing an enum, which should be a byte array type
+type EnumLogicalType struct{ baseLogicalType }
+
+func (EnumLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (EnumLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": EnumLogicalType{}.String()})
+}
+
+func (EnumLogicalType) String() string {
+	return "Enum"
+}
+
+func (EnumLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Enum, DecimalMetadata{}
+}
+
+func (EnumLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Enum && !dec.IsSet
+}
+
+func (EnumLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (EnumLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{ENUM: format.NewEnumType()}
+}
+
+func (EnumLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(EnumLogicalType)
+	return ok
+}
+
+// NewDecimalLogicalType returns a Decimal logical type with the given
+// precision and scale.
+//
+// Panics if precision < 1 or scale is not in the range (0, precision)
+func NewDecimalLogicalType(precision int32, scale int32) LogicalType {
+	if precision < 1 {
+		panic("parquet: precision must be greater than or equal to 1 for decimal logical type")
+	}
+	if scale < 0 || scale > precision {
+		panic("parquet: scale must be a non-negative integer that does not exceed precision for decimal logical type")
+	}
+	return &DecimalLogicalType{typ: &format.DecimalType{Precision: precision, Scale: scale}}
+}
+
+// DecimalLogicalType is used to represent a decimal value of a given
+// precision and scale
+type DecimalLogicalType struct {
+	baseLogicalType
+	typ *format.DecimalType
+}
+
+func (t DecimalLogicalType) Precision() int32 {
+	return t.typ.Precision
+}
+
+func (t DecimalLogicalType) Scale() int32 {
+	return t.typ.Scale
+}
+
+func (DecimalLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t DecimalLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{"Type": "Decimal", "precision": t.typ.Precision, "scale": t.typ.Scale})
+}
+
+func (t DecimalLogicalType) String() string {
+	return fmt.Sprintf("Decimal(precision=%d, scale=%d)", t.typ.Precision, t.typ.Scale)
+}
+
+func (t DecimalLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Decimal, DecimalMetadata{IsSet: true, Scale: t.typ.GetScale(), Precision: t.typ.GetPrecision()}
+}
+
+func (t DecimalLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.Decimal &&
+		dec.IsSet && dec.Scale == t.typ.Scale && dec.Precision == t.typ.Precision
+}
+
+func (t DecimalLogicalType) IsApplicable(typ parquet.Type, tlen int32) bool {
+	switch typ {
+	case parquet.Types.Int32:
+		return 1 <= t.typ.Precision && t.typ.Precision <= 9
+	case parquet.Types.Int64:
+		if t.typ.Precision < 10 {
+			debug.Log("int64 used for decimal logical, precision is small enough to use int32")
+		}
+		return 1 <= t.typ.Precision && t.typ.Precision <= 18
+	case parquet.Types.FixedLenByteArray:
+		return t.typ.Precision <= int32(math.Floor(math.Log10(math.Pow(2.0, (8.0*float64(tlen)-1.0)))))
+	case parquet.Types.ByteArray:
+		return true
+	}
+	return false
+}
+
+func (t DecimalLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DECIMAL: t.typ}
+}
+
+func (t DecimalLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*DecimalLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.Precision == other.typ.Precision && t.typ.Scale == other.typ.Scale
+}
+
+// DateLogicalType is an int32 representing the number of days since the Unix Epoch
+// 1 January 1970
+type DateLogicalType struct{ baseLogicalType }
+
+func (DateLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (DateLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": DateLogicalType{}.String()})
+}
+
+func (DateLogicalType) String() string {
+	return "Date"
+}
+
+func (DateLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Date, DecimalMetadata{}
+}
+
+func (DateLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Date && !dec.IsSet
+}
+
+func (DateLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int32
+}
+
+func (DateLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DATE: format.NewDateType()}
+}
+
+func (DateLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(DateLogicalType)
+	return ok
+}
+
+func timeUnitFromThrift(unit *format.TimeUnit) TimeUnitType {
+	switch {
+	case unit == nil:
+		return TimeUnitUnknown
+	case unit.IsSetMILLIS():
+		return TimeUnitMillis
+	case unit.IsSetMICROS():
+		return TimeUnitMicros
+	case unit.IsSetNANOS():
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func timeUnitToString(unit *format.TimeUnit) string {
+	switch {
+	case unit == nil:
+		return "unknown"
+	case unit.IsSetMILLIS():
+		return "milliseconds"
+	case unit.IsSetMICROS():
+		return "microseconds"
+	case unit.IsSetNANOS():
+		return "nanoseconds"
+	default:
+		return "unknown"
+	}
+}
+
+func timeUnitFromString(v string) TimeUnitType {
+	switch v {
+	case "millis":
+		return TimeUnitMillis
+	case "micros":
+		return TimeUnitMicros
+	case "nanos":
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func createTimeUnit(unit TimeUnitType) *format.TimeUnit {
+	tunit := format.NewTimeUnit()
+	switch unit {
+	case TimeUnitMicros:
+		tunit.MICROS = format.NewMicroSeconds()
+	case TimeUnitMillis:
+		tunit.MILLIS = format.NewMilliSeconds()
+	case TimeUnitNanos:
+		tunit.NANOS = format.NewNanoSeconds()
+	default:
+		panic("parquet: time unit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+	}
+	return tunit
+}
+
+// NewTimeLogicalType returns a time type of the given unit.
+func NewTimeLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimeLogicalType{typ: &format.TimeType{
+		IsAdjustedToUTC: isAdjustedToUTC,
+		Unit:            createTimeUnit(unit),
+	}}
+}
+
+// TimeLogicalType is a time type without a date and must be an
+// int32 for milliseconds, or an int64 for micro or nano seconds.
+type TimeLogicalType struct {
+	baseLogicalType
+	typ *format.TimeType
+}
+
+func (t TimeLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimeLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimeLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimeLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Time", "isAdjustedToUTC": t.typ.IsAdjustedToUTC, "timeUnit": timeUnitToString(t.typ.GetUnit())})
+}
+
+func (t TimeLogicalType) String() string {
+	return fmt.Sprintf("Time(isAdjustedToUTC=%t, timeUnit=%s)", t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()))
+}
+
+func (t TimeLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimeMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimeMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimeLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return c == ConvertedTypes.TimeMillis
+		case TimeUnitMicros:
+			return c == ConvertedTypes.TimeMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (t TimeLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetUnit().IsSetMILLIS()) ||
+		(typ == parquet.Types.Int64 &&
+			(t.typ.GetUnit().IsSetMICROS() || t.typ.GetUnit().IsSetNANOS()))
+}
+
+func (t TimeLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIME: t.typ}
+}
+
+func (t TimeLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimeLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewTimestampLogicalType returns a logical timestamp type with "forceConverted"
+// set to false
+func NewTimestampLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: false,
+		fromConverted:  false,
+	}
+}
+
+// NewTimestampLogicalTypeForce returns a timestamp logical type with
+// "forceConverted" set to true
+func NewTimestampLogicalTypeForce(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: true,
+		fromConverted:  false,
+	}
+}
+
+// TimestampLogicalType represents an int64 number that can be decoded
+// into a year, month, day, hour, minute, second, and subsecond
+type TimestampLogicalType struct {
+	baseLogicalType
+	typ            *format.TimestampType
+	forceConverted bool

Review comment:
       comments added.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-839892994


   Sorry didn't get a chance to look at this today, will see if I can squeeze in a review tonight otherwise, I'll prioritize tomorrow morning.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619882252



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16

Review comment:
       OK.  might be worth adding, that at this level, a corresponding value is present.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-837499789


   @emkornfield @nickpoorman @sbinet any chance at getting re-reviewed and merged?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-843512955


   @emkornfield I've removed the auto-generation of the fieldIDs and updated the PR. So i think we're all set here! :smile:


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-821269124


   Tagging @emkornfield @sbinet @nickpoorman for visability


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r631935011



##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval
+	IsSerialized() bool
+	// Returns true if not NoLogicalType
+	IsValid() bool
+	// Returns true if it is NoType
+	IsNone() bool
+	// returns a string representation of the Logical Type
+	String() string
+	toThrift() *format.LogicalType
+	// Return the equivalent ConvertedType for legacy Parquet systems
+	ToConvertedType() (ConvertedType, DecimalMetadata)
+	// Returns true if the specified ConvertedType is compatible with this
+	// logical type
+	IsCompatible(ConvertedType, DecimalMetadata) bool
+	// Returns true if this logical type can be used with the provided physical type
+	IsApplicable(t parquet.Type, tlen int32) bool
+	// Returns true if the logical types are the same
+	Equals(LogicalType) bool
+	// Returns the default stat sort order for this logical type
+	SortOrder() SortOrder
+}
+
+// TemporalLogicalType is a smaller interface for Time based logical types
+// like Time / Timestamp
+type TemporalLogicalType interface {
+	LogicalType
+	IsAdjustedToUTC() bool
+	TimeUnit() TimeUnitType
+}
+
+// SortOrder mirrors the parquet.thrift sort order type
+type SortOrder int8
+
+// Constants for the Stat sort order definitions
+const (
+	SortSIGNED SortOrder = iota
+	SortUNSIGNED
+	SortUNKNOWN
+)
+
+// DefaultSortOrder returns the default stat sort order for the given physical type
+func DefaultSortOrder(primitive format.Type) SortOrder {
+	switch primitive {
+	case format.Type_BOOLEAN, format.Type_INT32, format.Type_INT64, format.Type_FLOAT, format.Type_DOUBLE:
+		return SortSIGNED
+	case format.Type_BYTE_ARRAY, format.Type_FIXED_LEN_BYTE_ARRAY:
+		return SortUNSIGNED
+	case format.Type_INT96:
+		fallthrough
+	default:
+		return SortUNKNOWN
+	}
+}
+
+// GetLogicalSortOrder returns the default sort order for this logical type
+// or falls back to the default sort order for the physical type if not valid
+func GetLogicalSortOrder(logical LogicalType, primitive format.Type) SortOrder {
+	switch {
+	case logical == nil || !logical.IsValid():
+		return SortUNKNOWN
+	case logical.Equals(NoLogicalType{}):
+		return DefaultSortOrder(primitive)
+	default:
+		return logical.SortOrder()
+	}
+}
+
+type baseLogicalType struct{}
+
+func (baseLogicalType) IsSerialized() bool {
+	return true
+}
+
+func (baseLogicalType) IsValid() bool {
+	return true
+}
+
+func (baseLogicalType) IsNested() bool {
+	return false
+}
+
+func (baseLogicalType) IsNone() bool { return false }
+
+// StringLogicalType is a UTF8 string, only usable with ByteArray and FixedLenByteArray
+type StringLogicalType struct{ baseLogicalType }
+
+func (StringLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (StringLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": StringLogicalType{}.String()})
+}
+
+func (StringLogicalType) String() string {
+	return "String"
+}
+
+func (StringLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.UTF8, DecimalMetadata{}
+}
+
+func (StringLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.UTF8 && !dec.IsSet
+}
+
+func (StringLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (StringLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{STRING: format.NewStringType()}
+}
+
+func (StringLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(StringLogicalType)
+	return ok
+}
+
+// MapLogicalType represents a mapped type
+type MapLogicalType struct{ baseLogicalType }
+
+func (MapLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (MapLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": MapLogicalType{}.String()})
+}
+
+func (MapLogicalType) String() string {
+	return "Map"
+}
+
+func (MapLogicalType) IsNested() bool {
+	return true
+}
+
+func (MapLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Map, DecimalMetadata{}
+}
+
+func (MapLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return (t == ConvertedTypes.Map || t == ConvertedTypes.MapKeyValue) && !dec.IsSet
+}
+
+func (MapLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (MapLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{MAP: format.NewMapType()}
+}
+
+func (MapLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(MapLogicalType)
+	return ok
+}
+
+func NewListLogicalType() LogicalType {
+	return ListLogicalType{}
+}
+
+// ListLogicalType is used for columns which are themselves nested lists
+type ListLogicalType struct{ baseLogicalType }
+
+func (ListLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (ListLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": ListLogicalType{}.String()})
+}
+
+func (ListLogicalType) String() string {
+	return "List"
+}
+
+func (ListLogicalType) IsNested() bool {
+	return true
+}
+
+func (ListLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.List, DecimalMetadata{}
+}
+
+func (ListLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.List && !dec.IsSet
+}
+
+func (ListLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (ListLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{LIST: format.NewListType()}
+}
+
+func (ListLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(ListLogicalType)
+	return ok
+}
+
+// EnumLogicalType is for representing an enum, which should be a byte array type
+type EnumLogicalType struct{ baseLogicalType }
+
+func (EnumLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (EnumLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": EnumLogicalType{}.String()})
+}
+
+func (EnumLogicalType) String() string {
+	return "Enum"
+}
+
+func (EnumLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Enum, DecimalMetadata{}
+}
+
+func (EnumLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Enum && !dec.IsSet
+}
+
+func (EnumLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (EnumLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{ENUM: format.NewEnumType()}
+}
+
+func (EnumLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(EnumLogicalType)
+	return ok
+}
+
+// NewDecimalLogicalType returns a Decimal logical type with the given
+// precision and scale.
+//
+// Panics if precision < 1 or scale is not in the range (0, precision)
+func NewDecimalLogicalType(precision int32, scale int32) LogicalType {
+	if precision < 1 {
+		panic("parquet: precision must be greater than or equal to 1 for decimal logical type")
+	}
+	if scale < 0 || scale > precision {
+		panic("parquet: scale must be a non-negative integer that does not exceed precision for decimal logical type")
+	}
+	return &DecimalLogicalType{typ: &format.DecimalType{Precision: precision, Scale: scale}}
+}
+
+// DecimalLogicalType is used to represent a decimal value of a given
+// precision and scale
+type DecimalLogicalType struct {
+	baseLogicalType
+	typ *format.DecimalType
+}
+
+func (t DecimalLogicalType) Precision() int32 {
+	return t.typ.Precision
+}
+
+func (t DecimalLogicalType) Scale() int32 {
+	return t.typ.Scale
+}
+
+func (DecimalLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t DecimalLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{"Type": "Decimal", "precision": t.typ.Precision, "scale": t.typ.Scale})
+}
+
+func (t DecimalLogicalType) String() string {
+	return fmt.Sprintf("Decimal(precision=%d, scale=%d)", t.typ.Precision, t.typ.Scale)
+}
+
+func (t DecimalLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Decimal, DecimalMetadata{IsSet: true, Scale: t.typ.GetScale(), Precision: t.typ.GetPrecision()}
+}
+
+func (t DecimalLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.Decimal &&
+		dec.IsSet && dec.Scale == t.typ.Scale && dec.Precision == t.typ.Precision
+}
+
+func (t DecimalLogicalType) IsApplicable(typ parquet.Type, tlen int32) bool {
+	switch typ {
+	case parquet.Types.Int32:
+		return 1 <= t.typ.Precision && t.typ.Precision <= 9
+	case parquet.Types.Int64:
+		if t.typ.Precision < 10 {
+			debug.Log("int64 used for decimal logical, precision is small enough to use int32")
+		}
+		return 1 <= t.typ.Precision && t.typ.Precision <= 18
+	case parquet.Types.FixedLenByteArray:
+		return t.typ.Precision <= int32(math.Floor(math.Log10(math.Pow(2.0, (8.0*float64(tlen)-1.0)))))
+	case parquet.Types.ByteArray:
+		return true
+	}
+	return false
+}
+
+func (t DecimalLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DECIMAL: t.typ}
+}
+
+func (t DecimalLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*DecimalLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.Precision == other.typ.Precision && t.typ.Scale == other.typ.Scale
+}
+
+// DateLogicalType is an int32 representing the number of days since the Unix Epoch
+// 1 January 1970
+type DateLogicalType struct{ baseLogicalType }
+
+func (DateLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (DateLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": DateLogicalType{}.String()})
+}
+
+func (DateLogicalType) String() string {
+	return "Date"
+}
+
+func (DateLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Date, DecimalMetadata{}
+}
+
+func (DateLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Date && !dec.IsSet
+}
+
+func (DateLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int32
+}
+
+func (DateLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DATE: format.NewDateType()}
+}
+
+func (DateLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(DateLogicalType)
+	return ok
+}
+
+func timeUnitFromThrift(unit *format.TimeUnit) TimeUnitType {
+	switch {
+	case unit == nil:
+		return TimeUnitUnknown
+	case unit.IsSetMILLIS():
+		return TimeUnitMillis
+	case unit.IsSetMICROS():
+		return TimeUnitMicros
+	case unit.IsSetNANOS():
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func timeUnitToString(unit *format.TimeUnit) string {
+	switch {
+	case unit == nil:
+		return "unknown"
+	case unit.IsSetMILLIS():
+		return "milliseconds"
+	case unit.IsSetMICROS():
+		return "microseconds"
+	case unit.IsSetNANOS():
+		return "nanoseconds"
+	default:
+		return "unknown"
+	}
+}
+
+func timeUnitFromString(v string) TimeUnitType {
+	switch v {
+	case "millis":
+		return TimeUnitMillis
+	case "micros":
+		return TimeUnitMicros
+	case "nanos":
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func createTimeUnit(unit TimeUnitType) *format.TimeUnit {
+	tunit := format.NewTimeUnit()
+	switch unit {
+	case TimeUnitMicros:
+		tunit.MICROS = format.NewMicroSeconds()
+	case TimeUnitMillis:
+		tunit.MILLIS = format.NewMilliSeconds()
+	case TimeUnitNanos:
+		tunit.NANOS = format.NewNanoSeconds()
+	default:
+		panic("parquet: time unit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+	}
+	return tunit
+}
+
+// NewTimeLogicalType returns a time type of the given unit.
+func NewTimeLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimeLogicalType{typ: &format.TimeType{
+		IsAdjustedToUTC: isAdjustedToUTC,
+		Unit:            createTimeUnit(unit),
+	}}
+}
+
+// TimeLogicalType is a time type without a date and must be an
+// int32 for milliseconds, or an int64 for micro or nano seconds.
+type TimeLogicalType struct {
+	baseLogicalType
+	typ *format.TimeType
+}
+
+func (t TimeLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimeLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimeLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimeLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Time", "isAdjustedToUTC": t.typ.IsAdjustedToUTC, "timeUnit": timeUnitToString(t.typ.GetUnit())})
+}
+
+func (t TimeLogicalType) String() string {
+	return fmt.Sprintf("Time(isAdjustedToUTC=%t, timeUnit=%s)", t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()))
+}
+
+func (t TimeLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimeMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimeMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimeLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return c == ConvertedTypes.TimeMillis
+		case TimeUnitMicros:
+			return c == ConvertedTypes.TimeMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (t TimeLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetUnit().IsSetMILLIS()) ||
+		(typ == parquet.Types.Int64 &&
+			(t.typ.GetUnit().IsSetMICROS() || t.typ.GetUnit().IsSetNANOS()))
+}
+
+func (t TimeLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIME: t.typ}
+}
+
+func (t TimeLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimeLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewTimestampLogicalType returns a logical timestamp type with "forceConverted"
+// set to false
+func NewTimestampLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: false,
+		fromConverted:  false,
+	}
+}
+
+// NewTimestampLogicalTypeForce returns a timestamp logical type with
+// "forceConverted" set to true
+func NewTimestampLogicalTypeForce(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: true,
+		fromConverted:  false,
+	}
+}
+
+// TimestampLogicalType represents an int64 number that can be decoded
+// into a year, month, day, hour, minute, second, and subsecond
+type TimestampLogicalType struct {
+	baseLogicalType
+	typ            *format.TimestampType
+	forceConverted bool
+	fromConverted  bool
+}
+
+func (t TimestampLogicalType) IsFromConvertedType() bool {
+	return t.fromConverted
+}
+
+func (t TimestampLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimestampLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimestampLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimestampLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type":                     "Timestamp",
+		"isAdjustedToUTC":          t.typ.IsAdjustedToUTC,
+		"timeUnit":                 timeUnitToString(t.typ.GetUnit()),
+		"is_from_converted_type":   t.fromConverted,
+		"force_set_converted_type": t.forceConverted,
+	})
+}
+
+func (t TimestampLogicalType) IsSerialized() bool {
+	return !t.fromConverted
+}
+
+func (t TimestampLogicalType) String() string {
+	return fmt.Sprintf("Timestamp(isAdjustedToUTC=%t, timeUnit=%s, is_from_converted_type=%t, force_set_converted_type=%t)",
+		t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()), t.fromConverted, t.forceConverted)
+}
+
+func (t TimestampLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC || t.forceConverted {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimestampMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimestampMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimestampLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+
+	switch timeUnitFromThrift(t.typ.Unit) {
+	case TimeUnitMillis:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMillis
+		}
+	case TimeUnitMicros:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (TimestampLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int64
+}
+
+func (t TimestampLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIMESTAMP: t.typ}
+}
+
+func (t TimestampLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimestampLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewIntLogicalType creates an integer logical type of the desired bitwidth
+// and whether it is signed or not.
+//
+// Bit width must be exactly 8, 16, 32 or 64 for an integer logical type
+func NewIntLogicalType(bitWidth int8, signed bool) LogicalType {
+	switch bitWidth {
+	case 8, 16, 32, 64:

Review comment:
       correct, the default behavior of go in a switch is to break when it hits the next case statement. You have to explicitly use the `fallthrough` keyword if you want it to fallthrough to the next case.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-823400210


   Started reviewing, left some comments, will try to finish all files over the next few days.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619923697



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,110 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode *PrimitiveNode
+	// the maximum definition level in this column
+	maxDefLvl int16
+	// the maximum repetition level in this column

Review comment:
       updated

##########
File path: go/parquet/schema/converted_types.go
##########
@@ -0,0 +1,191 @@
+// 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.
+
+package schema
+
+import (
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// ConvertedType corresponds to the ConvertedType in the parquet.Thrift,
+// with added values of None and NA for handling when these values are not
+// set in the metadata
+type ConvertedType format.ConvertedType
+
+var (
+	// ConvertedTypes is a struct containing the constants for the types
+	// to make it easy to reference them while making it clear what they are
+	ConvertedTypes = struct {
+		None            ConvertedType
+		UTF8            ConvertedType
+		Map             ConvertedType
+		MapKeyValue     ConvertedType
+		List            ConvertedType
+		Enum            ConvertedType
+		Decimal         ConvertedType
+		Date            ConvertedType
+		TimeMillis      ConvertedType
+		TimeMicros      ConvertedType
+		TimestampMillis ConvertedType
+		TimestampMicros ConvertedType
+		Uint8           ConvertedType
+		Uint16          ConvertedType
+		Uint32          ConvertedType
+		Uint64          ConvertedType
+		Int8            ConvertedType
+		Int16           ConvertedType
+		Int32           ConvertedType
+		Int64           ConvertedType
+		JSON            ConvertedType
+		BSON            ConvertedType
+		Interval        ConvertedType
+		NA              ConvertedType
+	}{
+		None:            -1, // thrift enum starts at 0, so we know this will not be used
+		UTF8:            ConvertedType(format.ConvertedType_UTF8),
+		Map:             ConvertedType(format.ConvertedType_MAP),
+		MapKeyValue:     ConvertedType(format.ConvertedType_MAP_KEY_VALUE),
+		List:            ConvertedType(format.ConvertedType_LIST),
+		Enum:            ConvertedType(format.ConvertedType_ENUM),
+		Decimal:         ConvertedType(format.ConvertedType_DECIMAL),
+		Date:            ConvertedType(format.ConvertedType_DATE),
+		TimeMillis:      ConvertedType(format.ConvertedType_TIME_MILLIS),
+		TimeMicros:      ConvertedType(format.ConvertedType_TIME_MICROS),
+		TimestampMillis: ConvertedType(format.ConvertedType_TIMESTAMP_MILLIS),
+		TimestampMicros: ConvertedType(format.ConvertedType_TIMESTAMP_MICROS),
+		Uint8:           ConvertedType(format.ConvertedType_UINT_8),
+		Uint16:          ConvertedType(format.ConvertedType_UINT_16),
+		Uint32:          ConvertedType(format.ConvertedType_UINT_32),
+		Uint64:          ConvertedType(format.ConvertedType_UINT_64),
+		Int8:            ConvertedType(format.ConvertedType_INT_8),
+		Int16:           ConvertedType(format.ConvertedType_INT_16),
+		Int32:           ConvertedType(format.ConvertedType_INT_32),
+		Int64:           ConvertedType(format.ConvertedType_INT_64),
+		JSON:            ConvertedType(format.ConvertedType_JSON),
+		BSON:            ConvertedType(format.ConvertedType_BSON),
+		Interval:        ConvertedType(format.ConvertedType_INTERVAL),
+		NA:              24, // should always be the last values after Interval
+	}
+)
+
+func (p ConvertedType) String() string {
+	switch p {
+	case ConvertedTypes.None:
+		return "NONE"
+	case ConvertedTypes.NA:
+		return "UNKNOWN"
+	default:
+		return format.ConvertedType(p).String()
+	}
+}
+
+// ToLogicalType returns the correct LogicalType for the given ConvertedType, using the decimal
+// metadata provided to define the precision/scale if necessary
+func (p ConvertedType) ToLogicalType(convertedDecimal DecimalMetadata) LogicalType {
+	switch p {
+	case ConvertedTypes.UTF8:
+		return StringLogicalType{}
+	case ConvertedTypes.Map, ConvertedTypes.MapKeyValue:
+		return MapLogicalType{}
+	case ConvertedTypes.List:
+		return ListLogicalType{}
+	case ConvertedTypes.Enum:
+		return EnumLogicalType{}
+	case ConvertedTypes.Decimal:
+		return NewDecimalLogicalType(convertedDecimal.Precision, convertedDecimal.Scale)
+	case ConvertedTypes.Date:
+		return DateLogicalType{}
+	case ConvertedTypes.TimeMillis:
+		return NewTimeLogicalType(true, TimeUnitMillis)

Review comment:
       done

##########
File path: go/parquet/schema/converted_types.go
##########
@@ -0,0 +1,191 @@
+// 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.
+
+package schema
+
+import (
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// ConvertedType corresponds to the ConvertedType in the parquet.Thrift,
+// with added values of None and NA for handling when these values are not
+// set in the metadata
+type ConvertedType format.ConvertedType
+
+var (
+	// ConvertedTypes is a struct containing the constants for the types
+	// to make it easy to reference them while making it clear what they are
+	ConvertedTypes = struct {
+		None            ConvertedType
+		UTF8            ConvertedType
+		Map             ConvertedType
+		MapKeyValue     ConvertedType
+		List            ConvertedType
+		Enum            ConvertedType
+		Decimal         ConvertedType
+		Date            ConvertedType
+		TimeMillis      ConvertedType
+		TimeMicros      ConvertedType
+		TimestampMillis ConvertedType
+		TimestampMicros ConvertedType
+		Uint8           ConvertedType
+		Uint16          ConvertedType
+		Uint32          ConvertedType
+		Uint64          ConvertedType
+		Int8            ConvertedType
+		Int16           ConvertedType
+		Int32           ConvertedType
+		Int64           ConvertedType
+		JSON            ConvertedType
+		BSON            ConvertedType
+		Interval        ConvertedType
+		NA              ConvertedType
+	}{
+		None:            -1, // thrift enum starts at 0, so we know this will not be used
+		UTF8:            ConvertedType(format.ConvertedType_UTF8),
+		Map:             ConvertedType(format.ConvertedType_MAP),
+		MapKeyValue:     ConvertedType(format.ConvertedType_MAP_KEY_VALUE),
+		List:            ConvertedType(format.ConvertedType_LIST),
+		Enum:            ConvertedType(format.ConvertedType_ENUM),
+		Decimal:         ConvertedType(format.ConvertedType_DECIMAL),
+		Date:            ConvertedType(format.ConvertedType_DATE),
+		TimeMillis:      ConvertedType(format.ConvertedType_TIME_MILLIS),
+		TimeMicros:      ConvertedType(format.ConvertedType_TIME_MICROS),
+		TimestampMillis: ConvertedType(format.ConvertedType_TIMESTAMP_MILLIS),
+		TimestampMicros: ConvertedType(format.ConvertedType_TIMESTAMP_MICROS),
+		Uint8:           ConvertedType(format.ConvertedType_UINT_8),
+		Uint16:          ConvertedType(format.ConvertedType_UINT_16),
+		Uint32:          ConvertedType(format.ConvertedType_UINT_32),
+		Uint64:          ConvertedType(format.ConvertedType_UINT_64),
+		Int8:            ConvertedType(format.ConvertedType_INT_8),
+		Int16:           ConvertedType(format.ConvertedType_INT_16),
+		Int32:           ConvertedType(format.ConvertedType_INT_32),
+		Int64:           ConvertedType(format.ConvertedType_INT_64),
+		JSON:            ConvertedType(format.ConvertedType_JSON),
+		BSON:            ConvertedType(format.ConvertedType_BSON),
+		Interval:        ConvertedType(format.ConvertedType_INTERVAL),
+		NA:              24, // should always be the last values after Interval
+	}
+)
+
+func (p ConvertedType) String() string {
+	switch p {
+	case ConvertedTypes.None:
+		return "NONE"
+	case ConvertedTypes.NA:
+		return "UNKNOWN"
+	default:
+		return format.ConvertedType(p).String()
+	}
+}
+
+// ToLogicalType returns the correct LogicalType for the given ConvertedType, using the decimal
+// metadata provided to define the precision/scale if necessary
+func (p ConvertedType) ToLogicalType(convertedDecimal DecimalMetadata) LogicalType {
+	switch p {
+	case ConvertedTypes.UTF8:
+		return StringLogicalType{}
+	case ConvertedTypes.Map, ConvertedTypes.MapKeyValue:
+		return MapLogicalType{}
+	case ConvertedTypes.List:
+		return ListLogicalType{}
+	case ConvertedTypes.Enum:
+		return EnumLogicalType{}
+	case ConvertedTypes.Decimal:
+		return NewDecimalLogicalType(convertedDecimal.Precision, convertedDecimal.Scale)
+	case ConvertedTypes.Date:
+		return DateLogicalType{}
+	case ConvertedTypes.TimeMillis:
+		return NewTimeLogicalType(true, TimeUnitMillis)
+	case ConvertedTypes.TimeMicros:
+		return NewTimeLogicalType(true, TimeUnitMicros)
+	case ConvertedTypes.TimestampMillis:
+		t := NewTimestampLogicalType(true, TimeUnitMillis)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.TimestampMicros:
+		t := NewTimestampLogicalType(true, TimeUnitMicros)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.Interval:
+		return IntervalLogicalType{}
+	case ConvertedTypes.Int8:
+		return NewIntLogicalType(8, true)

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r634397663



##########
File path: go/parquet/schema/reflection.go
##########
@@ -0,0 +1,827 @@
+// 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.
+
+package schema
+
+import (
+	"reflect"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+type taggedInfo struct {
+	Name string
+
+	Type      parquet.Type
+	KeyType   parquet.Type
+	ValueType parquet.Type
+
+	Length      int32
+	KeyLength   int32
+	ValueLength int32
+
+	Scale      int32
+	KeyScale   int32
+	ValueScale int32
+
+	Precision      int32
+	KeyPrecision   int32
+	ValuePrecision int32
+
+	FieldID      int32
+	KeyFieldID   int32
+	ValueFieldID int32
+
+	RepetitionType  parquet.Repetition
+	ValueRepetition parquet.Repetition
+
+	Converted      ConvertedType
+	KeyConverted   ConvertedType
+	ValueConverted ConvertedType
+
+	LogicalFields      map[string]string
+	KeyLogicalFields   map[string]string
+	ValueLogicalFields map[string]string
+
+	LogicalType      LogicalType
+	KeyLogicalType   LogicalType
+	ValueLogicalType LogicalType
+}
+
+func (t *taggedInfo) CopyForKey() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.KeyType
+	ret.Length = t.KeyLength
+	ret.Scale = t.KeyScale
+	ret.Precision = t.KeyPrecision
+	ret.FieldID = t.KeyFieldID
+	ret.RepetitionType = parquet.Repetitions.Required
+	ret.Converted = t.KeyConverted
+	ret.LogicalType = t.KeyLogicalType
+	return
+}
+
+func (t *taggedInfo) CopyForValue() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.ValueType
+	ret.Length = t.ValueLength
+	ret.Scale = t.ValueScale
+	ret.Precision = t.ValuePrecision
+	ret.FieldID = t.ValueFieldID
+	ret.RepetitionType = t.ValueRepetition
+	ret.Converted = t.ValueConverted
+	ret.LogicalType = t.ValueLogicalType
+	return
+}
+
+func (t *taggedInfo) UpdateLogicalTypes() {
+	processLogicalType := func(fields map[string]string, precision, scale int32) LogicalType {
+		t, ok := fields["type"]
+		if !ok {
+			return NoLogicalType{}
+		}
+
+		switch strings.ToLower(t) {
+		case "string":
+			return StringLogicalType{}
+		case "map":
+			return MapLogicalType{}
+		case "list":
+			return ListLogicalType{}
+		case "enum":
+			return EnumLogicalType{}
+		case "decimal":
+			if v, ok := fields["precision"]; ok {
+				precision = int32FromType(v)
+			}
+			if v, ok := fields["scale"]; ok {
+				scale = int32FromType(v)
+			}
+			return NewDecimalLogicalType(precision, scale)
+		case "date":
+			return DateLogicalType{}
+		case "time":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimeLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(strings.ToLower(unit)))
+		case "timestamp":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimestampLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(unit))
+		case "integer":
+			width, ok := fields["bitwidth"]
+			if !ok {
+				panic("must specify bitwidth if explicitly setting integer logical type")
+			}
+			signed, ok := fields["signed"]
+			if !ok {
+				signed = "true"
+			}
+
+			return NewIntLogicalType(int8(int32FromType(width)), boolFromStr(signed))
+		case "null":
+			return NullLogicalType{}
+		case "json":
+			return JSONLogicalType{}
+		case "bson":
+			return BSONLogicalType{}
+		case "uuid":
+			return UUIDLogicalType{}
+		default:
+			panic(xerrors.Errorf("invalid logical type specified: %s", t))
+		}
+	}
+
+	t.LogicalType = processLogicalType(t.LogicalFields, t.Precision, t.Scale)
+	t.KeyLogicalType = processLogicalType(t.KeyLogicalFields, t.KeyPrecision, t.KeyScale)
+	t.ValueLogicalType = processLogicalType(t.ValueLogicalFields, t.ValuePrecision, t.ValueScale)
+}
+
+func newTaggedInfo() taggedInfo {
+	return taggedInfo{
+		Type:               parquet.Types.Undefined,
+		KeyType:            parquet.Types.Undefined,
+		ValueType:          parquet.Types.Undefined,
+		RepetitionType:     parquet.Repetitions.Undefined,
+		ValueRepetition:    parquet.Repetitions.Undefined,
+		Converted:          ConvertedTypes.NA,
+		KeyConverted:       ConvertedTypes.NA,
+		ValueConverted:     ConvertedTypes.NA,
+		FieldID:            -1,
+		KeyFieldID:         -1,
+		ValueFieldID:       -1,
+		LogicalFields:      make(map[string]string),
+		KeyLogicalFields:   make(map[string]string),
+		ValueLogicalFields: make(map[string]string),
+		LogicalType:        NoLogicalType{},
+		KeyLogicalType:     NoLogicalType{},
+		ValueLogicalType:   NoLogicalType{},
+	}
+}
+
+var int32FromType = func(v string) int32 {
+	val, err := strconv.Atoi(v)
+	if err != nil {
+		panic(err)
+	}
+	return int32(val)
+}
+
+var boolFromStr = func(v string) bool {
+	val, err := strconv.ParseBool(v)
+	if err != nil {
+		panic(err)
+	}
+	return val
+}
+
+func infoFromTags(f reflect.StructTag) *taggedInfo {
+	typeFromStr := func(v string) parquet.Type {
+		t, err := format.TypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(xerrors.Errorf("invalid type specified: %s", v))
+		}
+		return parquet.Type(t)
+	}
+
+	repFromStr := func(v string) parquet.Repetition {
+		r, err := format.FieldRepetitionTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return parquet.Repetition(r)
+	}
+
+	convertedFromStr := func(v string) ConvertedType {
+		c, err := format.ConvertedTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return ConvertedType(c)
+	}
+
+	if ptags, ok := f.Lookup("parquet"); ok {
+		info := newTaggedInfo()
+		for _, tag := range strings.Split(strings.Replace(ptags, "\t", "", -1), ",") {
+			tag = strings.TrimSpace(tag)
+			kv := strings.SplitN(tag, "=", 2)
+			key := strings.TrimSpace(strings.ToLower(kv[0]))
+			value := strings.TrimSpace(kv[1])
+
+			switch key {
+			case "name":
+				info.Name = value
+			case "type":
+				info.Type = typeFromStr(value)
+			case "keytype":
+				info.KeyType = typeFromStr(value)
+			case "valuetype":
+				info.ValueType = typeFromStr(value)
+			case "length":
+				info.Length = int32FromType(value)
+			case "keylength":
+				info.KeyLength = int32FromType(value)
+			case "valuelength":
+				info.ValueLength = int32FromType(value)
+			case "scale":
+				info.Scale = int32FromType(value)
+			case "keyscale":
+				info.KeyScale = int32FromType(value)
+			case "valuescale":
+				info.ValueScale = int32FromType(value)
+			case "precision":
+				info.Precision = int32FromType(value)
+			case "keyprecision":
+				info.KeyPrecision = int32FromType(value)
+			case "valueprecision":
+				info.ValuePrecision = int32FromType(value)
+			case "fieldid":
+				info.FieldID = int32FromType(value)
+			case "keyfieldid":
+				info.KeyFieldID = int32FromType(value)
+			case "valuefieldid":
+				info.ValueFieldID = int32FromType(value)
+			case "repetition":
+				info.RepetitionType = repFromStr(value)
+			case "valuerepetition":
+				info.ValueRepetition = repFromStr(value)
+			case "converted":
+				info.Converted = convertedFromStr(value)
+			case "keyconverted":
+				info.KeyConverted = convertedFromStr(value)
+			case "valueconverted":
+				info.ValueConverted = convertedFromStr(value)
+			case "logical":
+				info.LogicalFields["type"] = value
+			case "keylogical":
+				info.KeyLogicalFields["type"] = value
+			case "valuelogical":
+				info.ValueLogicalFields["type"] = value
+			default:
+				switch {
+				case strings.HasPrefix(key, "logical."):
+					info.LogicalFields[strings.TrimPrefix(key, "logical.")] = value
+				case strings.HasPrefix(key, "keylogical."):
+					info.KeyLogicalFields[strings.TrimPrefix(key, "keylogical.")] = value
+				case strings.HasPrefix(key, "valuelogical."):
+					info.ValueLogicalFields[strings.TrimPrefix(key, "valuelogical.")] = value
+				}
+			}
+		}
+		info.UpdateLogicalTypes()
+		return &info
+	}
+	return nil
+}
+
+// typeToNode recurseively converts a physical type and the tag info into parquet Nodes
+//
+// to avoid having to propagate errors up potentially high numbers of recursive calls
+// we use panics and then recover in the public function NewSchemaFromStruct so that a
+// failure very far down the stack quickly unwinds.
+func typeToNode(name string, typ reflect.Type, repType parquet.Repetition, info *taggedInfo) Node {
+	// set up our default values for everything
+	var (
+		converted             = ConvertedTypes.None
+		logical   LogicalType = NoLogicalType{}
+		fieldID               = int32(-1)
+		physical              = parquet.Types.Undefined
+		typeLen               = 0
+		precision             = 0
+		scale                 = 0
+	)
+	if info != nil { // we have struct tag info to process
+		fieldID = info.FieldID

Review comment:
       Fair enough. I'm fine with pulling out the automatic field id generation here. The current implementation i have here would persist user set ids and only auto generates if they are left as -1, but it's easy to just leave the -1's or pass through what is given. I'll comment after I push that change.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r619902201



##########
File path: go/parquet/schema/converted_types.go
##########
@@ -0,0 +1,191 @@
+// 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.
+
+package schema
+
+import (
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// ConvertedType corresponds to the ConvertedType in the parquet.Thrift,
+// with added values of None and NA for handling when these values are not
+// set in the metadata
+type ConvertedType format.ConvertedType
+
+var (
+	// ConvertedTypes is a struct containing the constants for the types
+	// to make it easy to reference them while making it clear what they are
+	ConvertedTypes = struct {
+		None            ConvertedType
+		UTF8            ConvertedType
+		Map             ConvertedType
+		MapKeyValue     ConvertedType
+		List            ConvertedType
+		Enum            ConvertedType
+		Decimal         ConvertedType
+		Date            ConvertedType
+		TimeMillis      ConvertedType
+		TimeMicros      ConvertedType
+		TimestampMillis ConvertedType
+		TimestampMicros ConvertedType
+		Uint8           ConvertedType
+		Uint16          ConvertedType
+		Uint32          ConvertedType
+		Uint64          ConvertedType
+		Int8            ConvertedType
+		Int16           ConvertedType
+		Int32           ConvertedType
+		Int64           ConvertedType
+		JSON            ConvertedType
+		BSON            ConvertedType
+		Interval        ConvertedType
+		NA              ConvertedType
+	}{
+		None:            -1, // thrift enum starts at 0, so we know this will not be used
+		UTF8:            ConvertedType(format.ConvertedType_UTF8),
+		Map:             ConvertedType(format.ConvertedType_MAP),
+		MapKeyValue:     ConvertedType(format.ConvertedType_MAP_KEY_VALUE),
+		List:            ConvertedType(format.ConvertedType_LIST),
+		Enum:            ConvertedType(format.ConvertedType_ENUM),
+		Decimal:         ConvertedType(format.ConvertedType_DECIMAL),
+		Date:            ConvertedType(format.ConvertedType_DATE),
+		TimeMillis:      ConvertedType(format.ConvertedType_TIME_MILLIS),
+		TimeMicros:      ConvertedType(format.ConvertedType_TIME_MICROS),
+		TimestampMillis: ConvertedType(format.ConvertedType_TIMESTAMP_MILLIS),
+		TimestampMicros: ConvertedType(format.ConvertedType_TIMESTAMP_MICROS),
+		Uint8:           ConvertedType(format.ConvertedType_UINT_8),
+		Uint16:          ConvertedType(format.ConvertedType_UINT_16),
+		Uint32:          ConvertedType(format.ConvertedType_UINT_32),
+		Uint64:          ConvertedType(format.ConvertedType_UINT_64),
+		Int8:            ConvertedType(format.ConvertedType_INT_8),
+		Int16:           ConvertedType(format.ConvertedType_INT_16),
+		Int32:           ConvertedType(format.ConvertedType_INT_32),
+		Int64:           ConvertedType(format.ConvertedType_INT_64),
+		JSON:            ConvertedType(format.ConvertedType_JSON),
+		BSON:            ConvertedType(format.ConvertedType_BSON),
+		Interval:        ConvertedType(format.ConvertedType_INTERVAL),
+		NA:              24, // should always be the last values after Interval
+	}
+)
+
+func (p ConvertedType) String() string {
+	switch p {
+	case ConvertedTypes.None:
+		return "NONE"
+	case ConvertedTypes.NA:
+		return "UNKNOWN"
+	default:
+		return format.ConvertedType(p).String()
+	}
+}
+
+// ToLogicalType returns the correct LogicalType for the given ConvertedType, using the decimal
+// metadata provided to define the precision/scale if necessary
+func (p ConvertedType) ToLogicalType(convertedDecimal DecimalMetadata) LogicalType {
+	switch p {
+	case ConvertedTypes.UTF8:
+		return StringLogicalType{}
+	case ConvertedTypes.Map, ConvertedTypes.MapKeyValue:
+		return MapLogicalType{}
+	case ConvertedTypes.List:
+		return ListLogicalType{}
+	case ConvertedTypes.Enum:
+		return EnumLogicalType{}
+	case ConvertedTypes.Decimal:
+		return NewDecimalLogicalType(convertedDecimal.Precision, convertedDecimal.Scale)
+	case ConvertedTypes.Date:
+		return DateLogicalType{}
+	case ConvertedTypes.TimeMillis:
+		return NewTimeLogicalType(true, TimeUnitMillis)
+	case ConvertedTypes.TimeMicros:
+		return NewTimeLogicalType(true, TimeUnitMicros)
+	case ConvertedTypes.TimestampMillis:
+		t := NewTimestampLogicalType(true, TimeUnitMillis)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.TimestampMicros:
+		t := NewTimestampLogicalType(true, TimeUnitMicros)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.Interval:
+		return IntervalLogicalType{}
+	case ConvertedTypes.Int8:
+		return NewIntLogicalType(8, true)
+	case ConvertedTypes.Int16:
+		return NewIntLogicalType(16, true)
+	case ConvertedTypes.Int32:
+		return NewIntLogicalType(32, true)
+	case ConvertedTypes.Int64:
+		return NewIntLogicalType(64, true)
+	case ConvertedTypes.Uint8:
+		return NewIntLogicalType(8, false)
+	case ConvertedTypes.Uint16:
+		return NewIntLogicalType(16, false)
+	case ConvertedTypes.Uint32:
+		return NewIntLogicalType(32, false)
+	case ConvertedTypes.Uint64:
+		return NewIntLogicalType(64, false)
+	case ConvertedTypes.JSON:
+		return JSONLogicalType{}
+	case ConvertedTypes.BSON:
+		return BSONLogicalType{}
+	case ConvertedTypes.None:
+		return NoLogicalType{}
+	case ConvertedTypes.NA:
+		fallthrough
+	default:
+		return UnknownLogicalType{}
+	}
+}
+
+// GetSortOrder defaults to the sort order based on the physical type if convert
+// is ConvertedTypes.None, otherwise determines the sort order by the converted type.
+func GetSortOrder(convert ConvertedType, primitive format.Type) SortOrder {
+	if convert == ConvertedTypes.None {
+		return DefaultSortOrder(primitive)
+	}
+	switch convert {
+	case ConvertedTypes.Int8,
+		ConvertedTypes.Int16,
+		ConvertedTypes.Int32,
+		ConvertedTypes.Int64,
+		ConvertedTypes.Date,
+		ConvertedTypes.TimeMicros,
+		ConvertedTypes.TimeMillis,
+		ConvertedTypes.TimestampMicros,
+		ConvertedTypes.TimestampMillis:
+		return SortSIGNED
+	case ConvertedTypes.Uint8,
+		ConvertedTypes.Uint16,
+		ConvertedTypes.Uint32,
+		ConvertedTypes.Uint64,
+		ConvertedTypes.Enum,
+		ConvertedTypes.UTF8,
+		ConvertedTypes.BSON,
+		ConvertedTypes.JSON:
+		return SortUNSIGNED
+	case ConvertedTypes.Decimal,

Review comment:
       so i definitely copied this from the C++ implementation, i believe it's "sortUNKNOWN" because it can differ based on the underlying type, that said it means my current implementation of statistic sorting for decimals is incorrect now that i read the spec :) so i'll fix that in the rest of my code so it's correct before I put that part of the chunks up haha. 
   
   That said, given that int32 and int64 as the underling types for the decimal are signed comparison, i think i'll shift this to being marked as SortSIGNED since the spec defines decimal as sort by signed comparison.

##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval

Review comment:
       https://github.com/apache/arrow/blob/master/cpp/src/parquet/parquet.thrift#L340
   
   The current parquet.thrift file has the logical type for INTERVAL commented out so the generated code doesn't contain a LogicalType for Interval and thus it's unable to be serialized. This also comes from the C++ implementation too https://github.com/apache/arrow/blob/master/cpp/src/parquet/schema.cc#L520

##########
File path: go/parquet/schema/reflection_test.go
##########
@@ -0,0 +1,397 @@
+// 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.
+
+package schema_test
+
+import (
+	"log"
+	"os"
+	"reflect"
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/schema"
+	"github.com/stretchr/testify/assert"
+)
+
+func ExampleNewSchemaFromStruct_primitives() {
+	type Schema struct {
+		Bool              bool
+		Int8              int8
+		Uint16            uint16
+		Int32             int32
+		Int64             int64
+		Int96             parquet.Int96
+		Float             float32
+		Double            float64
+		ByteArray         string
+		FixedLenByteArray [10]byte
+	}
+
+	sc, err := schema.NewSchemaFromStruct(Schema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 Schema {
+	//   required boolean field_id=-1 Bool;
+	//   required int32 field_id=-1 Int8 (Int(bitWidth=8, isSigned=true));
+	//   required int32 field_id=-1 Uint16 (Int(bitWidth=16, isSigned=false));
+	//   required int32 field_id=-1 Int32 (Int(bitWidth=32, isSigned=true));
+	//   required int64 field_id=-1 Int64 (Int(bitWidth=64, isSigned=true));
+	//   required int96 field_id=-1 Int96;
+	//   required float field_id=-1 Float;
+	//   required double field_id=-1 Double;
+	//   required byte_array field_id=-1 ByteArray;
+	//   required fixed_len_byte_array field_id=-1 FixedLenByteArray;
+	// }
+}
+
+func ExampleNewSchemaFromStruct_convertedtypes() {
+	type ConvertedSchema struct {
+		Utf8           string        `parquet:"name=utf8, converted=UTF8"`
+		Uint32         uint32        `parquet:"converted=INT_32"`
+		Date           int32         `parquet:"name=date, converted=date"`
+		TimeMilli      int32         `parquet:"name=timemilli, converted=TIME_MILLIS"`
+		TimeMicro      int64         `parquet:"name=timemicro, converted=time_micros"`
+		TimeStampMilli int64         `parquet:"converted=timestamp_millis"`
+		TimeStampMicro int64         `parquet:"converted=timestamp_micros"`
+		Interval       parquet.Int96 `parquet:"converted=INTERVAL"`
+		Decimal1       int32         `parquet:"converted=decimal, scale=2, precision=9"`
+		Decimal2       int64         `parquet:"converted=decimal, scale=2, precision=18"`
+		Decimal3       [12]byte      `parquet:"converted=decimal, scale=2, precision=10"`
+		Decimal4       string        `parquet:"converted=decimal, scale=2, precision=20"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(&ConvertedSchema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 ConvertedSchema {
+	//   required byte_array field_id=-1 utf8 (String);
+	//   required int32 field_id=-1 Uint32 (Int(bitWidth=32, isSigned=true));
+	//   required int32 field_id=-1 date (Date);
+	//   required int32 field_id=-1 timemilli (Time(isAdjustedToUTC=true, timeUnit=milliseconds));
+	//   required int64 field_id=-1 timemicro (Time(isAdjustedToUTC=true, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeStampMilli (Timestamp(isAdjustedToUTC=true, timeUnit=milliseconds, is_from_converted_type=true, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimeStampMicro (Timestamp(isAdjustedToUTC=true, timeUnit=microseconds, is_from_converted_type=true, force_set_converted_type=false));
+	//   required int96 field_id=-1 Interval;
+	//   required int32 field_id=-1 Decimal1 (Decimal(precision=9, scale=2));
+	//   required int64 field_id=-1 Decimal2 (Decimal(precision=18, scale=2));
+	//   required fixed_len_byte_array field_id=-1 Decimal3 (Decimal(precision=10, scale=2));
+	//   required byte_array field_id=-1 Decimal4 (Decimal(precision=20, scale=2));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_repetition() {
+	type RepetitionSchema struct {
+		List     []int64 `parquet:"fieldid=1"`
+		Repeated []int64 `parquet:"repetition=repeated, fieldid=2"`
+		Optional *int64  `parquet:"fieldid=3"`
+		Required *int64  `parquet:"repetition=REQUIRED, fieldid=4"`
+		Opt      int64   `parquet:"repetition=OPTIONAL, fieldid=5"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(RepetitionSchema{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 RepetitionSchema {
+	//   required group field_id=1 List (List) {
+	//     repeated group field_id=-1 list {
+	//       required int64 field_id=-1 element (Int(bitWidth=64, isSigned=true));
+	//     }
+	//   }
+	//   repeated int64 field_id=2 Repeated (Int(bitWidth=64, isSigned=true));
+	//   optional int64 field_id=3 Optional (Int(bitWidth=64, isSigned=true));
+	//   required int64 field_id=4 Required (Int(bitWidth=64, isSigned=true));
+	//   optional int64 field_id=5 Opt (Int(bitWidth=64, isSigned=true));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_logicaltypes() {
+	type LogicalTypes struct {
+		String                []byte   `parquet:"logical=String"`
+		Enum                  string   `parquet:"logical=enum"`
+		Date                  int32    `parquet:"logical=date"`
+		Decimal1              int32    `parquet:"logical=decimal, precision=9, scale=2"`
+		Decimal2              int32    `parquet:"logical=decimal, logical.precision=9, scale=2"`
+		Decimal3              int32    `parquet:"logical=decimal, precision=5, logical.precision=9, scale=1, logical.scale=3"`
+		TimeMilliUTC          int32    `parquet:"logical=TIME, logical.unit=millis"`
+		TimeMilli             int32    `parquet:"logical=Time, logical.unit=millis, logical.isadjustedutc=false"`
+		TimeMicros            int64    `parquet:"logical=time, logical.unit=micros, logical.isadjustedutc=false"`
+		TimeMicrosUTC         int64    `parquet:"logical=time, logical.unit=micros, logical.isadjustedutc=true"`
+		TimeNanos             int64    `parquet:"logical=time, logical.unit=nanos"`
+		TimestampMilli        int64    `parquet:"logical=timestamp, logical.unit=millis"`
+		TimestampMicrosNotUTC int64    `parquet:"logical=timestamp, logical.unit=micros, logical.isadjustedutc=false"`
+		TimestampNanos        int64    `parquet:"logical=timestamp, logical.unit=nanos"`
+		JSON                  string   `parquet:"logical=json"`
+		BSON                  []byte   `parquet:"logical=BSON"`
+		UUID                  [16]byte `parquet:"logical=uuid"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(LogicalTypes{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 LogicalTypes {
+	//   required byte_array field_id=-1 String (String);
+	//   required byte_array field_id=-1 Enum (Enum);
+	//   required int32 field_id=-1 Date (Date);
+	//   required int32 field_id=-1 Decimal1 (Decimal(precision=9, scale=2));
+	//   required int32 field_id=-1 Decimal2 (Decimal(precision=9, scale=2));
+	//   required int32 field_id=-1 Decimal3 (Decimal(precision=9, scale=3));
+	//   required int32 field_id=-1 TimeMilliUTC (Time(isAdjustedToUTC=true, timeUnit=milliseconds));
+	//   required int32 field_id=-1 TimeMilli (Time(isAdjustedToUTC=false, timeUnit=milliseconds));
+	//   required int64 field_id=-1 TimeMicros (Time(isAdjustedToUTC=false, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeMicrosUTC (Time(isAdjustedToUTC=true, timeUnit=microseconds));
+	//   required int64 field_id=-1 TimeNanos (Time(isAdjustedToUTC=true, timeUnit=nanoseconds));
+	//   required int64 field_id=-1 TimestampMilli (Timestamp(isAdjustedToUTC=true, timeUnit=milliseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimestampMicrosNotUTC (Timestamp(isAdjustedToUTC=false, timeUnit=microseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required int64 field_id=-1 TimestampNanos (Timestamp(isAdjustedToUTC=true, timeUnit=nanoseconds, is_from_converted_type=false, force_set_converted_type=false));
+	//   required byte_array field_id=-1 JSON (JSON);
+	//   required byte_array field_id=-1 BSON (BSON);
+	//   required fixed_len_byte_array field_id=-1 UUID (UUID);
+	// }
+}
+
+func ExampleNewSchemaFromStruct_physicaltype() {
+	type ChangeTypes struct {
+		Int32        int64  `parquet:"type=int32"`
+		FixedLen     string `parquet:"type=fixed_len_byte_array, length=10"`
+		SliceAsFixed []byte `parquet:"type=fixed_len_byte_array, length=12"`
+		Int          int    `parquet:"type=int32"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(ChangeTypes{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 ChangeTypes {
+	//   required int32 field_id=-1 Int32 (Int(bitWidth=32, isSigned=true));
+	//   required fixed_len_byte_array field_id=-1 FixedLen;
+	//   required fixed_len_byte_array field_id=-1 SliceAsFixed;
+	//   required int32 field_id=-1 Int (Int(bitWidth=32, isSigned=true));
+	// }
+}
+
+func ExampleNewSchemaFromStruct_nestedtypes() {
+	type Other struct {
+		OptionalMap *map[string]*string `parquet:"valuerepetition=required, keylogical=String, valueconverted=BSON"`
+	}
+
+	type MyMap map[int32]string
+
+	type Nested struct {
+		SimpleMap     map[int32]string
+		FixedLenMap   map[string][]byte `parquet:"keytype=fixed_len_byte_array, keyfieldid=10, valuefieldid=11, keylength=10"`
+		DecimalMap    map[int32]string  `parquet:"logical=map, keyconverted=DECIMAL, keyscale=3, keyprecision=7, valuetype=fixed_len_byte_array, valuelength=4, valuelogical=decimal, valuelogical.precision=9, valuescale=2"`
+		OtherList     []*Other
+		OtherRepeated []Other  `parquet:"repetition=repeated"`
+		DateArray     [5]int32 `parquet:"valuelogical=date, logical=list"`
+		DateMap       MyMap    `parquet:"keylogical=TIME, keylogical.unit=MILLIS, keylogical.isadjustedutc=false, valuelogical=enum"`
+	}
+
+	sc, err := schema.NewSchemaFromStruct(Nested{})
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	schema.PrintSchema(sc.Root(), os.Stdout, 2)
+
+	// Output:
+	// repeated group field_id=-1 Nested {

Review comment:
       I'm not sure what you mean, what assertion are you referring to?

##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16

Review comment:
       comment added. 

##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,110 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode *PrimitiveNode
+	// the maximum definition level in this column
+	maxDefLvl int16
+	// the maximum repetition level in this column

Review comment:
       updated

##########
File path: go/parquet/schema/converted_types.go
##########
@@ -0,0 +1,191 @@
+// 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.
+
+package schema
+
+import (
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// ConvertedType corresponds to the ConvertedType in the parquet.Thrift,
+// with added values of None and NA for handling when these values are not
+// set in the metadata
+type ConvertedType format.ConvertedType
+
+var (
+	// ConvertedTypes is a struct containing the constants for the types
+	// to make it easy to reference them while making it clear what they are
+	ConvertedTypes = struct {
+		None            ConvertedType
+		UTF8            ConvertedType
+		Map             ConvertedType
+		MapKeyValue     ConvertedType
+		List            ConvertedType
+		Enum            ConvertedType
+		Decimal         ConvertedType
+		Date            ConvertedType
+		TimeMillis      ConvertedType
+		TimeMicros      ConvertedType
+		TimestampMillis ConvertedType
+		TimestampMicros ConvertedType
+		Uint8           ConvertedType
+		Uint16          ConvertedType
+		Uint32          ConvertedType
+		Uint64          ConvertedType
+		Int8            ConvertedType
+		Int16           ConvertedType
+		Int32           ConvertedType
+		Int64           ConvertedType
+		JSON            ConvertedType
+		BSON            ConvertedType
+		Interval        ConvertedType
+		NA              ConvertedType
+	}{
+		None:            -1, // thrift enum starts at 0, so we know this will not be used
+		UTF8:            ConvertedType(format.ConvertedType_UTF8),
+		Map:             ConvertedType(format.ConvertedType_MAP),
+		MapKeyValue:     ConvertedType(format.ConvertedType_MAP_KEY_VALUE),
+		List:            ConvertedType(format.ConvertedType_LIST),
+		Enum:            ConvertedType(format.ConvertedType_ENUM),
+		Decimal:         ConvertedType(format.ConvertedType_DECIMAL),
+		Date:            ConvertedType(format.ConvertedType_DATE),
+		TimeMillis:      ConvertedType(format.ConvertedType_TIME_MILLIS),
+		TimeMicros:      ConvertedType(format.ConvertedType_TIME_MICROS),
+		TimestampMillis: ConvertedType(format.ConvertedType_TIMESTAMP_MILLIS),
+		TimestampMicros: ConvertedType(format.ConvertedType_TIMESTAMP_MICROS),
+		Uint8:           ConvertedType(format.ConvertedType_UINT_8),
+		Uint16:          ConvertedType(format.ConvertedType_UINT_16),
+		Uint32:          ConvertedType(format.ConvertedType_UINT_32),
+		Uint64:          ConvertedType(format.ConvertedType_UINT_64),
+		Int8:            ConvertedType(format.ConvertedType_INT_8),
+		Int16:           ConvertedType(format.ConvertedType_INT_16),
+		Int32:           ConvertedType(format.ConvertedType_INT_32),
+		Int64:           ConvertedType(format.ConvertedType_INT_64),
+		JSON:            ConvertedType(format.ConvertedType_JSON),
+		BSON:            ConvertedType(format.ConvertedType_BSON),
+		Interval:        ConvertedType(format.ConvertedType_INTERVAL),
+		NA:              24, // should always be the last values after Interval
+	}
+)
+
+func (p ConvertedType) String() string {
+	switch p {
+	case ConvertedTypes.None:
+		return "NONE"
+	case ConvertedTypes.NA:
+		return "UNKNOWN"
+	default:
+		return format.ConvertedType(p).String()
+	}
+}
+
+// ToLogicalType returns the correct LogicalType for the given ConvertedType, using the decimal
+// metadata provided to define the precision/scale if necessary
+func (p ConvertedType) ToLogicalType(convertedDecimal DecimalMetadata) LogicalType {
+	switch p {
+	case ConvertedTypes.UTF8:
+		return StringLogicalType{}
+	case ConvertedTypes.Map, ConvertedTypes.MapKeyValue:
+		return MapLogicalType{}
+	case ConvertedTypes.List:
+		return ListLogicalType{}
+	case ConvertedTypes.Enum:
+		return EnumLogicalType{}
+	case ConvertedTypes.Decimal:
+		return NewDecimalLogicalType(convertedDecimal.Precision, convertedDecimal.Scale)
+	case ConvertedTypes.Date:
+		return DateLogicalType{}
+	case ConvertedTypes.TimeMillis:
+		return NewTimeLogicalType(true, TimeUnitMillis)

Review comment:
       done

##########
File path: go/parquet/schema/converted_types.go
##########
@@ -0,0 +1,191 @@
+// 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.
+
+package schema
+
+import (
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// ConvertedType corresponds to the ConvertedType in the parquet.Thrift,
+// with added values of None and NA for handling when these values are not
+// set in the metadata
+type ConvertedType format.ConvertedType
+
+var (
+	// ConvertedTypes is a struct containing the constants for the types
+	// to make it easy to reference them while making it clear what they are
+	ConvertedTypes = struct {
+		None            ConvertedType
+		UTF8            ConvertedType
+		Map             ConvertedType
+		MapKeyValue     ConvertedType
+		List            ConvertedType
+		Enum            ConvertedType
+		Decimal         ConvertedType
+		Date            ConvertedType
+		TimeMillis      ConvertedType
+		TimeMicros      ConvertedType
+		TimestampMillis ConvertedType
+		TimestampMicros ConvertedType
+		Uint8           ConvertedType
+		Uint16          ConvertedType
+		Uint32          ConvertedType
+		Uint64          ConvertedType
+		Int8            ConvertedType
+		Int16           ConvertedType
+		Int32           ConvertedType
+		Int64           ConvertedType
+		JSON            ConvertedType
+		BSON            ConvertedType
+		Interval        ConvertedType
+		NA              ConvertedType
+	}{
+		None:            -1, // thrift enum starts at 0, so we know this will not be used
+		UTF8:            ConvertedType(format.ConvertedType_UTF8),
+		Map:             ConvertedType(format.ConvertedType_MAP),
+		MapKeyValue:     ConvertedType(format.ConvertedType_MAP_KEY_VALUE),
+		List:            ConvertedType(format.ConvertedType_LIST),
+		Enum:            ConvertedType(format.ConvertedType_ENUM),
+		Decimal:         ConvertedType(format.ConvertedType_DECIMAL),
+		Date:            ConvertedType(format.ConvertedType_DATE),
+		TimeMillis:      ConvertedType(format.ConvertedType_TIME_MILLIS),
+		TimeMicros:      ConvertedType(format.ConvertedType_TIME_MICROS),
+		TimestampMillis: ConvertedType(format.ConvertedType_TIMESTAMP_MILLIS),
+		TimestampMicros: ConvertedType(format.ConvertedType_TIMESTAMP_MICROS),
+		Uint8:           ConvertedType(format.ConvertedType_UINT_8),
+		Uint16:          ConvertedType(format.ConvertedType_UINT_16),
+		Uint32:          ConvertedType(format.ConvertedType_UINT_32),
+		Uint64:          ConvertedType(format.ConvertedType_UINT_64),
+		Int8:            ConvertedType(format.ConvertedType_INT_8),
+		Int16:           ConvertedType(format.ConvertedType_INT_16),
+		Int32:           ConvertedType(format.ConvertedType_INT_32),
+		Int64:           ConvertedType(format.ConvertedType_INT_64),
+		JSON:            ConvertedType(format.ConvertedType_JSON),
+		BSON:            ConvertedType(format.ConvertedType_BSON),
+		Interval:        ConvertedType(format.ConvertedType_INTERVAL),
+		NA:              24, // should always be the last values after Interval
+	}
+)
+
+func (p ConvertedType) String() string {
+	switch p {
+	case ConvertedTypes.None:
+		return "NONE"
+	case ConvertedTypes.NA:
+		return "UNKNOWN"
+	default:
+		return format.ConvertedType(p).String()
+	}
+}
+
+// ToLogicalType returns the correct LogicalType for the given ConvertedType, using the decimal
+// metadata provided to define the precision/scale if necessary
+func (p ConvertedType) ToLogicalType(convertedDecimal DecimalMetadata) LogicalType {
+	switch p {
+	case ConvertedTypes.UTF8:
+		return StringLogicalType{}
+	case ConvertedTypes.Map, ConvertedTypes.MapKeyValue:
+		return MapLogicalType{}
+	case ConvertedTypes.List:
+		return ListLogicalType{}
+	case ConvertedTypes.Enum:
+		return EnumLogicalType{}
+	case ConvertedTypes.Decimal:
+		return NewDecimalLogicalType(convertedDecimal.Precision, convertedDecimal.Scale)
+	case ConvertedTypes.Date:
+		return DateLogicalType{}
+	case ConvertedTypes.TimeMillis:
+		return NewTimeLogicalType(true, TimeUnitMillis)
+	case ConvertedTypes.TimeMicros:
+		return NewTimeLogicalType(true, TimeUnitMicros)
+	case ConvertedTypes.TimestampMillis:
+		t := NewTimestampLogicalType(true, TimeUnitMillis)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.TimestampMicros:
+		t := NewTimestampLogicalType(true, TimeUnitMicros)
+		t.(*TimestampLogicalType).fromConverted = true
+		return t
+	case ConvertedTypes.Interval:
+		return IntervalLogicalType{}
+	case ConvertedTypes.Int8:
+		return NewIntLogicalType(8, true)

Review comment:
       done

##########
File path: go/parquet/schema/helpers.go
##########
@@ -0,0 +1,102 @@
+// 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.
+
+package schema
+
+import "github.com/apache/arrow/go/parquet"
+
+// ListOf is a convenience helper function to create a properly structured
+// list structure according to the Parquet Spec.
+//
+// <list-repetition> group <name> (LIST) {
+//   repeated group list {
+//     <element-repetition> <element-type> element;
+//   }
+// }
+//
+// <list-repetition> can only be optional or required. panics if repeated.
+// <element-repetition> can only be optional or required. panics if repeated.
+func ListOf(n Node, rep parquet.Repetition, fieldID int32) *GroupNode {
+	if rep == parquet.Repetitions.Repeated || n.RepetitionType() == parquet.Repetitions.Repeated {
+		panic("parquet: listof repetition and element repetition must not be repeated.")
+	}
+	listName := n.Name()
+
+	switch n := n.(type) {
+	case *PrimitiveNode:
+		n.name = "element"
+	case *GroupNode:
+		n.name = "element"
+	}
+
+	return NewGroupNodeLogical(listName, rep, FieldList{
+		NewGroupNode("list", parquet.Repetitions.Repeated, FieldList{n}, -1)},

Review comment:
       done

##########
File path: go/parquet/schema/helpers.go
##########
@@ -0,0 +1,102 @@
+// 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.
+
+package schema
+
+import "github.com/apache/arrow/go/parquet"
+
+// ListOf is a convenience helper function to create a properly structured
+// list structure according to the Parquet Spec.
+//
+// <list-repetition> group <name> (LIST) {
+//   repeated group list {
+//     <element-repetition> <element-type> element;
+//   }
+// }
+//
+// <list-repetition> can only be optional or required. panics if repeated.
+// <element-repetition> can only be optional or required. panics if repeated.
+func ListOf(n Node, rep parquet.Repetition, fieldID int32) *GroupNode {
+	if rep == parquet.Repetitions.Repeated || n.RepetitionType() == parquet.Repetitions.Repeated {
+		panic("parquet: listof repetition and element repetition must not be repeated.")
+	}
+	listName := n.Name()
+
+	switch n := n.(type) {
+	case *PrimitiveNode:
+		n.name = "element"
+	case *GroupNode:
+		n.name = "element"
+	}
+
+	return NewGroupNodeLogical(listName, rep, FieldList{
+		NewGroupNode("list", parquet.Repetitions.Repeated, FieldList{n}, -1)},
+		ListLogicalType{}, fieldID)
+}
+
+// MapOf is a convenience helper function to create a properly structured
+// parquet map node setup according to the Parquet Spec.
+//
+// <map-repetition> group <name> (MAP) {
+// 	 repeated group key_value {
+// 	   required <key-type> key;
+//     <value-repetition> <value-type> value;
+//   }
+// }
+//
+// key node will be renamed to "key", value node if not nil will be renamed to "value"
+//
+// <map-repetition> must be only optional or required. panics if repeated is passed.
+//
+// the key node *must* be required repetition. panics if optional or repeated
+//
+// value node can be nil (omitted) or have a repetition of required or optional *only*.
+// panics if value node is not nil and has a repetition of repeated.
+func MapOf(name string, key Node, value Node, mapRep parquet.Repetition, fieldID int32) *GroupNode {
+	if mapRep == parquet.Repetitions.Repeated {
+		panic("parquet: map repetition cannot be Repeated")
+	}
+	if key.RepetitionType() != parquet.Repetitions.Required {
+		panic("parquet: map key repetition must be Required")
+	}
+	if value != nil {
+		if value.RepetitionType() == parquet.Repetitions.Repeated {
+			panic("parquet: map value cannot have repetition Repeated")
+		}
+		switch value := value.(type) {
+		case *PrimitiveNode:
+			value.name = "value"
+		case *GroupNode:
+			value.name = "value"
+		}
+	}
+
+	switch key := key.(type) {
+	case *PrimitiveNode:
+		key.name = "key"
+	case *GroupNode:
+		key.name = "key"
+	}
+
+	keyval := FieldList{key}
+	if value != nil {
+		keyval = append(keyval, value)
+	}
+
+	return NewGroupNodeLogical(name, mapRep, FieldList{
+		NewGroupNode("key_value", parquet.Repetitions.Repeated, keyval, -1),

Review comment:
       done

##########
File path: go/parquet/schema/logical_types_test.go
##########
@@ -0,0 +1,551 @@
+// 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.
+
+package schema_test
+
+import (
+	"encoding/json"
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/schema"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestConvertedLogicalEquivalences(t *testing.T) {
+	tests := []struct {
+		name      string
+		converted schema.ConvertedType
+		logical   schema.LogicalType
+		expected  schema.LogicalType
+	}{
+		{"utf8", schema.ConvertedTypes.UTF8, schema.StringLogicalType{}, schema.StringLogicalType{}},
+		{"map", schema.ConvertedTypes.Map, schema.MapLogicalType{}, schema.MapLogicalType{}},
+		{"mapkeyval", schema.ConvertedTypes.MapKeyValue, schema.MapLogicalType{}, schema.MapLogicalType{}},
+		{"list", schema.ConvertedTypes.List, schema.NewListLogicalType(), schema.NewListLogicalType()},
+		{"enum", schema.ConvertedTypes.Enum, schema.EnumLogicalType{}, schema.EnumLogicalType{}},
+		{"date", schema.ConvertedTypes.Date, schema.DateLogicalType{}, schema.DateLogicalType{}},
+		{"timemilli", schema.ConvertedTypes.TimeMillis, schema.NewTimeLogicalType(true, schema.TimeUnitMillis), &schema.TimeLogicalType{}},
+		{"timemicro", schema.ConvertedTypes.TimeMicros, schema.NewTimeLogicalType(true, schema.TimeUnitMicros), &schema.TimeLogicalType{}},
+		{"timestampmilli", schema.ConvertedTypes.TimestampMillis, schema.NewTimestampLogicalType(true, schema.TimeUnitMillis), &schema.TimestampLogicalType{}},
+		{"timestampmicro", schema.ConvertedTypes.TimestampMicros, schema.NewTimestampLogicalType(true, schema.TimeUnitMicros), &schema.TimestampLogicalType{}},
+		{"uint8", schema.ConvertedTypes.Uint8, schema.NewIntLogicalType(8, false), &schema.IntLogicalType{}},
+		{"uint16", schema.ConvertedTypes.Uint16, schema.NewIntLogicalType(16, false), &schema.IntLogicalType{}},
+		{"uint32", schema.ConvertedTypes.Uint32, schema.NewIntLogicalType(32, false), &schema.IntLogicalType{}},
+		{"uint64", schema.ConvertedTypes.Uint64, schema.NewIntLogicalType(64, false), &schema.IntLogicalType{}},
+		{"int8", schema.ConvertedTypes.Int8, schema.NewIntLogicalType(8, true), &schema.IntLogicalType{}},

Review comment:
       done

##########
File path: go/parquet/schema/node.go
##########
@@ -0,0 +1,591 @@
+// 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.
+
+package schema
+
+import (
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/thrift/lib/go/thrift"
+	"golang.org/x/xerrors"
+)
+
+// NodeType describes whether the Node is a Primitive or Group node
+type NodeType int
+
+// the available constants for NodeType
+const (
+	Primitive NodeType = iota
+	Group
+)
+
+// Node is the interface for both Group and Primitive Nodes.
+// A logical schema type has a name, repetition level, and optionally
+// a logical type (ConvertedType in Parquet metadata parlance)

Review comment:
       fixed

##########
File path: go/parquet/schema/node.go
##########
@@ -0,0 +1,591 @@
+// 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.
+
+package schema
+
+import (
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/thrift/lib/go/thrift"
+	"golang.org/x/xerrors"
+)
+
+// NodeType describes whether the Node is a Primitive or Group node
+type NodeType int
+
+// the available constants for NodeType
+const (
+	Primitive NodeType = iota
+	Group
+)
+
+// Node is the interface for both Group and Primitive Nodes.
+// A logical schema type has a name, repetition level, and optionally
+// a logical type (ConvertedType in Parquet metadata parlance)
+type Node interface {
+	Name() string
+	Type() NodeType
+	RepetitionType() parquet.Repetition
+	ConvertedType() ConvertedType
+	LogicalType() LogicalType
+	FieldID() int32
+	Parent() Node
+	SetParent(Node)
+	Path() string
+	Equals(Node) bool
+	Visit(v Visitor)
+	toThrift() *format.SchemaElement
+}
+
+// Visitor is an interface for creating functionality to walk the schema tree.
+//
+// A visitor can be passed to the Visit function of a Node in order to walk
+// the tree. VisitPre is called the first time a node is encountered. If
+// it is a group node, the return is checked and if it is false, the children
+// will be skipped.
+//
+// VisitPost is called after visiting any children
+type Visitor interface {
+	VisitPre(Node) bool
+	VisitPost(Node)
+}
+
+// ColumnPathFromNode walks the parents of the given node to construct it's
+// column path
+func ColumnPathFromNode(n Node) parquet.ColumnPath {
+	if n == nil {
+		return nil
+	}
+
+	c := make([]string, 0)
+
+	cursor := n
+	for cursor.Parent() != nil {
+		c = append(c, cursor.Name())
+		cursor = cursor.Parent()
+	}
+
+	for i := len(c)/2 - 1; i >= 0; i-- {

Review comment:
       added comments

##########
File path: go/parquet/schema/reflection.go
##########
@@ -0,0 +1,791 @@
+// 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.
+
+package schema
+
+import (
+	"reflect"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+type taggedInfo struct {
+	Name string
+
+	Type      parquet.Type
+	KeyType   parquet.Type
+	ValueType parquet.Type
+
+	Length      int32
+	KeyLength   int32
+	ValueLength int32
+
+	Scale      int32
+	KeyScale   int32
+	ValueScale int32
+
+	Precision      int32
+	KeyPrecision   int32
+	ValuePrecision int32
+
+	FieldID      int32
+	KeyFieldID   int32
+	ValueFieldID int32
+
+	RepetitionType  parquet.Repetition
+	ValueRepetition parquet.Repetition
+
+	Converted      ConvertedType
+	KeyConverted   ConvertedType
+	ValueConverted ConvertedType
+
+	LogicalFields      map[string]string
+	KeyLogicalFields   map[string]string
+	ValueLogicalFields map[string]string
+
+	LogicalType      LogicalType
+	KeyLogicalType   LogicalType
+	ValueLogicalType LogicalType
+}
+
+func (t *taggedInfo) CopyForKey() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.KeyType
+	ret.Length = t.KeyLength
+	ret.Scale = t.KeyScale
+	ret.Precision = t.KeyPrecision
+	ret.FieldID = t.KeyFieldID
+	ret.RepetitionType = parquet.Repetitions.Required
+	ret.Converted = t.KeyConverted
+	ret.LogicalType = t.KeyLogicalType
+	return
+}
+
+func (t *taggedInfo) CopyForValue() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.ValueType
+	ret.Length = t.ValueLength
+	ret.Scale = t.ValueScale
+	ret.Precision = t.ValuePrecision
+	ret.FieldID = t.ValueFieldID
+	ret.RepetitionType = t.ValueRepetition
+	ret.Converted = t.ValueConverted
+	ret.LogicalType = t.ValueLogicalType
+	return
+}
+
+func (t *taggedInfo) UpdateLogicalTypes() {
+	processLogicalType := func(fields map[string]string, precision, scale int32) LogicalType {
+		t, ok := fields["type"]
+		if !ok {
+			return NoLogicalType{}
+		}
+
+		switch strings.ToLower(t) {
+		case "string":
+			return StringLogicalType{}
+		case "map":
+			return MapLogicalType{}
+		case "list":
+			return ListLogicalType{}
+		case "enum":
+			return EnumLogicalType{}
+		case "decimal":
+			if v, ok := fields["precision"]; ok {
+				precision = int32FromType(v)
+			}
+			if v, ok := fields["scale"]; ok {
+				scale = int32FromType(v)
+			}
+			return NewDecimalLogicalType(precision, scale)
+		case "date":
+			return DateLogicalType{}
+		case "time":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimeLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(strings.ToLower(unit)))
+		case "timestamp":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimestampLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(unit))
+		case "integer":
+			width, ok := fields["bitwidth"]
+			if !ok {
+				panic("must specify bitwidth if explicitly setting integer logical type")
+			}
+			signed, ok := fields["signed"]
+			if !ok {
+				signed = "true"
+			}
+
+			return NewIntLogicalType(int8(int32FromType(width)), boolFromStr(signed))
+		case "null":
+			return NullLogicalType{}
+		case "json":
+			return JSONLogicalType{}
+		case "bson":
+			return BSONLogicalType{}
+		case "uuid":
+			return UUIDLogicalType{}
+		default:
+			panic(xerrors.Errorf("invalid logical type specified: %s", t))
+		}
+	}
+
+	t.LogicalType = processLogicalType(t.LogicalFields, t.Precision, t.Scale)
+	t.KeyLogicalType = processLogicalType(t.KeyLogicalFields, t.KeyPrecision, t.KeyScale)
+	t.ValueLogicalType = processLogicalType(t.ValueLogicalFields, t.ValuePrecision, t.ValueScale)
+}
+
+func newTaggedInfo() taggedInfo {
+	return taggedInfo{
+		Type:               parquet.Types.Undefined,
+		KeyType:            parquet.Types.Undefined,
+		ValueType:          parquet.Types.Undefined,
+		RepetitionType:     parquet.Repetitions.Undefined,
+		ValueRepetition:    parquet.Repetitions.Undefined,
+		Converted:          ConvertedTypes.NA,
+		KeyConverted:       ConvertedTypes.NA,
+		ValueConverted:     ConvertedTypes.NA,
+		FieldID:            -1,
+		KeyFieldID:         -1,
+		ValueFieldID:       -1,
+		LogicalFields:      make(map[string]string),
+		KeyLogicalFields:   make(map[string]string),
+		ValueLogicalFields: make(map[string]string),
+		LogicalType:        NoLogicalType{},
+		KeyLogicalType:     NoLogicalType{},
+		ValueLogicalType:   NoLogicalType{},
+	}
+}
+
+var int32FromType = func(v string) int32 {
+	val, err := strconv.Atoi(v)
+	if err != nil {
+		panic(err)
+	}
+	return int32(val)
+}
+
+var boolFromStr = func(v string) bool {
+	val, err := strconv.ParseBool(v)
+	if err != nil {
+		panic(err)
+	}
+	return val
+}
+
+func infoFromTags(f reflect.StructTag) *taggedInfo {
+	typeFromStr := func(v string) parquet.Type {
+		t, err := format.TypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(xerrors.Errorf("invalid type specified: %s", v))
+		}
+		return parquet.Type(t)
+	}
+
+	repFromStr := func(v string) parquet.Repetition {
+		r, err := format.FieldRepetitionTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return parquet.Repetition(r)
+	}
+
+	convertedFromStr := func(v string) ConvertedType {
+		c, err := format.ConvertedTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return ConvertedType(c)
+	}
+
+	if ptags, ok := f.Lookup("parquet"); ok {
+		info := newTaggedInfo()
+		for _, tag := range strings.Split(strings.Replace(ptags, "\t", "", -1), ",") {
+			tag = strings.TrimSpace(tag)
+			kv := strings.SplitN(tag, "=", 2)
+			key := strings.TrimSpace(strings.ToLower(kv[0]))
+			value := strings.TrimSpace(kv[1])
+
+			switch key {
+			case "name":
+				info.Name = value
+			case "type":
+				info.Type = typeFromStr(value)
+			case "keytype":
+				info.KeyType = typeFromStr(value)
+			case "valuetype":
+				info.ValueType = typeFromStr(value)
+			case "length":
+				info.Length = int32FromType(value)
+			case "keylength":
+				info.KeyLength = int32FromType(value)
+			case "valuelength":
+				info.ValueLength = int32FromType(value)
+			case "scale":
+				info.Scale = int32FromType(value)
+			case "keyscale":
+				info.KeyScale = int32FromType(value)
+			case "valuescale":
+				info.ValueScale = int32FromType(value)
+			case "precision":
+				info.Precision = int32FromType(value)
+			case "keyprecision":
+				info.KeyPrecision = int32FromType(value)
+			case "valueprecision":
+				info.ValuePrecision = int32FromType(value)
+			case "fieldid":
+				info.FieldID = int32FromType(value)
+			case "keyfieldid":
+				info.KeyFieldID = int32FromType(value)
+			case "valuefieldid":
+				info.ValueFieldID = int32FromType(value)
+			case "repetition":
+				info.RepetitionType = repFromStr(value)
+			case "valuerepetition":
+				info.ValueRepetition = repFromStr(value)
+			case "converted":
+				info.Converted = convertedFromStr(value)
+			case "keyconverted":
+				info.KeyConverted = convertedFromStr(value)
+			case "valueconverted":
+				info.ValueConverted = convertedFromStr(value)
+			case "logical":
+				info.LogicalFields["type"] = value
+			case "keylogical":
+				info.KeyLogicalFields["type"] = value
+			case "valuelogical":
+				info.ValueLogicalFields["type"] = value
+			default:
+				switch {
+				case strings.HasPrefix(key, "logical."):
+					info.LogicalFields[strings.TrimPrefix(key, "logical.")] = value
+				case strings.HasPrefix(key, "keylogical."):
+					info.KeyLogicalFields[strings.TrimPrefix(key, "keylogical.")] = value
+				case strings.HasPrefix(key, "valuelogical."):
+					info.ValueLogicalFields[strings.TrimPrefix(key, "valuelogical.")] = value
+				}
+			}
+		}
+		info.UpdateLogicalTypes()
+		return &info
+	}
+	return nil
+}
+
+func typeToNode(name string, typ reflect.Type, repType parquet.Repetition, info *taggedInfo) Node {

Review comment:
       added comments to the function and the rest of the function body

##########
File path: go/parquet/schema/schema_element_test.go
##########
@@ -0,0 +1,432 @@
+// 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.
+
+package schema
+
+import (
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/suite"
+)
+
+type schemaElementConstruction struct {
+	node            Node
+	element         *format.SchemaElement
+	name            string
+	expectConverted bool
+	converted       ConvertedType
+	expectLogical   bool
+	checkLogical    func(*format.SchemaElement) bool
+}
+
+type decimalSchemaElementConstruction struct {
+	schemaElementConstruction
+	precision int
+	scale     int
+}
+
+type temporalSchemaElementConstruction struct {
+	schemaElementConstruction
+	adjusted bool
+	unit     TimeUnitType
+	getUnit  func(*format.SchemaElement) *format.TimeUnit
+}
+
+type intSchemaElementConstruction struct {
+	schemaElementConstruction
+	width  int8
+	signed bool
+}
+
+type legacySchemaElementConstructArgs struct {
+	name            string
+	physical        parquet.Type
+	len             int
+	expectConverted bool
+	converted       ConvertedType
+	expectLogical   bool
+	checkLogical    func(*format.SchemaElement) bool
+}
+
+type schemaElementConstructArgs struct {
+	name            string
+	logical         LogicalType
+	physical        parquet.Type
+	len             int
+	expectConverted bool
+	converted       ConvertedType
+	expectLogical   bool
+	checkLogical    func(*format.SchemaElement) bool
+}
+type SchemaElementConstructionSuite struct {
+	suite.Suite
+}
+
+func (s *SchemaElementConstructionSuite) reconstruct(c schemaElementConstructArgs) *schemaElementConstruction {
+	ret := &schemaElementConstruction{
+		node:            NewPrimitiveNodeLogical(c.name, parquet.Repetitions.Required, c.logical, c.physical, c.len, -1),
+		name:            c.name,
+		expectConverted: c.expectConverted,
+		converted:       c.converted,
+		expectLogical:   c.expectLogical,
+		checkLogical:    c.checkLogical,
+	}
+	ret.element = ret.node.toThrift()
+	return ret
+}
+
+func (s *SchemaElementConstructionSuite) legacyReconstruct(c legacySchemaElementConstructArgs) *schemaElementConstruction {
+	ret := &schemaElementConstruction{
+		node:            NewPrimitiveNodeConverted(c.name, parquet.Repetitions.Required, c.physical, c.converted, c.len, 0, 0, -1),
+		name:            c.name,
+		expectConverted: c.expectConverted,
+		converted:       c.converted,
+		expectLogical:   c.expectLogical,
+		checkLogical:    c.checkLogical,
+	}
+	ret.element = ret.node.toThrift()
+	return ret
+}
+
+func (s *SchemaElementConstructionSuite) inspect(c *schemaElementConstruction) {
+	if c.expectConverted {
+		s.True(c.element.IsSetConvertedType())
+		s.Equal(c.converted, ConvertedType(*c.element.ConvertedType))
+	} else {
+		s.False(c.element.IsSetConvertedType())
+	}
+	if c.expectLogical {
+		s.True(c.element.IsSetLogicalType())
+		s.True(c.checkLogical(c.element))
+	} else {
+		s.False(c.element.IsSetLogicalType())
+	}
+}
+
+func (s *SchemaElementConstructionSuite) TestSimple() {
+	checkNone := func(*format.SchemaElement) bool { return true }
+
+	tests := []struct {
+		name   string
+		args   *schemaElementConstructArgs
+		legacy *legacySchemaElementConstructArgs
+	}{
+		{"string", &schemaElementConstructArgs{
+			"string", StringLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.UTF8, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetSTRING() },
+		}, nil},
+		{"enum", &schemaElementConstructArgs{
+			"enum", EnumLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.Enum, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetENUM() },
+		}, nil},
+		{"date", &schemaElementConstructArgs{
+			"date", DateLogicalType{}, parquet.Types.Int32, -1, true, ConvertedTypes.Date, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetDATE() },
+		}, nil},
+		{"interval", &schemaElementConstructArgs{
+			"interval", IntervalLogicalType{}, parquet.Types.FixedLenByteArray, 12, true, ConvertedTypes.Interval, false,
+			checkNone,
+		}, nil},
+		{"null", &schemaElementConstructArgs{
+			"null", NullLogicalType{}, parquet.Types.Double, -1, false, ConvertedTypes.NA, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetUNKNOWN() },
+		}, nil},
+		{"json", &schemaElementConstructArgs{
+			"json", JSONLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.JSON, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetJSON() },
+		}, nil},
+		{"bson", &schemaElementConstructArgs{
+			"bson", BSONLogicalType{}, parquet.Types.ByteArray, -1, true, ConvertedTypes.BSON, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetBSON() },
+		}, nil},
+		{"uuid", &schemaElementConstructArgs{
+			"uuid", UUIDLogicalType{}, parquet.Types.FixedLenByteArray, 16, false, ConvertedTypes.NA, true,
+			func(e *format.SchemaElement) bool { return e.LogicalType.IsSetUUID() },
+		}, nil},
+		{"none", &schemaElementConstructArgs{
+			"none", NoLogicalType{}, parquet.Types.Int64, -1, false, ConvertedTypes.NA, false,
+			checkNone,
+		}, nil},
+		{"unknown", &schemaElementConstructArgs{
+			"unknown", UnknownLogicalType{}, parquet.Types.Int64, -1, true, ConvertedTypes.NA, false,
+			checkNone,
+		}, nil},
+		{"timestamp_ms", nil, &legacySchemaElementConstructArgs{
+			"timestamp_ms", parquet.Types.Int64, -1, true, ConvertedTypes.TimestampMillis, false, checkNone}},
+		{"timestamp_us", nil, &legacySchemaElementConstructArgs{
+			"timestamp_us", parquet.Types.Int64, -1, true, ConvertedTypes.TimestampMicros, false, checkNone}},
+	}
+	for _, tt := range tests {
+		s.Run(tt.name, func() {
+			var sc *schemaElementConstruction
+			if tt.args != nil {
+				sc = s.reconstruct(*tt.args)
+			} else {
+				sc = s.legacyReconstruct(*tt.legacy)
+			}
+			s.Equal(tt.name, sc.element.Name)
+			s.inspect(sc)
+		})
+	}
+}
+
+func (s *SchemaElementConstructionSuite) reconstructDecimal(c schemaElementConstructArgs) *decimalSchemaElementConstruction {
+	ret := s.reconstruct(c)
+	dec := c.logical.(*DecimalLogicalType)
+	return &decimalSchemaElementConstruction{*ret, int(dec.Precision()), int(dec.Scale())}
+}
+
+func (s *SchemaElementConstructionSuite) inspectDecimal(d *decimalSchemaElementConstruction) {
+	s.inspect(&d.schemaElementConstruction)
+	s.EqualValues(d.precision, d.element.GetPrecision())
+	s.EqualValues(d.scale, d.element.GetScale())
+	s.EqualValues(d.precision, d.element.LogicalType.DECIMAL.Precision)
+	s.EqualValues(d.scale, d.element.LogicalType.DECIMAL.Scale)
+}
+
+func (s *SchemaElementConstructionSuite) TestDecimal() {
+	checkDecimal := func(p *format.SchemaElement) bool { return p.LogicalType.IsSetDECIMAL() }
+
+	tests := []schemaElementConstructArgs{
+		{"decimal16_6", NewDecimalLogicalType(16, 6), parquet.Types.Int64, -1, true, ConvertedTypes.Decimal, true, checkDecimal},
+		{"decimal1_0", NewDecimalLogicalType(1, 0), parquet.Types.Int32, -1, true, ConvertedTypes.Decimal, true, checkDecimal},
+		{"decimal10", NewDecimalLogicalType(10, 0), parquet.Types.Int64, -1, true, ConvertedTypes.Decimal, true, checkDecimal},

Review comment:
       added

##########
File path: go/parquet/schema/schema_flatten_test.go
##########
@@ -0,0 +1,150 @@
+// 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.
+
+package schema
+
+import (
+	"testing"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/suite"
+)
+
+func NewPrimitive(name string, repetition format.FieldRepetitionType, typ format.Type, fieldID int32) *format.SchemaElement {
+	ret := &format.SchemaElement{
+		Name:           name,
+		RepetitionType: format.FieldRepetitionTypePtr(repetition),
+		Type:           format.TypePtr(typ),
+	}
+	if fieldID >= 0 {
+		ret.FieldID = &fieldID
+	}
+	return ret
+}
+
+func NewGroup(name string, repetition format.FieldRepetitionType, numChildren, fieldID int32) *format.SchemaElement {
+	ret := &format.SchemaElement{
+		Name:           name,
+		RepetitionType: format.FieldRepetitionTypePtr(repetition),
+		NumChildren:    &numChildren,
+	}
+	if fieldID >= 0 {
+		ret.FieldID = &fieldID
+	}
+	return ret
+}
+
+type SchemaFlattenSuite struct {
+	suite.Suite
+
+	name string
+}
+
+func (s *SchemaFlattenSuite) SetupSuite() {
+	s.name = "parquet_schema"
+}
+
+func (s *SchemaFlattenSuite) TestDecimalMetadata() {
+	group := NewGroupNodeConverted("group", parquet.Repetitions.Repeated, FieldList{
+		NewPrimitiveNodeConverted("decimal", parquet.Repetitions.Required, parquet.Types.Int64, ConvertedTypes.Decimal, 0, 8, 4, -1),

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r616854812



##########
File path: go/parquet/schema/column.go
##########
@@ -0,0 +1,125 @@
+// 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.
+
+package schema
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// Column encapsulates the information necessary to interpret primitive
+// column data in the context of a particular schema. We have to examine
+// the node structure of a column's path to the root in the schema tree
+// to be able to reassemble the nested structure from the repetition and
+// definition levels.
+type Column struct {
+	pnode     *PrimitiveNode
+	maxDefLvl int16
+	maxRepLvl int16
+}
+
+func (c *Column) hasSpacedValues() bool {
+	if c.maxRepLvl > 0 {
+		// repeated + flat case
+		return c.pnode.RepetitionType() != parquet.Repetitions.Required
+	}
+
+	// non-repeated+nested case
+	// find if a node forces nulls in the lowest level along the hierarchy
+	var n Node = c.pnode
+	for n != nil {
+		if n.RepetitionType() == parquet.Repetitions.Optional {

Review comment:
       although really, if this isn't related to arrow code, I don't think we need this method here, reading spaced values isn't something lower level parquet reading should really be concerned with




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-826910344


   @emkornfield i've added the comments requested and docs and answered questions. Lemme know if there's anything else we need. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] emkornfield commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r631929082



##########
File path: go/parquet/schema/logical_types.go
##########
@@ -0,0 +1,1089 @@
+// 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.
+
+package schema
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+)
+
+// DecimalMetadata is a struct for managing scale and precision information between
+// converted and logical types.
+type DecimalMetadata struct {
+	IsSet     bool
+	Scale     int32
+	Precision int32
+}
+
+func getLogicalType(l *format.LogicalType) LogicalType {
+	switch {
+	case l.IsSetSTRING():
+		return StringLogicalType{}
+	case l.IsSetMAP():
+		return MapLogicalType{}
+	case l.IsSetLIST():
+		return ListLogicalType{}
+	case l.IsSetENUM():
+		return EnumLogicalType{}
+	case l.IsSetDECIMAL():
+		return &DecimalLogicalType{typ: l.DECIMAL}
+	case l.IsSetDATE():
+		return DateLogicalType{}
+	case l.IsSetTIME():
+		if timeUnitFromThrift(l.TIME.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+		}
+		return &TimeLogicalType{typ: l.TIME}
+	case l.IsSetTIMESTAMP():
+		if timeUnitFromThrift(l.TIMESTAMP.Unit) == TimeUnitUnknown {
+			panic("parquet: TimeUnit must be one of MILLIS, MICROS, or NANOS for Timestamp logical type")
+		}
+		return &TimestampLogicalType{typ: l.TIMESTAMP}
+	case l.IsSetINTEGER():
+		return &IntLogicalType{typ: l.INTEGER}
+	case l.IsSetUNKNOWN():
+		return NullLogicalType{}
+	case l.IsSetJSON():
+		return JSONLogicalType{}
+	case l.IsSetBSON():
+		return BSONLogicalType{}
+	case l.IsSetUUID():
+		return UUIDLogicalType{}
+	case l == nil:
+		return NoLogicalType{}
+	default:
+		panic("invalid logical type")
+	}
+}
+
+// TimeUnitType is an enum for denoting whether a time based logical type
+// is using milliseconds, microseconds or nanoseconds.
+type TimeUnitType int
+
+// Constants for the TimeUnitType
+const (
+	TimeUnitMillis TimeUnitType = iota
+	TimeUnitMicros
+	TimeUnitNanos
+	TimeUnitUnknown
+)
+
+// LogicalType is the descriptor that defines the usage of a physical primitive
+// type in the schema, such as an Interval, Date, etc.
+type LogicalType interface {
+	// Returns true if a nested type like List or Map
+	IsNested() bool
+	// Returns true if this type can be serialized, ie: not Unknown/NoType/Interval
+	IsSerialized() bool
+	// Returns true if not NoLogicalType
+	IsValid() bool
+	// Returns true if it is NoType
+	IsNone() bool
+	// returns a string representation of the Logical Type
+	String() string
+	toThrift() *format.LogicalType
+	// Return the equivalent ConvertedType for legacy Parquet systems
+	ToConvertedType() (ConvertedType, DecimalMetadata)
+	// Returns true if the specified ConvertedType is compatible with this
+	// logical type
+	IsCompatible(ConvertedType, DecimalMetadata) bool
+	// Returns true if this logical type can be used with the provided physical type
+	IsApplicable(t parquet.Type, tlen int32) bool
+	// Returns true if the logical types are the same
+	Equals(LogicalType) bool
+	// Returns the default stat sort order for this logical type
+	SortOrder() SortOrder
+}
+
+// TemporalLogicalType is a smaller interface for Time based logical types
+// like Time / Timestamp
+type TemporalLogicalType interface {
+	LogicalType
+	IsAdjustedToUTC() bool
+	TimeUnit() TimeUnitType
+}
+
+// SortOrder mirrors the parquet.thrift sort order type
+type SortOrder int8
+
+// Constants for the Stat sort order definitions
+const (
+	SortSIGNED SortOrder = iota
+	SortUNSIGNED
+	SortUNKNOWN
+)
+
+// DefaultSortOrder returns the default stat sort order for the given physical type
+func DefaultSortOrder(primitive format.Type) SortOrder {
+	switch primitive {
+	case format.Type_BOOLEAN, format.Type_INT32, format.Type_INT64, format.Type_FLOAT, format.Type_DOUBLE:
+		return SortSIGNED
+	case format.Type_BYTE_ARRAY, format.Type_FIXED_LEN_BYTE_ARRAY:
+		return SortUNSIGNED
+	case format.Type_INT96:
+		fallthrough
+	default:
+		return SortUNKNOWN
+	}
+}
+
+// GetLogicalSortOrder returns the default sort order for this logical type
+// or falls back to the default sort order for the physical type if not valid
+func GetLogicalSortOrder(logical LogicalType, primitive format.Type) SortOrder {
+	switch {
+	case logical == nil || !logical.IsValid():
+		return SortUNKNOWN
+	case logical.Equals(NoLogicalType{}):
+		return DefaultSortOrder(primitive)
+	default:
+		return logical.SortOrder()
+	}
+}
+
+type baseLogicalType struct{}
+
+func (baseLogicalType) IsSerialized() bool {
+	return true
+}
+
+func (baseLogicalType) IsValid() bool {
+	return true
+}
+
+func (baseLogicalType) IsNested() bool {
+	return false
+}
+
+func (baseLogicalType) IsNone() bool { return false }
+
+// StringLogicalType is a UTF8 string, only usable with ByteArray and FixedLenByteArray
+type StringLogicalType struct{ baseLogicalType }
+
+func (StringLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (StringLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": StringLogicalType{}.String()})
+}
+
+func (StringLogicalType) String() string {
+	return "String"
+}
+
+func (StringLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.UTF8, DecimalMetadata{}
+}
+
+func (StringLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.UTF8 && !dec.IsSet
+}
+
+func (StringLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (StringLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{STRING: format.NewStringType()}
+}
+
+func (StringLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(StringLogicalType)
+	return ok
+}
+
+// MapLogicalType represents a mapped type
+type MapLogicalType struct{ baseLogicalType }
+
+func (MapLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (MapLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": MapLogicalType{}.String()})
+}
+
+func (MapLogicalType) String() string {
+	return "Map"
+}
+
+func (MapLogicalType) IsNested() bool {
+	return true
+}
+
+func (MapLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Map, DecimalMetadata{}
+}
+
+func (MapLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return (t == ConvertedTypes.Map || t == ConvertedTypes.MapKeyValue) && !dec.IsSet
+}
+
+func (MapLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (MapLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{MAP: format.NewMapType()}
+}
+
+func (MapLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(MapLogicalType)
+	return ok
+}
+
+func NewListLogicalType() LogicalType {
+	return ListLogicalType{}
+}
+
+// ListLogicalType is used for columns which are themselves nested lists
+type ListLogicalType struct{ baseLogicalType }
+
+func (ListLogicalType) SortOrder() SortOrder {
+	return SortUNKNOWN
+}
+
+func (ListLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": ListLogicalType{}.String()})
+}
+
+func (ListLogicalType) String() string {
+	return "List"
+}
+
+func (ListLogicalType) IsNested() bool {
+	return true
+}
+
+func (ListLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.List, DecimalMetadata{}
+}
+
+func (ListLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.List && !dec.IsSet
+}
+
+func (ListLogicalType) IsApplicable(parquet.Type, int32) bool {
+	return false
+}
+
+func (ListLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{LIST: format.NewListType()}
+}
+
+func (ListLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(ListLogicalType)
+	return ok
+}
+
+// EnumLogicalType is for representing an enum, which should be a byte array type
+type EnumLogicalType struct{ baseLogicalType }
+
+func (EnumLogicalType) SortOrder() SortOrder {
+	return SortUNSIGNED
+}
+
+func (EnumLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": EnumLogicalType{}.String()})
+}
+
+func (EnumLogicalType) String() string {
+	return "Enum"
+}
+
+func (EnumLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Enum, DecimalMetadata{}
+}
+
+func (EnumLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Enum && !dec.IsSet
+}
+
+func (EnumLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.ByteArray
+}
+
+func (EnumLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{ENUM: format.NewEnumType()}
+}
+
+func (EnumLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(EnumLogicalType)
+	return ok
+}
+
+// NewDecimalLogicalType returns a Decimal logical type with the given
+// precision and scale.
+//
+// Panics if precision < 1 or scale is not in the range (0, precision)
+func NewDecimalLogicalType(precision int32, scale int32) LogicalType {
+	if precision < 1 {
+		panic("parquet: precision must be greater than or equal to 1 for decimal logical type")
+	}
+	if scale < 0 || scale > precision {
+		panic("parquet: scale must be a non-negative integer that does not exceed precision for decimal logical type")
+	}
+	return &DecimalLogicalType{typ: &format.DecimalType{Precision: precision, Scale: scale}}
+}
+
+// DecimalLogicalType is used to represent a decimal value of a given
+// precision and scale
+type DecimalLogicalType struct {
+	baseLogicalType
+	typ *format.DecimalType
+}
+
+func (t DecimalLogicalType) Precision() int32 {
+	return t.typ.Precision
+}
+
+func (t DecimalLogicalType) Scale() int32 {
+	return t.typ.Scale
+}
+
+func (DecimalLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t DecimalLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{"Type": "Decimal", "precision": t.typ.Precision, "scale": t.typ.Scale})
+}
+
+func (t DecimalLogicalType) String() string {
+	return fmt.Sprintf("Decimal(precision=%d, scale=%d)", t.typ.Precision, t.typ.Scale)
+}
+
+func (t DecimalLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Decimal, DecimalMetadata{IsSet: true, Scale: t.typ.GetScale(), Precision: t.typ.GetPrecision()}
+}
+
+func (t DecimalLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	return c == ConvertedTypes.Decimal &&
+		dec.IsSet && dec.Scale == t.typ.Scale && dec.Precision == t.typ.Precision
+}
+
+func (t DecimalLogicalType) IsApplicable(typ parquet.Type, tlen int32) bool {
+	switch typ {
+	case parquet.Types.Int32:
+		return 1 <= t.typ.Precision && t.typ.Precision <= 9
+	case parquet.Types.Int64:
+		if t.typ.Precision < 10 {
+			debug.Log("int64 used for decimal logical, precision is small enough to use int32")
+		}
+		return 1 <= t.typ.Precision && t.typ.Precision <= 18
+	case parquet.Types.FixedLenByteArray:
+		return t.typ.Precision <= int32(math.Floor(math.Log10(math.Pow(2.0, (8.0*float64(tlen)-1.0)))))
+	case parquet.Types.ByteArray:
+		return true
+	}
+	return false
+}
+
+func (t DecimalLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DECIMAL: t.typ}
+}
+
+func (t DecimalLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*DecimalLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.Precision == other.typ.Precision && t.typ.Scale == other.typ.Scale
+}
+
+// DateLogicalType is an int32 representing the number of days since the Unix Epoch
+// 1 January 1970
+type DateLogicalType struct{ baseLogicalType }
+
+func (DateLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (DateLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]string{"Type": DateLogicalType{}.String()})
+}
+
+func (DateLogicalType) String() string {
+	return "Date"
+}
+
+func (DateLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	return ConvertedTypes.Date, DecimalMetadata{}
+}
+
+func (DateLogicalType) IsCompatible(t ConvertedType, dec DecimalMetadata) bool {
+	return t == ConvertedTypes.Date && !dec.IsSet
+}
+
+func (DateLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int32
+}
+
+func (DateLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{DATE: format.NewDateType()}
+}
+
+func (DateLogicalType) Equals(rhs LogicalType) bool {
+	_, ok := rhs.(DateLogicalType)
+	return ok
+}
+
+func timeUnitFromThrift(unit *format.TimeUnit) TimeUnitType {
+	switch {
+	case unit == nil:
+		return TimeUnitUnknown
+	case unit.IsSetMILLIS():
+		return TimeUnitMillis
+	case unit.IsSetMICROS():
+		return TimeUnitMicros
+	case unit.IsSetNANOS():
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func timeUnitToString(unit *format.TimeUnit) string {
+	switch {
+	case unit == nil:
+		return "unknown"
+	case unit.IsSetMILLIS():
+		return "milliseconds"
+	case unit.IsSetMICROS():
+		return "microseconds"
+	case unit.IsSetNANOS():
+		return "nanoseconds"
+	default:
+		return "unknown"
+	}
+}
+
+func timeUnitFromString(v string) TimeUnitType {
+	switch v {
+	case "millis":
+		return TimeUnitMillis
+	case "micros":
+		return TimeUnitMicros
+	case "nanos":
+		return TimeUnitNanos
+	default:
+		return TimeUnitUnknown
+	}
+}
+
+func createTimeUnit(unit TimeUnitType) *format.TimeUnit {
+	tunit := format.NewTimeUnit()
+	switch unit {
+	case TimeUnitMicros:
+		tunit.MICROS = format.NewMicroSeconds()
+	case TimeUnitMillis:
+		tunit.MILLIS = format.NewMilliSeconds()
+	case TimeUnitNanos:
+		tunit.NANOS = format.NewNanoSeconds()
+	default:
+		panic("parquet: time unit must be one of MILLIS, MICROS, or NANOS for Time logical type")
+	}
+	return tunit
+}
+
+// NewTimeLogicalType returns a time type of the given unit.
+func NewTimeLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimeLogicalType{typ: &format.TimeType{
+		IsAdjustedToUTC: isAdjustedToUTC,
+		Unit:            createTimeUnit(unit),
+	}}
+}
+
+// TimeLogicalType is a time type without a date and must be an
+// int32 for milliseconds, or an int64 for micro or nano seconds.
+type TimeLogicalType struct {
+	baseLogicalType
+	typ *format.TimeType
+}
+
+func (t TimeLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimeLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimeLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimeLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type": "Time", "isAdjustedToUTC": t.typ.IsAdjustedToUTC, "timeUnit": timeUnitToString(t.typ.GetUnit())})
+}
+
+func (t TimeLogicalType) String() string {
+	return fmt.Sprintf("Time(isAdjustedToUTC=%t, timeUnit=%s)", t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()))
+}
+
+func (t TimeLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimeMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimeMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimeLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC {
+		switch unit {
+		case TimeUnitMillis:
+			return c == ConvertedTypes.TimeMillis
+		case TimeUnitMicros:
+			return c == ConvertedTypes.TimeMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (t TimeLogicalType) IsApplicable(typ parquet.Type, _ int32) bool {
+	return (typ == parquet.Types.Int32 && t.typ.GetUnit().IsSetMILLIS()) ||
+		(typ == parquet.Types.Int64 &&
+			(t.typ.GetUnit().IsSetMICROS() || t.typ.GetUnit().IsSetNANOS()))
+}
+
+func (t TimeLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIME: t.typ}
+}
+
+func (t TimeLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimeLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewTimestampLogicalType returns a logical timestamp type with "forceConverted"
+// set to false
+func NewTimestampLogicalType(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: false,
+		fromConverted:  false,
+	}
+}
+
+// NewTimestampLogicalTypeForce returns a timestamp logical type with
+// "forceConverted" set to true
+func NewTimestampLogicalTypeForce(isAdjustedToUTC bool, unit TimeUnitType) LogicalType {
+	return &TimestampLogicalType{
+		typ: &format.TimestampType{
+			IsAdjustedToUTC: isAdjustedToUTC,
+			Unit:            createTimeUnit(unit),
+		},
+		forceConverted: true,
+		fromConverted:  false,
+	}
+}
+
+// TimestampLogicalType represents an int64 number that can be decoded
+// into a year, month, day, hour, minute, second, and subsecond
+type TimestampLogicalType struct {
+	baseLogicalType
+	typ            *format.TimestampType
+	forceConverted bool
+	fromConverted  bool
+}
+
+func (t TimestampLogicalType) IsFromConvertedType() bool {
+	return t.fromConverted
+}
+
+func (t TimestampLogicalType) IsAdjustedToUTC() bool {
+	return t.typ.IsAdjustedToUTC
+}
+
+func (t TimestampLogicalType) TimeUnit() TimeUnitType {
+	return timeUnitFromThrift(t.typ.Unit)
+}
+
+func (TimestampLogicalType) SortOrder() SortOrder {
+	return SortSIGNED
+}
+
+func (t TimestampLogicalType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(map[string]interface{}{
+		"Type":                     "Timestamp",
+		"isAdjustedToUTC":          t.typ.IsAdjustedToUTC,
+		"timeUnit":                 timeUnitToString(t.typ.GetUnit()),
+		"is_from_converted_type":   t.fromConverted,
+		"force_set_converted_type": t.forceConverted,
+	})
+}
+
+func (t TimestampLogicalType) IsSerialized() bool {
+	return !t.fromConverted
+}
+
+func (t TimestampLogicalType) String() string {
+	return fmt.Sprintf("Timestamp(isAdjustedToUTC=%t, timeUnit=%s, is_from_converted_type=%t, force_set_converted_type=%t)",
+		t.typ.GetIsAdjustedToUTC(), timeUnitToString(t.typ.GetUnit()), t.fromConverted, t.forceConverted)
+}
+
+func (t TimestampLogicalType) ToConvertedType() (ConvertedType, DecimalMetadata) {
+	unit := timeUnitFromThrift(t.typ.Unit)
+	if t.typ.IsAdjustedToUTC || t.forceConverted {
+		switch unit {
+		case TimeUnitMillis:
+			return ConvertedTypes.TimestampMillis, DecimalMetadata{}
+		case TimeUnitMicros:
+			return ConvertedTypes.TimestampMicros, DecimalMetadata{}
+		}
+	}
+	return ConvertedTypes.None, DecimalMetadata{}
+}
+
+func (t TimestampLogicalType) IsCompatible(c ConvertedType, dec DecimalMetadata) bool {
+	if dec.IsSet {
+		return false
+	}
+
+	switch timeUnitFromThrift(t.typ.Unit) {
+	case TimeUnitMillis:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMillis
+		}
+	case TimeUnitMicros:
+		if t.typ.GetIsAdjustedToUTC() || t.forceConverted {
+			return c == ConvertedTypes.TimestampMicros
+		}
+	}
+
+	return c == ConvertedTypes.None || c == ConvertedTypes.NA
+}
+
+func (TimestampLogicalType) IsApplicable(t parquet.Type, _ int32) bool {
+	return t == parquet.Types.Int64
+}
+
+func (t TimestampLogicalType) toThrift() *format.LogicalType {
+	return &format.LogicalType{TIMESTAMP: t.typ}
+}
+
+func (t TimestampLogicalType) Equals(rhs LogicalType) bool {
+	other, ok := rhs.(*TimestampLogicalType)
+	if !ok {
+		return false
+	}
+	return t.typ.IsAdjustedToUTC == other.typ.IsAdjustedToUTC &&
+		timeUnitFromThrift(t.typ.Unit) == timeUnitFromThrift(other.typ.Unit)
+}
+
+// NewIntLogicalType creates an integer logical type of the desired bitwidth
+// and whether it is signed or not.
+//
+// Bit width must be exactly 8, 16, 32 or 64 for an integer logical type
+func NewIntLogicalType(bitWidth int8, signed bool) LogicalType {
+	switch bitWidth {
+	case 8, 16, 32, 64:

Review comment:
       go automatically doesn't require break statements?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] westonpace commented on a change in pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#discussion_r634031679



##########
File path: go/parquet/schema/reflection.go
##########
@@ -0,0 +1,827 @@
+// 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.
+
+package schema
+
+import (
+	"reflect"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"golang.org/x/xerrors"
+)
+
+type taggedInfo struct {
+	Name string
+
+	Type      parquet.Type
+	KeyType   parquet.Type
+	ValueType parquet.Type
+
+	Length      int32
+	KeyLength   int32
+	ValueLength int32
+
+	Scale      int32
+	KeyScale   int32
+	ValueScale int32
+
+	Precision      int32
+	KeyPrecision   int32
+	ValuePrecision int32
+
+	FieldID      int32
+	KeyFieldID   int32
+	ValueFieldID int32
+
+	RepetitionType  parquet.Repetition
+	ValueRepetition parquet.Repetition
+
+	Converted      ConvertedType
+	KeyConverted   ConvertedType
+	ValueConverted ConvertedType
+
+	LogicalFields      map[string]string
+	KeyLogicalFields   map[string]string
+	ValueLogicalFields map[string]string
+
+	LogicalType      LogicalType
+	KeyLogicalType   LogicalType
+	ValueLogicalType LogicalType
+}
+
+func (t *taggedInfo) CopyForKey() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.KeyType
+	ret.Length = t.KeyLength
+	ret.Scale = t.KeyScale
+	ret.Precision = t.KeyPrecision
+	ret.FieldID = t.KeyFieldID
+	ret.RepetitionType = parquet.Repetitions.Required
+	ret.Converted = t.KeyConverted
+	ret.LogicalType = t.KeyLogicalType
+	return
+}
+
+func (t *taggedInfo) CopyForValue() (ret taggedInfo) {
+	ret = *t
+	ret.Type = t.ValueType
+	ret.Length = t.ValueLength
+	ret.Scale = t.ValueScale
+	ret.Precision = t.ValuePrecision
+	ret.FieldID = t.ValueFieldID
+	ret.RepetitionType = t.ValueRepetition
+	ret.Converted = t.ValueConverted
+	ret.LogicalType = t.ValueLogicalType
+	return
+}
+
+func (t *taggedInfo) UpdateLogicalTypes() {
+	processLogicalType := func(fields map[string]string, precision, scale int32) LogicalType {
+		t, ok := fields["type"]
+		if !ok {
+			return NoLogicalType{}
+		}
+
+		switch strings.ToLower(t) {
+		case "string":
+			return StringLogicalType{}
+		case "map":
+			return MapLogicalType{}
+		case "list":
+			return ListLogicalType{}
+		case "enum":
+			return EnumLogicalType{}
+		case "decimal":
+			if v, ok := fields["precision"]; ok {
+				precision = int32FromType(v)
+			}
+			if v, ok := fields["scale"]; ok {
+				scale = int32FromType(v)
+			}
+			return NewDecimalLogicalType(precision, scale)
+		case "date":
+			return DateLogicalType{}
+		case "time":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimeLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(strings.ToLower(unit)))
+		case "timestamp":
+			unit, ok := fields["unit"]
+			if !ok {
+				panic("must specify unit for time logical type")
+			}
+			adjustedToUtc, ok := fields["isadjustedutc"]
+			if !ok {
+				adjustedToUtc = "true"
+			}
+			return NewTimestampLogicalType(boolFromStr(adjustedToUtc), timeUnitFromString(unit))
+		case "integer":
+			width, ok := fields["bitwidth"]
+			if !ok {
+				panic("must specify bitwidth if explicitly setting integer logical type")
+			}
+			signed, ok := fields["signed"]
+			if !ok {
+				signed = "true"
+			}
+
+			return NewIntLogicalType(int8(int32FromType(width)), boolFromStr(signed))
+		case "null":
+			return NullLogicalType{}
+		case "json":
+			return JSONLogicalType{}
+		case "bson":
+			return BSONLogicalType{}
+		case "uuid":
+			return UUIDLogicalType{}
+		default:
+			panic(xerrors.Errorf("invalid logical type specified: %s", t))
+		}
+	}
+
+	t.LogicalType = processLogicalType(t.LogicalFields, t.Precision, t.Scale)
+	t.KeyLogicalType = processLogicalType(t.KeyLogicalFields, t.KeyPrecision, t.KeyScale)
+	t.ValueLogicalType = processLogicalType(t.ValueLogicalFields, t.ValuePrecision, t.ValueScale)
+}
+
+func newTaggedInfo() taggedInfo {
+	return taggedInfo{
+		Type:               parquet.Types.Undefined,
+		KeyType:            parquet.Types.Undefined,
+		ValueType:          parquet.Types.Undefined,
+		RepetitionType:     parquet.Repetitions.Undefined,
+		ValueRepetition:    parquet.Repetitions.Undefined,
+		Converted:          ConvertedTypes.NA,
+		KeyConverted:       ConvertedTypes.NA,
+		ValueConverted:     ConvertedTypes.NA,
+		FieldID:            -1,
+		KeyFieldID:         -1,
+		ValueFieldID:       -1,
+		LogicalFields:      make(map[string]string),
+		KeyLogicalFields:   make(map[string]string),
+		ValueLogicalFields: make(map[string]string),
+		LogicalType:        NoLogicalType{},
+		KeyLogicalType:     NoLogicalType{},
+		ValueLogicalType:   NoLogicalType{},
+	}
+}
+
+var int32FromType = func(v string) int32 {
+	val, err := strconv.Atoi(v)
+	if err != nil {
+		panic(err)
+	}
+	return int32(val)
+}
+
+var boolFromStr = func(v string) bool {
+	val, err := strconv.ParseBool(v)
+	if err != nil {
+		panic(err)
+	}
+	return val
+}
+
+func infoFromTags(f reflect.StructTag) *taggedInfo {
+	typeFromStr := func(v string) parquet.Type {
+		t, err := format.TypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(xerrors.Errorf("invalid type specified: %s", v))
+		}
+		return parquet.Type(t)
+	}
+
+	repFromStr := func(v string) parquet.Repetition {
+		r, err := format.FieldRepetitionTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return parquet.Repetition(r)
+	}
+
+	convertedFromStr := func(v string) ConvertedType {
+		c, err := format.ConvertedTypeFromString(strings.ToUpper(v))
+		if err != nil {
+			panic(err)
+		}
+		return ConvertedType(c)
+	}
+
+	if ptags, ok := f.Lookup("parquet"); ok {
+		info := newTaggedInfo()
+		for _, tag := range strings.Split(strings.Replace(ptags, "\t", "", -1), ",") {
+			tag = strings.TrimSpace(tag)
+			kv := strings.SplitN(tag, "=", 2)
+			key := strings.TrimSpace(strings.ToLower(kv[0]))
+			value := strings.TrimSpace(kv[1])
+
+			switch key {
+			case "name":
+				info.Name = value
+			case "type":
+				info.Type = typeFromStr(value)
+			case "keytype":
+				info.KeyType = typeFromStr(value)
+			case "valuetype":
+				info.ValueType = typeFromStr(value)
+			case "length":
+				info.Length = int32FromType(value)
+			case "keylength":
+				info.KeyLength = int32FromType(value)
+			case "valuelength":
+				info.ValueLength = int32FromType(value)
+			case "scale":
+				info.Scale = int32FromType(value)
+			case "keyscale":
+				info.KeyScale = int32FromType(value)
+			case "valuescale":
+				info.ValueScale = int32FromType(value)
+			case "precision":
+				info.Precision = int32FromType(value)
+			case "keyprecision":
+				info.KeyPrecision = int32FromType(value)
+			case "valueprecision":
+				info.ValuePrecision = int32FromType(value)
+			case "fieldid":
+				info.FieldID = int32FromType(value)
+			case "keyfieldid":
+				info.KeyFieldID = int32FromType(value)
+			case "valuefieldid":
+				info.ValueFieldID = int32FromType(value)
+			case "repetition":
+				info.RepetitionType = repFromStr(value)
+			case "valuerepetition":
+				info.ValueRepetition = repFromStr(value)
+			case "converted":
+				info.Converted = convertedFromStr(value)
+			case "keyconverted":
+				info.KeyConverted = convertedFromStr(value)
+			case "valueconverted":
+				info.ValueConverted = convertedFromStr(value)
+			case "logical":
+				info.LogicalFields["type"] = value
+			case "keylogical":
+				info.KeyLogicalFields["type"] = value
+			case "valuelogical":
+				info.ValueLogicalFields["type"] = value
+			default:
+				switch {
+				case strings.HasPrefix(key, "logical."):
+					info.LogicalFields[strings.TrimPrefix(key, "logical.")] = value
+				case strings.HasPrefix(key, "keylogical."):
+					info.KeyLogicalFields[strings.TrimPrefix(key, "keylogical.")] = value
+				case strings.HasPrefix(key, "valuelogical."):
+					info.ValueLogicalFields[strings.TrimPrefix(key, "valuelogical.")] = value
+				}
+			}
+		}
+		info.UpdateLogicalTypes()
+		return &info
+	}
+	return nil
+}
+
+// typeToNode recurseively converts a physical type and the tag info into parquet Nodes
+//
+// to avoid having to propagate errors up potentially high numbers of recursive calls
+// we use panics and then recover in the public function NewSchemaFromStruct so that a
+// failure very far down the stack quickly unwinds.
+func typeToNode(name string, typ reflect.Type, repType parquet.Repetition, info *taggedInfo) Node {
+	// set up our default values for everything
+	var (
+		converted             = ConvertedTypes.None
+		logical   LogicalType = NoLogicalType{}
+		fieldID               = int32(-1)
+		physical              = parquet.Types.Undefined
+		typeLen               = 0
+		precision             = 0
+		scale                 = 0
+	)
+	if info != nil { // we have struct tag info to process
+		fieldID = info.FieldID

Review comment:
       Yes.  No specific reason outside of reducing complexity/noise.  Arrow simply does not have enough information to generate anything meaningful here (the current "field order" assignment is not really informative and could be easily regenerated by a user if they truly needed it).
   
   Instead the C++ implementation will just pass through the value to/from the parquet layer.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [arrow] zeroshade commented on pull request #10071: ARROW-12424: [Go][Parquet] Adding Schema Package for Go Parquet

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10071:
URL: https://github.com/apache/arrow/pull/10071#issuecomment-829327162


   bump 😄 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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