zeroshade commented on code in PR #524:
URL: https://github.com/apache/iceberg-go/pull/524#discussion_r2255227725
##########
schema_conversions.go:
##########
@@ -30,33 +30,35 @@ func partitionTypeToAvroSchema(t *StructType) (avro.Schema,
error) {
var sc avro.Schema
switch typ := f.Type.(type) {
case Int32Type:
- sc = internal.IntSchema
+ sc = internal.NullableSchema(internal.IntSchema)
Review Comment:
why this change?
##########
table/partitioned_fanout_writer.go:
##########
@@ -0,0 +1,309 @@
+// 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 table
+
+import (
+ "context"
+ "fmt"
+ "iter"
+
+ "github.com/apache/arrow-go/v18/arrow"
+ "github.com/apache/arrow-go/v18/arrow/array"
+ "github.com/apache/arrow-go/v18/arrow/compute"
+ "github.com/apache/iceberg-go"
+ "golang.org/x/sync/errgroup"
+)
+
+// PartitionedFanoutWriter distributes Arrow records across multiple
partitions based on
+// a partition specification, writing data to separate files for each
partition using
+// a fanout pattern with configurable parallelism.
+type PartitionedFanoutWriter struct {
+ partitionSpec iceberg.PartitionSpec
+ schema *iceberg.Schema
+ itr iter.Seq2[arrow.Record, error]
+ writers *WriterFactory
+}
+
+// PartitionInfo holds the row indices and partition values for a specific
partition,
+// used during the fanout process to group rows by their partition key.
+type PartitionInfo struct {
+ rows []int64
+ partitionValues map[int]any
+}
+
+// NewPartitionedFanoutWriter creates a new PartitionedFanoutWriter with the
specified
+// partition specification, schema, and record iterator.
+func NewPartitionedFanoutWriter(partitionSpec iceberg.PartitionSpec, schema
*iceberg.Schema, itr iter.Seq2[arrow.Record, error]) *PartitionedFanoutWriter {
+ return &PartitionedFanoutWriter{
+ partitionSpec: partitionSpec,
+ schema: schema,
+ itr: itr,
+ }
+}
+
+func (p *PartitionedFanoutWriter) partitionPath(data partitionRecord) string {
+ return p.partitionSpec.PartitionToPath(data, p.schema)
+}
+
+// Write writes the Arrow records to the specified location using a fanout
pattern with
+// the specified number of workers. The returned iterator yields the data
files written
+// by the fanout process.
+func (p *PartitionedFanoutWriter) Write(ctx context.Context, workers int)
iter.Seq2[iceberg.DataFile, error] {
+ inputRecordsCh := make(chan arrow.Record, workers)
+ outputDataFilesCh := make(chan iceberg.DataFile, workers)
+
+ fanoutWorkers, ctx := errgroup.WithContext(ctx)
+ if err := p.startRecordFeeder(ctx, fanoutWorkers, inputRecordsCh); err
!= nil {
+ return func(yield func(iceberg.DataFile, error) bool) {
+ yield(nil, fmt.Errorf("failed to start record feeder:
%w", err))
+ }
+ }
+
+ for i := 0; i < workers; i++ {
Review Comment:
`for i := range workers`
##########
table/partitioned_fanout_writer.go:
##########
@@ -0,0 +1,309 @@
+// 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 table
+
+import (
+ "context"
+ "fmt"
+ "iter"
+
+ "github.com/apache/arrow-go/v18/arrow"
+ "github.com/apache/arrow-go/v18/arrow/array"
+ "github.com/apache/arrow-go/v18/arrow/compute"
+ "github.com/apache/iceberg-go"
+ "golang.org/x/sync/errgroup"
+)
+
+// PartitionedFanoutWriter distributes Arrow records across multiple
partitions based on
+// a partition specification, writing data to separate files for each
partition using
+// a fanout pattern with configurable parallelism.
+type PartitionedFanoutWriter struct {
+ partitionSpec iceberg.PartitionSpec
+ schema *iceberg.Schema
+ itr iter.Seq2[arrow.Record, error]
+ writers *WriterFactory
+}
+
+// PartitionInfo holds the row indices and partition values for a specific
partition,
+// used during the fanout process to group rows by their partition key.
+type PartitionInfo struct {
+ rows []int64
+ partitionValues map[int]any
+}
+
+// NewPartitionedFanoutWriter creates a new PartitionedFanoutWriter with the
specified
+// partition specification, schema, and record iterator.
+func NewPartitionedFanoutWriter(partitionSpec iceberg.PartitionSpec, schema
*iceberg.Schema, itr iter.Seq2[arrow.Record, error]) *PartitionedFanoutWriter {
+ return &PartitionedFanoutWriter{
+ partitionSpec: partitionSpec,
+ schema: schema,
+ itr: itr,
+ }
+}
+
+func (p *PartitionedFanoutWriter) partitionPath(data partitionRecord) string {
+ return p.partitionSpec.PartitionToPath(data, p.schema)
+}
+
+// Write writes the Arrow records to the specified location using a fanout
pattern with
+// the specified number of workers. The returned iterator yields the data
files written
+// by the fanout process.
+func (p *PartitionedFanoutWriter) Write(ctx context.Context, workers int)
iter.Seq2[iceberg.DataFile, error] {
+ inputRecordsCh := make(chan arrow.Record, workers)
+ outputDataFilesCh := make(chan iceberg.DataFile, workers)
+
+ fanoutWorkers, ctx := errgroup.WithContext(ctx)
+ if err := p.startRecordFeeder(ctx, fanoutWorkers, inputRecordsCh); err
!= nil {
+ return func(yield func(iceberg.DataFile, error) bool) {
+ yield(nil, fmt.Errorf("failed to start record feeder:
%w", err))
+ }
+ }
+
+ for i := 0; i < workers; i++ {
+ fanoutWorkers.Go(func() error {
+ return p.fanout(ctx, inputRecordsCh, outputDataFilesCh)
+ })
+ }
+
+ return p.yieldDataFiles(ctx, fanoutWorkers, outputDataFilesCh)
+}
+
+func (p *PartitionedFanoutWriter) startRecordFeeder(ctx context.Context,
fanoutWorkers *errgroup.Group, inputRecordsCh chan<- arrow.Record) error {
+ fanoutWorkers.Go(func() error {
+ defer close(inputRecordsCh)
+
+ for record, err := range p.itr {
+ if err != nil {
+ return err
+ }
+
+ select {
+ case <-ctx.Done():
+ record.Release()
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case inputRecordsCh <- record:
+ }
+ }
+ return nil
+ })
+
+ return nil
+}
+
+func (p *PartitionedFanoutWriter) fanout(ctx context.Context, inputRecordsCh
<-chan arrow.Record, dataFilesChannel chan<- iceberg.DataFile) error {
+ for {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case record, ok := <-inputRecordsCh:
+ if !ok {
+ return nil
+ }
+ defer record.Release()
+
+ partitionMap, err := p.getPartitionMap(record)
+ if err != nil {
+ return err
+ }
+
+ for partition, val := range partitionMap {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err !=
nil {
+ return err
+ }
+ return nil
+ default:
+ }
+
+ partitionRecord, err :=
partitionBatchByKey(ctx)(record, val.rows)
+ if err != nil {
+ return err
+ }
+
+ rollingDataWriter, err :=
p.writers.getOrCreateRollingDataWriter(partition, val.partitionValues)
+ if err != nil {
+ return err
+ }
+
+ err = rollingDataWriter.Add(ctx,
partitionRecord, dataFilesChannel)
+ if err != nil {
+ return err
+ }
+ }
+ }
+ }
+}
+
+func (p *PartitionedFanoutWriter) yieldDataFiles(ctx context.Context,
fanoutWorkers *errgroup.Group, outputDataFilesCh chan iceberg.DataFile)
iter.Seq2[iceberg.DataFile, error] {
+ return iter.Seq2[iceberg.DataFile, error](func(yield
func(iceberg.DataFile, error) bool) {
+ var dataFiles []iceberg.DataFile
+
+ done := make(chan struct{})
+ bufferDone := make(chan struct{})
+
+ var waitErr, closeErr error
+
+ go func() {
+ defer close(bufferDone)
+ for dataFile := range outputDataFilesCh {
+ dataFiles = append(dataFiles, dataFile)
+ }
+ }()
+
+ go func() {
+ defer close(done)
+ defer close(outputDataFilesCh)
+ waitErr = fanoutWorkers.Wait()
+ closeErr = p.writers.closeAll(ctx, outputDataFilesCh)
+ }()
+
+ <-done
+ <-bufferDone
Review Comment:
you should handle this outside of the function you're returning. More
importantly, this is going to wait until ALL of the files are created rather
than yielding them as they are done, which isn't what you want.
Instead you should do something like:
```go
var err error
go func() {
defer close(outputDataFilesCh)
err = fanoutWorkers.Wait()
err = errors.Join(err, p.writers.closeAll(ctx))
}()
return func(yield func(iceberg.DataFile, error) bool) {
defer func() {
for range outputDataFilesCh {} // drain any remaining
output files
}()
for f := range outputDataFilesCh {
if !yield(f, nil) {
return
}
}
if err != nil {
yield(nil, err)
}
}
```
##########
table/partitioned_fanout_writer.go:
##########
@@ -0,0 +1,309 @@
+// 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 table
+
+import (
+ "context"
+ "fmt"
+ "iter"
+
+ "github.com/apache/arrow-go/v18/arrow"
+ "github.com/apache/arrow-go/v18/arrow/array"
+ "github.com/apache/arrow-go/v18/arrow/compute"
+ "github.com/apache/iceberg-go"
+ "golang.org/x/sync/errgroup"
+)
+
+// PartitionedFanoutWriter distributes Arrow records across multiple
partitions based on
+// a partition specification, writing data to separate files for each
partition using
+// a fanout pattern with configurable parallelism.
+type PartitionedFanoutWriter struct {
+ partitionSpec iceberg.PartitionSpec
+ schema *iceberg.Schema
+ itr iter.Seq2[arrow.Record, error]
+ writers *WriterFactory
+}
+
+// PartitionInfo holds the row indices and partition values for a specific
partition,
+// used during the fanout process to group rows by their partition key.
+type PartitionInfo struct {
+ rows []int64
+ partitionValues map[int]any
+}
+
+// NewPartitionedFanoutWriter creates a new PartitionedFanoutWriter with the
specified
+// partition specification, schema, and record iterator.
+func NewPartitionedFanoutWriter(partitionSpec iceberg.PartitionSpec, schema
*iceberg.Schema, itr iter.Seq2[arrow.Record, error]) *PartitionedFanoutWriter {
+ return &PartitionedFanoutWriter{
+ partitionSpec: partitionSpec,
+ schema: schema,
+ itr: itr,
+ }
+}
+
+func (p *PartitionedFanoutWriter) partitionPath(data partitionRecord) string {
+ return p.partitionSpec.PartitionToPath(data, p.schema)
+}
+
+// Write writes the Arrow records to the specified location using a fanout
pattern with
+// the specified number of workers. The returned iterator yields the data
files written
+// by the fanout process.
+func (p *PartitionedFanoutWriter) Write(ctx context.Context, workers int)
iter.Seq2[iceberg.DataFile, error] {
+ inputRecordsCh := make(chan arrow.Record, workers)
+ outputDataFilesCh := make(chan iceberg.DataFile, workers)
+
+ fanoutWorkers, ctx := errgroup.WithContext(ctx)
+ if err := p.startRecordFeeder(ctx, fanoutWorkers, inputRecordsCh); err
!= nil {
+ return func(yield func(iceberg.DataFile, error) bool) {
+ yield(nil, fmt.Errorf("failed to start record feeder:
%w", err))
+ }
+ }
+
+ for i := 0; i < workers; i++ {
+ fanoutWorkers.Go(func() error {
+ return p.fanout(ctx, inputRecordsCh, outputDataFilesCh)
+ })
+ }
+
+ return p.yieldDataFiles(ctx, fanoutWorkers, outputDataFilesCh)
+}
+
+func (p *PartitionedFanoutWriter) startRecordFeeder(ctx context.Context,
fanoutWorkers *errgroup.Group, inputRecordsCh chan<- arrow.Record) error {
+ fanoutWorkers.Go(func() error {
+ defer close(inputRecordsCh)
+
+ for record, err := range p.itr {
+ if err != nil {
+ return err
+ }
+
+ select {
+ case <-ctx.Done():
+ record.Release()
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case inputRecordsCh <- record:
+ }
+ }
+ return nil
+ })
+
+ return nil
+}
+
+func (p *PartitionedFanoutWriter) fanout(ctx context.Context, inputRecordsCh
<-chan arrow.Record, dataFilesChannel chan<- iceberg.DataFile) error {
+ for {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case record, ok := <-inputRecordsCh:
+ if !ok {
+ return nil
+ }
+ defer record.Release()
+
+ partitionMap, err := p.getPartitionMap(record)
+ if err != nil {
+ return err
+ }
+
+ for partition, val := range partitionMap {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err !=
nil {
+ return err
+ }
+ return nil
+ default:
+ }
+
+ partitionRecord, err :=
partitionBatchByKey(ctx)(record, val.rows)
+ if err != nil {
+ return err
+ }
+
+ rollingDataWriter, err :=
p.writers.getOrCreateRollingDataWriter(partition, val.partitionValues)
+ if err != nil {
+ return err
+ }
+
+ err = rollingDataWriter.Add(ctx,
partitionRecord, dataFilesChannel)
+ if err != nil {
+ return err
+ }
+ }
+ }
+ }
+}
+
+func (p *PartitionedFanoutWriter) yieldDataFiles(ctx context.Context,
fanoutWorkers *errgroup.Group, outputDataFilesCh chan iceberg.DataFile)
iter.Seq2[iceberg.DataFile, error] {
+ return iter.Seq2[iceberg.DataFile, error](func(yield
func(iceberg.DataFile, error) bool) {
Review Comment:
you don't need to wrap it with an explicit `iter.Seq2[iceberg.DataFile,
error](` etc. It'll automatically handle that as long as you're returning the
correct function signature
##########
table/partitioned_fanout_writer.go:
##########
@@ -0,0 +1,309 @@
+// 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 table
+
+import (
+ "context"
+ "fmt"
+ "iter"
+
+ "github.com/apache/arrow-go/v18/arrow"
+ "github.com/apache/arrow-go/v18/arrow/array"
+ "github.com/apache/arrow-go/v18/arrow/compute"
+ "github.com/apache/iceberg-go"
+ "golang.org/x/sync/errgroup"
+)
+
+// PartitionedFanoutWriter distributes Arrow records across multiple
partitions based on
+// a partition specification, writing data to separate files for each
partition using
+// a fanout pattern with configurable parallelism.
+type PartitionedFanoutWriter struct {
+ partitionSpec iceberg.PartitionSpec
+ schema *iceberg.Schema
+ itr iter.Seq2[arrow.Record, error]
+ writers *WriterFactory
+}
+
+// PartitionInfo holds the row indices and partition values for a specific
partition,
+// used during the fanout process to group rows by their partition key.
+type PartitionInfo struct {
+ rows []int64
+ partitionValues map[int]any
+}
+
+// NewPartitionedFanoutWriter creates a new PartitionedFanoutWriter with the
specified
+// partition specification, schema, and record iterator.
+func NewPartitionedFanoutWriter(partitionSpec iceberg.PartitionSpec, schema
*iceberg.Schema, itr iter.Seq2[arrow.Record, error]) *PartitionedFanoutWriter {
+ return &PartitionedFanoutWriter{
+ partitionSpec: partitionSpec,
+ schema: schema,
+ itr: itr,
+ }
+}
+
+func (p *PartitionedFanoutWriter) partitionPath(data partitionRecord) string {
+ return p.partitionSpec.PartitionToPath(data, p.schema)
+}
+
+// Write writes the Arrow records to the specified location using a fanout
pattern with
+// the specified number of workers. The returned iterator yields the data
files written
+// by the fanout process.
+func (p *PartitionedFanoutWriter) Write(ctx context.Context, workers int)
iter.Seq2[iceberg.DataFile, error] {
+ inputRecordsCh := make(chan arrow.Record, workers)
+ outputDataFilesCh := make(chan iceberg.DataFile, workers)
+
+ fanoutWorkers, ctx := errgroup.WithContext(ctx)
+ if err := p.startRecordFeeder(ctx, fanoutWorkers, inputRecordsCh); err
!= nil {
+ return func(yield func(iceberg.DataFile, error) bool) {
+ yield(nil, fmt.Errorf("failed to start record feeder:
%w", err))
+ }
+ }
+
+ for i := 0; i < workers; i++ {
+ fanoutWorkers.Go(func() error {
+ return p.fanout(ctx, inputRecordsCh, outputDataFilesCh)
+ })
+ }
+
+ return p.yieldDataFiles(ctx, fanoutWorkers, outputDataFilesCh)
+}
+
+func (p *PartitionedFanoutWriter) startRecordFeeder(ctx context.Context,
fanoutWorkers *errgroup.Group, inputRecordsCh chan<- arrow.Record) error {
+ fanoutWorkers.Go(func() error {
+ defer close(inputRecordsCh)
+
+ for record, err := range p.itr {
+ if err != nil {
+ return err
+ }
+
+ select {
+ case <-ctx.Done():
+ record.Release()
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case inputRecordsCh <- record:
+ }
+ }
+ return nil
+ })
+
+ return nil
+}
+
+func (p *PartitionedFanoutWriter) fanout(ctx context.Context, inputRecordsCh
<-chan arrow.Record, dataFilesChannel chan<- iceberg.DataFile) error {
+ for {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
Review Comment:
this can be reduced to `return context.Cause(ctx)`
##########
table/partitioned_fanout_writer.go:
##########
@@ -0,0 +1,309 @@
+// 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 table
+
+import (
+ "context"
+ "fmt"
+ "iter"
+
+ "github.com/apache/arrow-go/v18/arrow"
+ "github.com/apache/arrow-go/v18/arrow/array"
+ "github.com/apache/arrow-go/v18/arrow/compute"
+ "github.com/apache/iceberg-go"
+ "golang.org/x/sync/errgroup"
+)
+
+// PartitionedFanoutWriter distributes Arrow records across multiple
partitions based on
+// a partition specification, writing data to separate files for each
partition using
+// a fanout pattern with configurable parallelism.
+type PartitionedFanoutWriter struct {
+ partitionSpec iceberg.PartitionSpec
+ schema *iceberg.Schema
+ itr iter.Seq2[arrow.Record, error]
+ writers *WriterFactory
+}
+
+// PartitionInfo holds the row indices and partition values for a specific
partition,
+// used during the fanout process to group rows by their partition key.
+type PartitionInfo struct {
+ rows []int64
+ partitionValues map[int]any
+}
+
+// NewPartitionedFanoutWriter creates a new PartitionedFanoutWriter with the
specified
+// partition specification, schema, and record iterator.
+func NewPartitionedFanoutWriter(partitionSpec iceberg.PartitionSpec, schema
*iceberg.Schema, itr iter.Seq2[arrow.Record, error]) *PartitionedFanoutWriter {
+ return &PartitionedFanoutWriter{
+ partitionSpec: partitionSpec,
+ schema: schema,
+ itr: itr,
+ }
+}
+
+func (p *PartitionedFanoutWriter) partitionPath(data partitionRecord) string {
+ return p.partitionSpec.PartitionToPath(data, p.schema)
+}
+
+// Write writes the Arrow records to the specified location using a fanout
pattern with
+// the specified number of workers. The returned iterator yields the data
files written
+// by the fanout process.
+func (p *PartitionedFanoutWriter) Write(ctx context.Context, workers int)
iter.Seq2[iceberg.DataFile, error] {
+ inputRecordsCh := make(chan arrow.Record, workers)
+ outputDataFilesCh := make(chan iceberg.DataFile, workers)
+
+ fanoutWorkers, ctx := errgroup.WithContext(ctx)
+ if err := p.startRecordFeeder(ctx, fanoutWorkers, inputRecordsCh); err
!= nil {
+ return func(yield func(iceberg.DataFile, error) bool) {
+ yield(nil, fmt.Errorf("failed to start record feeder:
%w", err))
+ }
+ }
+
+ for i := 0; i < workers; i++ {
+ fanoutWorkers.Go(func() error {
+ return p.fanout(ctx, inputRecordsCh, outputDataFilesCh)
+ })
+ }
+
+ return p.yieldDataFiles(ctx, fanoutWorkers, outputDataFilesCh)
+}
+
+func (p *PartitionedFanoutWriter) startRecordFeeder(ctx context.Context,
fanoutWorkers *errgroup.Group, inputRecordsCh chan<- arrow.Record) error {
+ fanoutWorkers.Go(func() error {
+ defer close(inputRecordsCh)
+
+ for record, err := range p.itr {
+ if err != nil {
+ return err
+ }
+
+ select {
+ case <-ctx.Done():
+ record.Release()
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case inputRecordsCh <- record:
+ }
+ }
+ return nil
+ })
+
+ return nil
+}
+
+func (p *PartitionedFanoutWriter) fanout(ctx context.Context, inputRecordsCh
<-chan arrow.Record, dataFilesChannel chan<- iceberg.DataFile) error {
+ for {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case record, ok := <-inputRecordsCh:
+ if !ok {
+ return nil
+ }
+ defer record.Release()
+
+ partitionMap, err := p.getPartitionMap(record)
+ if err != nil {
+ return err
+ }
+
+ for partition, val := range partitionMap {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err !=
nil {
+ return err
+ }
+ return nil
+ default:
+ }
+
+ partitionRecord, err :=
partitionBatchByKey(ctx)(record, val.rows)
+ if err != nil {
+ return err
+ }
+
+ rollingDataWriter, err :=
p.writers.getOrCreateRollingDataWriter(partition, val.partitionValues)
+ if err != nil {
+ return err
+ }
+
+ err = rollingDataWriter.Add(ctx,
partitionRecord, dataFilesChannel)
+ if err != nil {
+ return err
+ }
+ }
+ }
+ }
+}
+
+func (p *PartitionedFanoutWriter) yieldDataFiles(ctx context.Context,
fanoutWorkers *errgroup.Group, outputDataFilesCh chan iceberg.DataFile)
iter.Seq2[iceberg.DataFile, error] {
+ return iter.Seq2[iceberg.DataFile, error](func(yield
func(iceberg.DataFile, error) bool) {
+ var dataFiles []iceberg.DataFile
+
+ done := make(chan struct{})
+ bufferDone := make(chan struct{})
+
+ var waitErr, closeErr error
+
+ go func() {
+ defer close(bufferDone)
+ for dataFile := range outputDataFilesCh {
+ dataFiles = append(dataFiles, dataFile)
+ }
+ }()
+
+ go func() {
+ defer close(done)
+ defer close(outputDataFilesCh)
+ waitErr = fanoutWorkers.Wait()
+ closeErr = p.writers.closeAll(ctx, outputDataFilesCh)
+ }()
+
+ <-done
+ <-bufferDone
+
+ if waitErr != nil {
+ _ = yield(nil, waitErr)
+ return
+ }
+
+ if closeErr != nil {
+ _ = yield(nil, closeErr)
+ return
+ }
+
+ for _, dataFile := range dataFiles {
+ if !yield(dataFile, nil) {
+ return
+ }
+ }
+ })
+}
+
+func (p *PartitionedFanoutWriter) getPartitionMap(record arrow.Record)
(map[string]PartitionInfo, error) {
+ partitionMap := make(map[string]PartitionInfo)
+ partitionFields := p.partitionSpec.PartitionType(p.schema).FieldList
+
+ for row := range record.NumRows() {
+ partitionRec := make(partitionRecord, len(partitionFields))
+ partitionValues := make(map[int]any)
+
+ for i := range partitionFields {
+ sourceField := p.partitionSpec.Field(i)
+ colName, _ :=
p.schema.FindColumnName(sourceField.SourceID)
+ colIdx := record.Schema().FieldIndices(colName)[0]
+ col := record.Column(colIdx)
Review Comment:
can we do this outside the row-loop and construct a mapping of
partitionField -> `arrow.Array` that we use for each iteration? It's wasteful
to do the column lookup for every column for every row.
##########
table/partitioned_fanout_writer.go:
##########
@@ -0,0 +1,309 @@
+// 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 table
+
+import (
+ "context"
+ "fmt"
+ "iter"
+
+ "github.com/apache/arrow-go/v18/arrow"
+ "github.com/apache/arrow-go/v18/arrow/array"
+ "github.com/apache/arrow-go/v18/arrow/compute"
+ "github.com/apache/iceberg-go"
+ "golang.org/x/sync/errgroup"
+)
+
+// PartitionedFanoutWriter distributes Arrow records across multiple
partitions based on
+// a partition specification, writing data to separate files for each
partition using
+// a fanout pattern with configurable parallelism.
+type PartitionedFanoutWriter struct {
+ partitionSpec iceberg.PartitionSpec
+ schema *iceberg.Schema
+ itr iter.Seq2[arrow.Record, error]
+ writers *WriterFactory
+}
+
+// PartitionInfo holds the row indices and partition values for a specific
partition,
+// used during the fanout process to group rows by their partition key.
+type PartitionInfo struct {
+ rows []int64
+ partitionValues map[int]any
+}
+
+// NewPartitionedFanoutWriter creates a new PartitionedFanoutWriter with the
specified
+// partition specification, schema, and record iterator.
+func NewPartitionedFanoutWriter(partitionSpec iceberg.PartitionSpec, schema
*iceberg.Schema, itr iter.Seq2[arrow.Record, error]) *PartitionedFanoutWriter {
+ return &PartitionedFanoutWriter{
+ partitionSpec: partitionSpec,
+ schema: schema,
+ itr: itr,
+ }
+}
+
+func (p *PartitionedFanoutWriter) partitionPath(data partitionRecord) string {
+ return p.partitionSpec.PartitionToPath(data, p.schema)
+}
+
+// Write writes the Arrow records to the specified location using a fanout
pattern with
+// the specified number of workers. The returned iterator yields the data
files written
+// by the fanout process.
+func (p *PartitionedFanoutWriter) Write(ctx context.Context, workers int)
iter.Seq2[iceberg.DataFile, error] {
+ inputRecordsCh := make(chan arrow.Record, workers)
+ outputDataFilesCh := make(chan iceberg.DataFile, workers)
+
+ fanoutWorkers, ctx := errgroup.WithContext(ctx)
+ if err := p.startRecordFeeder(ctx, fanoutWorkers, inputRecordsCh); err
!= nil {
+ return func(yield func(iceberg.DataFile, error) bool) {
+ yield(nil, fmt.Errorf("failed to start record feeder:
%w", err))
+ }
+ }
+
+ for i := 0; i < workers; i++ {
+ fanoutWorkers.Go(func() error {
+ return p.fanout(ctx, inputRecordsCh, outputDataFilesCh)
+ })
+ }
+
+ return p.yieldDataFiles(ctx, fanoutWorkers, outputDataFilesCh)
+}
+
+func (p *PartitionedFanoutWriter) startRecordFeeder(ctx context.Context,
fanoutWorkers *errgroup.Group, inputRecordsCh chan<- arrow.Record) error {
+ fanoutWorkers.Go(func() error {
+ defer close(inputRecordsCh)
+
+ for record, err := range p.itr {
+ if err != nil {
+ return err
+ }
+
+ select {
+ case <-ctx.Done():
+ record.Release()
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
Review Comment:
this reduces to `return context.Cause(ctx)`
##########
table/arrow_utils.go:
##########
@@ -1307,8 +1309,19 @@ func recordsToDataFiles(ctx context.Context,
rootLocation string, meta *Metadata
}
}
- return writeFiles(ctx, rootLocation, args.fs, meta, tasks)
- }
+ return writeFiles(ctx, rootLocation, args.fs, meta, nil, tasks)
+ } else {
+ partitionWriter :=
NewPartitionedFanoutWriter(meta.CurrentSpec(), meta.CurrentSchema(), args.itr)
+ rollingDataWriters := NewWriterFactory(rootLocation, args,
meta, taskSchema, targetFileSize)
+ rollingDataWriters.nextCount = nextCount
+ rollingDataWriters.stopCount = stopCount
+
+ partitionWriter.writers = &rollingDataWriters
+ workers := meta.props.GetInt(FanoutWriterWorkersKey,
FanoutWriterWorkersDefault)
+ if workers <= 0 {
+ workers = runtime.NumCPU()
+ }
Review Comment:
rather than a specific key for the `FanoutWriterWorkers` let's have a more
generic `max-workers` key that overrides the config value
##########
table/partitioned_fanout_writer.go:
##########
@@ -0,0 +1,309 @@
+// 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 table
+
+import (
+ "context"
+ "fmt"
+ "iter"
+
+ "github.com/apache/arrow-go/v18/arrow"
+ "github.com/apache/arrow-go/v18/arrow/array"
+ "github.com/apache/arrow-go/v18/arrow/compute"
+ "github.com/apache/iceberg-go"
+ "golang.org/x/sync/errgroup"
+)
+
+// PartitionedFanoutWriter distributes Arrow records across multiple
partitions based on
+// a partition specification, writing data to separate files for each
partition using
+// a fanout pattern with configurable parallelism.
+type PartitionedFanoutWriter struct {
+ partitionSpec iceberg.PartitionSpec
+ schema *iceberg.Schema
+ itr iter.Seq2[arrow.Record, error]
+ writers *WriterFactory
+}
+
+// PartitionInfo holds the row indices and partition values for a specific
partition,
+// used during the fanout process to group rows by their partition key.
+type PartitionInfo struct {
+ rows []int64
+ partitionValues map[int]any
+}
+
+// NewPartitionedFanoutWriter creates a new PartitionedFanoutWriter with the
specified
+// partition specification, schema, and record iterator.
+func NewPartitionedFanoutWriter(partitionSpec iceberg.PartitionSpec, schema
*iceberg.Schema, itr iter.Seq2[arrow.Record, error]) *PartitionedFanoutWriter {
+ return &PartitionedFanoutWriter{
+ partitionSpec: partitionSpec,
+ schema: schema,
+ itr: itr,
+ }
+}
+
+func (p *PartitionedFanoutWriter) partitionPath(data partitionRecord) string {
+ return p.partitionSpec.PartitionToPath(data, p.schema)
+}
+
+// Write writes the Arrow records to the specified location using a fanout
pattern with
+// the specified number of workers. The returned iterator yields the data
files written
+// by the fanout process.
+func (p *PartitionedFanoutWriter) Write(ctx context.Context, workers int)
iter.Seq2[iceberg.DataFile, error] {
+ inputRecordsCh := make(chan arrow.Record, workers)
+ outputDataFilesCh := make(chan iceberg.DataFile, workers)
+
+ fanoutWorkers, ctx := errgroup.WithContext(ctx)
+ if err := p.startRecordFeeder(ctx, fanoutWorkers, inputRecordsCh); err
!= nil {
+ return func(yield func(iceberg.DataFile, error) bool) {
+ yield(nil, fmt.Errorf("failed to start record feeder:
%w", err))
+ }
+ }
+
+ for i := 0; i < workers; i++ {
+ fanoutWorkers.Go(func() error {
+ return p.fanout(ctx, inputRecordsCh, outputDataFilesCh)
+ })
+ }
+
+ return p.yieldDataFiles(ctx, fanoutWorkers, outputDataFilesCh)
+}
+
+func (p *PartitionedFanoutWriter) startRecordFeeder(ctx context.Context,
fanoutWorkers *errgroup.Group, inputRecordsCh chan<- arrow.Record) error {
+ fanoutWorkers.Go(func() error {
+ defer close(inputRecordsCh)
+
+ for record, err := range p.itr {
+ if err != nil {
+ return err
+ }
+
+ select {
+ case <-ctx.Done():
+ record.Release()
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case inputRecordsCh <- record:
+ }
+ }
+ return nil
+ })
+
+ return nil
+}
+
+func (p *PartitionedFanoutWriter) fanout(ctx context.Context, inputRecordsCh
<-chan arrow.Record, dataFilesChannel chan<- iceberg.DataFile) error {
+ for {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case record, ok := <-inputRecordsCh:
+ if !ok {
+ return nil
+ }
+ defer record.Release()
+
+ partitionMap, err := p.getPartitionMap(record)
+ if err != nil {
+ return err
+ }
+
+ for partition, val := range partitionMap {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err !=
nil {
+ return err
+ }
+ return nil
+ default:
Review Comment:
same as above, reduces to `return context.Cause(ctx)`
##########
table/partitioned_fanout_writer.go:
##########
@@ -0,0 +1,309 @@
+// 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 table
+
+import (
+ "context"
+ "fmt"
+ "iter"
+
+ "github.com/apache/arrow-go/v18/arrow"
+ "github.com/apache/arrow-go/v18/arrow/array"
+ "github.com/apache/arrow-go/v18/arrow/compute"
+ "github.com/apache/iceberg-go"
+ "golang.org/x/sync/errgroup"
+)
+
+// PartitionedFanoutWriter distributes Arrow records across multiple
partitions based on
+// a partition specification, writing data to separate files for each
partition using
+// a fanout pattern with configurable parallelism.
+type PartitionedFanoutWriter struct {
+ partitionSpec iceberg.PartitionSpec
+ schema *iceberg.Schema
+ itr iter.Seq2[arrow.Record, error]
+ writers *WriterFactory
+}
+
+// PartitionInfo holds the row indices and partition values for a specific
partition,
+// used during the fanout process to group rows by their partition key.
+type PartitionInfo struct {
+ rows []int64
+ partitionValues map[int]any
+}
+
+// NewPartitionedFanoutWriter creates a new PartitionedFanoutWriter with the
specified
+// partition specification, schema, and record iterator.
+func NewPartitionedFanoutWriter(partitionSpec iceberg.PartitionSpec, schema
*iceberg.Schema, itr iter.Seq2[arrow.Record, error]) *PartitionedFanoutWriter {
+ return &PartitionedFanoutWriter{
+ partitionSpec: partitionSpec,
+ schema: schema,
+ itr: itr,
+ }
+}
+
+func (p *PartitionedFanoutWriter) partitionPath(data partitionRecord) string {
+ return p.partitionSpec.PartitionToPath(data, p.schema)
+}
+
+// Write writes the Arrow records to the specified location using a fanout
pattern with
+// the specified number of workers. The returned iterator yields the data
files written
+// by the fanout process.
+func (p *PartitionedFanoutWriter) Write(ctx context.Context, workers int)
iter.Seq2[iceberg.DataFile, error] {
+ inputRecordsCh := make(chan arrow.Record, workers)
+ outputDataFilesCh := make(chan iceberg.DataFile, workers)
+
+ fanoutWorkers, ctx := errgroup.WithContext(ctx)
+ if err := p.startRecordFeeder(ctx, fanoutWorkers, inputRecordsCh); err
!= nil {
+ return func(yield func(iceberg.DataFile, error) bool) {
+ yield(nil, fmt.Errorf("failed to start record feeder:
%w", err))
+ }
+ }
+
+ for i := 0; i < workers; i++ {
+ fanoutWorkers.Go(func() error {
+ return p.fanout(ctx, inputRecordsCh, outputDataFilesCh)
+ })
+ }
+
+ return p.yieldDataFiles(ctx, fanoutWorkers, outputDataFilesCh)
+}
+
+func (p *PartitionedFanoutWriter) startRecordFeeder(ctx context.Context,
fanoutWorkers *errgroup.Group, inputRecordsCh chan<- arrow.Record) error {
+ fanoutWorkers.Go(func() error {
+ defer close(inputRecordsCh)
+
+ for record, err := range p.itr {
+ if err != nil {
+ return err
+ }
+
+ select {
+ case <-ctx.Done():
+ record.Release()
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case inputRecordsCh <- record:
+ }
+ }
+ return nil
+ })
+
+ return nil
Review Comment:
this is guaranteed to return nil, why do we even have it return an error at
all?
##########
table/partitioned_fanout_writer.go:
##########
@@ -0,0 +1,309 @@
+// 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 table
+
+import (
+ "context"
+ "fmt"
+ "iter"
+
+ "github.com/apache/arrow-go/v18/arrow"
+ "github.com/apache/arrow-go/v18/arrow/array"
+ "github.com/apache/arrow-go/v18/arrow/compute"
+ "github.com/apache/iceberg-go"
+ "golang.org/x/sync/errgroup"
+)
+
+// PartitionedFanoutWriter distributes Arrow records across multiple
partitions based on
+// a partition specification, writing data to separate files for each
partition using
+// a fanout pattern with configurable parallelism.
+type PartitionedFanoutWriter struct {
+ partitionSpec iceberg.PartitionSpec
+ schema *iceberg.Schema
+ itr iter.Seq2[arrow.Record, error]
+ writers *WriterFactory
+}
+
+// PartitionInfo holds the row indices and partition values for a specific
partition,
+// used during the fanout process to group rows by their partition key.
+type PartitionInfo struct {
+ rows []int64
+ partitionValues map[int]any
+}
+
+// NewPartitionedFanoutWriter creates a new PartitionedFanoutWriter with the
specified
+// partition specification, schema, and record iterator.
+func NewPartitionedFanoutWriter(partitionSpec iceberg.PartitionSpec, schema
*iceberg.Schema, itr iter.Seq2[arrow.Record, error]) *PartitionedFanoutWriter {
+ return &PartitionedFanoutWriter{
+ partitionSpec: partitionSpec,
+ schema: schema,
+ itr: itr,
+ }
+}
+
+func (p *PartitionedFanoutWriter) partitionPath(data partitionRecord) string {
+ return p.partitionSpec.PartitionToPath(data, p.schema)
+}
+
+// Write writes the Arrow records to the specified location using a fanout
pattern with
+// the specified number of workers. The returned iterator yields the data
files written
+// by the fanout process.
+func (p *PartitionedFanoutWriter) Write(ctx context.Context, workers int)
iter.Seq2[iceberg.DataFile, error] {
+ inputRecordsCh := make(chan arrow.Record, workers)
+ outputDataFilesCh := make(chan iceberg.DataFile, workers)
+
+ fanoutWorkers, ctx := errgroup.WithContext(ctx)
+ if err := p.startRecordFeeder(ctx, fanoutWorkers, inputRecordsCh); err
!= nil {
+ return func(yield func(iceberg.DataFile, error) bool) {
+ yield(nil, fmt.Errorf("failed to start record feeder:
%w", err))
+ }
+ }
+
+ for i := 0; i < workers; i++ {
+ fanoutWorkers.Go(func() error {
+ return p.fanout(ctx, inputRecordsCh, outputDataFilesCh)
+ })
+ }
+
+ return p.yieldDataFiles(ctx, fanoutWorkers, outputDataFilesCh)
+}
+
+func (p *PartitionedFanoutWriter) startRecordFeeder(ctx context.Context,
fanoutWorkers *errgroup.Group, inputRecordsCh chan<- arrow.Record) error {
+ fanoutWorkers.Go(func() error {
+ defer close(inputRecordsCh)
+
+ for record, err := range p.itr {
+ if err != nil {
+ return err
+ }
+
+ select {
+ case <-ctx.Done():
+ record.Release()
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case inputRecordsCh <- record:
+ }
+ }
+ return nil
+ })
+
+ return nil
+}
+
+func (p *PartitionedFanoutWriter) fanout(ctx context.Context, inputRecordsCh
<-chan arrow.Record, dataFilesChannel chan<- iceberg.DataFile) error {
+ for {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case record, ok := <-inputRecordsCh:
+ if !ok {
+ return nil
+ }
+ defer record.Release()
+
+ partitionMap, err := p.getPartitionMap(record)
+ if err != nil {
+ return err
+ }
+
+ for partition, val := range partitionMap {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err !=
nil {
+ return err
+ }
+ return nil
+ default:
+ }
+
+ partitionRecord, err :=
partitionBatchByKey(ctx)(record, val.rows)
+ if err != nil {
+ return err
+ }
+
+ rollingDataWriter, err :=
p.writers.getOrCreateRollingDataWriter(partition, val.partitionValues)
+ if err != nil {
+ return err
+ }
+
+ err = rollingDataWriter.Add(ctx,
partitionRecord, dataFilesChannel)
+ if err != nil {
+ return err
+ }
+ }
+ }
+ }
+}
+
+func (p *PartitionedFanoutWriter) yieldDataFiles(ctx context.Context,
fanoutWorkers *errgroup.Group, outputDataFilesCh chan iceberg.DataFile)
iter.Seq2[iceberg.DataFile, error] {
+ return iter.Seq2[iceberg.DataFile, error](func(yield
func(iceberg.DataFile, error) bool) {
+ var dataFiles []iceberg.DataFile
+
+ done := make(chan struct{})
+ bufferDone := make(chan struct{})
+
+ var waitErr, closeErr error
+
+ go func() {
+ defer close(bufferDone)
+ for dataFile := range outputDataFilesCh {
+ dataFiles = append(dataFiles, dataFile)
+ }
+ }()
+
+ go func() {
+ defer close(done)
+ defer close(outputDataFilesCh)
+ waitErr = fanoutWorkers.Wait()
+ closeErr = p.writers.closeAll(ctx, outputDataFilesCh)
+ }()
+
+ <-done
+ <-bufferDone
+
+ if waitErr != nil {
+ _ = yield(nil, waitErr)
+ return
+ }
+
+ if closeErr != nil {
+ _ = yield(nil, closeErr)
+ return
+ }
+
+ for _, dataFile := range dataFiles {
+ if !yield(dataFile, nil) {
+ return
+ }
+ }
+ })
+}
+
+func (p *PartitionedFanoutWriter) getPartitionMap(record arrow.Record)
(map[string]PartitionInfo, error) {
+ partitionMap := make(map[string]PartitionInfo)
+ partitionFields := p.partitionSpec.PartitionType(p.schema).FieldList
+
+ for row := range record.NumRows() {
+ partitionRec := make(partitionRecord, len(partitionFields))
Review Comment:
shift this outside the loop and reuse it?
##########
table/partitioned_fanout_writer.go:
##########
@@ -0,0 +1,309 @@
+// 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 table
+
+import (
+ "context"
+ "fmt"
+ "iter"
+
+ "github.com/apache/arrow-go/v18/arrow"
+ "github.com/apache/arrow-go/v18/arrow/array"
+ "github.com/apache/arrow-go/v18/arrow/compute"
+ "github.com/apache/iceberg-go"
+ "golang.org/x/sync/errgroup"
+)
+
+// PartitionedFanoutWriter distributes Arrow records across multiple
partitions based on
+// a partition specification, writing data to separate files for each
partition using
+// a fanout pattern with configurable parallelism.
+type PartitionedFanoutWriter struct {
+ partitionSpec iceberg.PartitionSpec
+ schema *iceberg.Schema
+ itr iter.Seq2[arrow.Record, error]
+ writers *WriterFactory
+}
+
+// PartitionInfo holds the row indices and partition values for a specific
partition,
+// used during the fanout process to group rows by their partition key.
+type PartitionInfo struct {
+ rows []int64
+ partitionValues map[int]any
+}
+
+// NewPartitionedFanoutWriter creates a new PartitionedFanoutWriter with the
specified
+// partition specification, schema, and record iterator.
+func NewPartitionedFanoutWriter(partitionSpec iceberg.PartitionSpec, schema
*iceberg.Schema, itr iter.Seq2[arrow.Record, error]) *PartitionedFanoutWriter {
+ return &PartitionedFanoutWriter{
+ partitionSpec: partitionSpec,
+ schema: schema,
+ itr: itr,
+ }
+}
+
+func (p *PartitionedFanoutWriter) partitionPath(data partitionRecord) string {
+ return p.partitionSpec.PartitionToPath(data, p.schema)
+}
+
+// Write writes the Arrow records to the specified location using a fanout
pattern with
+// the specified number of workers. The returned iterator yields the data
files written
+// by the fanout process.
+func (p *PartitionedFanoutWriter) Write(ctx context.Context, workers int)
iter.Seq2[iceberg.DataFile, error] {
+ inputRecordsCh := make(chan arrow.Record, workers)
+ outputDataFilesCh := make(chan iceberg.DataFile, workers)
+
+ fanoutWorkers, ctx := errgroup.WithContext(ctx)
+ if err := p.startRecordFeeder(ctx, fanoutWorkers, inputRecordsCh); err
!= nil {
+ return func(yield func(iceberg.DataFile, error) bool) {
+ yield(nil, fmt.Errorf("failed to start record feeder:
%w", err))
+ }
+ }
+
+ for i := 0; i < workers; i++ {
+ fanoutWorkers.Go(func() error {
+ return p.fanout(ctx, inputRecordsCh, outputDataFilesCh)
+ })
+ }
+
+ return p.yieldDataFiles(ctx, fanoutWorkers, outputDataFilesCh)
+}
+
+func (p *PartitionedFanoutWriter) startRecordFeeder(ctx context.Context,
fanoutWorkers *errgroup.Group, inputRecordsCh chan<- arrow.Record) error {
+ fanoutWorkers.Go(func() error {
+ defer close(inputRecordsCh)
+
+ for record, err := range p.itr {
+ if err != nil {
+ return err
+ }
+
+ select {
+ case <-ctx.Done():
+ record.Release()
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case inputRecordsCh <- record:
+ }
+ }
+ return nil
+ })
+
+ return nil
+}
+
+func (p *PartitionedFanoutWriter) fanout(ctx context.Context, inputRecordsCh
<-chan arrow.Record, dataFilesChannel chan<- iceberg.DataFile) error {
+ for {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case record, ok := <-inputRecordsCh:
+ if !ok {
+ return nil
+ }
+ defer record.Release()
+
+ partitionMap, err := p.getPartitionMap(record)
+ if err != nil {
+ return err
+ }
+
+ for partition, val := range partitionMap {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err !=
nil {
+ return err
+ }
+ return nil
+ default:
+ }
+
+ partitionRecord, err :=
partitionBatchByKey(ctx)(record, val.rows)
+ if err != nil {
+ return err
+ }
+
+ rollingDataWriter, err :=
p.writers.getOrCreateRollingDataWriter(partition, val.partitionValues)
+ if err != nil {
+ return err
+ }
+
+ err = rollingDataWriter.Add(ctx,
partitionRecord, dataFilesChannel)
+ if err != nil {
+ return err
+ }
+ }
+ }
+ }
+}
+
+func (p *PartitionedFanoutWriter) yieldDataFiles(ctx context.Context,
fanoutWorkers *errgroup.Group, outputDataFilesCh chan iceberg.DataFile)
iter.Seq2[iceberg.DataFile, error] {
+ return iter.Seq2[iceberg.DataFile, error](func(yield
func(iceberg.DataFile, error) bool) {
+ var dataFiles []iceberg.DataFile
+
+ done := make(chan struct{})
+ bufferDone := make(chan struct{})
+
+ var waitErr, closeErr error
+
+ go func() {
+ defer close(bufferDone)
+ for dataFile := range outputDataFilesCh {
+ dataFiles = append(dataFiles, dataFile)
+ }
+ }()
+
+ go func() {
+ defer close(done)
+ defer close(outputDataFilesCh)
+ waitErr = fanoutWorkers.Wait()
+ closeErr = p.writers.closeAll(ctx, outputDataFilesCh)
+ }()
+
+ <-done
+ <-bufferDone
+
+ if waitErr != nil {
+ _ = yield(nil, waitErr)
+ return
+ }
+
+ if closeErr != nil {
+ _ = yield(nil, closeErr)
+ return
+ }
+
+ for _, dataFile := range dataFiles {
+ if !yield(dataFile, nil) {
+ return
+ }
+ }
+ })
+}
+
+func (p *PartitionedFanoutWriter) getPartitionMap(record arrow.Record)
(map[string]PartitionInfo, error) {
+ partitionMap := make(map[string]PartitionInfo)
+ partitionFields := p.partitionSpec.PartitionType(p.schema).FieldList
+
+ for row := range record.NumRows() {
+ partitionRec := make(partitionRecord, len(partitionFields))
+ partitionValues := make(map[int]any)
+
+ for i := range partitionFields {
+ sourceField := p.partitionSpec.Field(i)
+ colName, _ :=
p.schema.FindColumnName(sourceField.SourceID)
+ colIdx := record.Schema().FieldIndices(colName)[0]
+ col := record.Column(colIdx)
+
+ val, err := getArrowValueAsIcebergLiteral(col, int(row))
+ if err != nil {
+ return nil, fmt.Errorf("failed to get arrow
value as iceberg literal: %w", err)
+ }
+
+ transformedLiteral :=
sourceField.Transform.Apply(iceberg.Optional[iceberg.Literal]{
+ Valid: val != nil,
+ Val: val,
+ })
+
+ if transformedLiteral.Valid {
+ partitionRec[i] = transformedLiteral.Val.Any()
+ partitionValues[sourceField.FieldID] =
transformedLiteral.Val.Any()
+ }
+ }
+ partitionKey := p.partitionPath(partitionRec)
+ partVal := partitionMap[partitionKey]
+ partVal.rows = append(partitionMap[partitionKey].rows, row)
+ partVal.partitionValues = partitionValues
+ partitionMap[partitionKey] = partVal
+ }
+
+ return partitionMap, nil
+}
+
+type partitionBatchFn func(arrow.Record, []int64) (arrow.Record, error)
+
+func partitionBatchByKey(ctx context.Context) partitionBatchFn {
+ mem := compute.GetAllocator(ctx)
+
+ return func(record arrow.Record, rowIndices []int64) (arrow.Record,
error) {
+ bldr := array.NewInt64Builder(mem)
+ defer bldr.Release()
+
+ bldr.AppendValues(rowIndices, nil)
+ rowIndicesArr := bldr.NewInt64Array()
+ defer rowIndicesArr.Release()
+
+ partitionedRecord, err := compute.Take(
+ ctx,
+ *compute.DefaultTakeOptions(),
+ compute.NewDatumWithoutOwning(record),
+ compute.NewDatumWithoutOwning(rowIndicesArr),
+ )
+ if err != nil {
+ return nil, err
+ }
+
+ return partitionedRecord.(*compute.RecordDatum).Value, nil
+ }
+}
+
+func getArrowValueAsIcebergLiteral(column arrow.Array, row int)
(iceberg.Literal, error) {
+ if column.IsNull(row) {
+ return nil, nil
+ }
+
+ switch arr := column.(type) {
+ case *array.Date32:
+ return iceberg.NewLiteral(iceberg.Date(arr.Value(row))), nil
+ case *array.Time64:
+ return iceberg.NewLiteral(iceberg.Time(arr.Value(row))), nil
+ case *array.Timestamp:
+ return iceberg.NewLiteral(iceberg.Timestamp(arr.Value(row))),
nil
+ case *array.Decimal128:
Review Comment:
Decimal32/Decimal64?
##########
table/partitioned_fanout_writer.go:
##########
@@ -0,0 +1,309 @@
+// 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 table
+
+import (
+ "context"
+ "fmt"
+ "iter"
+
+ "github.com/apache/arrow-go/v18/arrow"
+ "github.com/apache/arrow-go/v18/arrow/array"
+ "github.com/apache/arrow-go/v18/arrow/compute"
+ "github.com/apache/iceberg-go"
+ "golang.org/x/sync/errgroup"
+)
+
+// PartitionedFanoutWriter distributes Arrow records across multiple
partitions based on
+// a partition specification, writing data to separate files for each
partition using
+// a fanout pattern with configurable parallelism.
+type PartitionedFanoutWriter struct {
+ partitionSpec iceberg.PartitionSpec
+ schema *iceberg.Schema
+ itr iter.Seq2[arrow.Record, error]
+ writers *WriterFactory
+}
+
+// PartitionInfo holds the row indices and partition values for a specific
partition,
+// used during the fanout process to group rows by their partition key.
+type PartitionInfo struct {
+ rows []int64
+ partitionValues map[int]any
+}
+
+// NewPartitionedFanoutWriter creates a new PartitionedFanoutWriter with the
specified
+// partition specification, schema, and record iterator.
+func NewPartitionedFanoutWriter(partitionSpec iceberg.PartitionSpec, schema
*iceberg.Schema, itr iter.Seq2[arrow.Record, error]) *PartitionedFanoutWriter {
+ return &PartitionedFanoutWriter{
+ partitionSpec: partitionSpec,
+ schema: schema,
+ itr: itr,
+ }
+}
+
+func (p *PartitionedFanoutWriter) partitionPath(data partitionRecord) string {
+ return p.partitionSpec.PartitionToPath(data, p.schema)
+}
+
+// Write writes the Arrow records to the specified location using a fanout
pattern with
+// the specified number of workers. The returned iterator yields the data
files written
+// by the fanout process.
+func (p *PartitionedFanoutWriter) Write(ctx context.Context, workers int)
iter.Seq2[iceberg.DataFile, error] {
+ inputRecordsCh := make(chan arrow.Record, workers)
+ outputDataFilesCh := make(chan iceberg.DataFile, workers)
+
+ fanoutWorkers, ctx := errgroup.WithContext(ctx)
+ if err := p.startRecordFeeder(ctx, fanoutWorkers, inputRecordsCh); err
!= nil {
+ return func(yield func(iceberg.DataFile, error) bool) {
+ yield(nil, fmt.Errorf("failed to start record feeder:
%w", err))
+ }
+ }
+
+ for i := 0; i < workers; i++ {
+ fanoutWorkers.Go(func() error {
+ return p.fanout(ctx, inputRecordsCh, outputDataFilesCh)
+ })
+ }
+
+ return p.yieldDataFiles(ctx, fanoutWorkers, outputDataFilesCh)
+}
+
+func (p *PartitionedFanoutWriter) startRecordFeeder(ctx context.Context,
fanoutWorkers *errgroup.Group, inputRecordsCh chan<- arrow.Record) error {
+ fanoutWorkers.Go(func() error {
+ defer close(inputRecordsCh)
+
+ for record, err := range p.itr {
+ if err != nil {
+ return err
+ }
+
+ select {
+ case <-ctx.Done():
+ record.Release()
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case inputRecordsCh <- record:
+ }
+ }
+ return nil
+ })
+
+ return nil
+}
+
+func (p *PartitionedFanoutWriter) fanout(ctx context.Context, inputRecordsCh
<-chan arrow.Record, dataFilesChannel chan<- iceberg.DataFile) error {
+ for {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err != nil {
+ return err
+ }
+ return nil
+ case record, ok := <-inputRecordsCh:
+ if !ok {
+ return nil
+ }
+ defer record.Release()
+
+ partitionMap, err := p.getPartitionMap(record)
+ if err != nil {
+ return err
+ }
+
+ for partition, val := range partitionMap {
+ select {
+ case <-ctx.Done():
+ if err := context.Cause(ctx); err !=
nil {
+ return err
+ }
+ return nil
+ default:
+ }
+
+ partitionRecord, err :=
partitionBatchByKey(ctx)(record, val.rows)
+ if err != nil {
+ return err
+ }
+
+ rollingDataWriter, err :=
p.writers.getOrCreateRollingDataWriter(partition, val.partitionValues)
+ if err != nil {
+ return err
+ }
+
+ err = rollingDataWriter.Add(ctx,
partitionRecord, dataFilesChannel)
+ if err != nil {
+ return err
+ }
+ }
+ }
+ }
+}
+
+func (p *PartitionedFanoutWriter) yieldDataFiles(ctx context.Context,
fanoutWorkers *errgroup.Group, outputDataFilesCh chan iceberg.DataFile)
iter.Seq2[iceberg.DataFile, error] {
+ return iter.Seq2[iceberg.DataFile, error](func(yield
func(iceberg.DataFile, error) bool) {
+ var dataFiles []iceberg.DataFile
+
+ done := make(chan struct{})
+ bufferDone := make(chan struct{})
+
+ var waitErr, closeErr error
+
+ go func() {
+ defer close(bufferDone)
+ for dataFile := range outputDataFilesCh {
+ dataFiles = append(dataFiles, dataFile)
+ }
+ }()
+
+ go func() {
+ defer close(done)
+ defer close(outputDataFilesCh)
+ waitErr = fanoutWorkers.Wait()
+ closeErr = p.writers.closeAll(ctx, outputDataFilesCh)
+ }()
+
+ <-done
+ <-bufferDone
+
+ if waitErr != nil {
+ _ = yield(nil, waitErr)
+ return
+ }
+
+ if closeErr != nil {
+ _ = yield(nil, closeErr)
+ return
+ }
+
+ for _, dataFile := range dataFiles {
+ if !yield(dataFile, nil) {
+ return
+ }
+ }
+ })
+}
+
+func (p *PartitionedFanoutWriter) getPartitionMap(record arrow.Record)
(map[string]PartitionInfo, error) {
+ partitionMap := make(map[string]PartitionInfo)
+ partitionFields := p.partitionSpec.PartitionType(p.schema).FieldList
+
+ for row := range record.NumRows() {
+ partitionRec := make(partitionRecord, len(partitionFields))
+ partitionValues := make(map[int]any)
+
+ for i := range partitionFields {
+ sourceField := p.partitionSpec.Field(i)
+ colName, _ :=
p.schema.FindColumnName(sourceField.SourceID)
+ colIdx := record.Schema().FieldIndices(colName)[0]
+ col := record.Column(colIdx)
+
+ val, err := getArrowValueAsIcebergLiteral(col, int(row))
+ if err != nil {
+ return nil, fmt.Errorf("failed to get arrow
value as iceberg literal: %w", err)
+ }
+
+ transformedLiteral :=
sourceField.Transform.Apply(iceberg.Optional[iceberg.Literal]{
+ Valid: val != nil,
+ Val: val,
+ })
+
+ if transformedLiteral.Valid {
+ partitionRec[i] = transformedLiteral.Val.Any()
+ partitionValues[sourceField.FieldID] =
transformedLiteral.Val.Any()
+ }
Review Comment:
short-circuit this entire thing by checking `col.IsNull(int(row))`?
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]