badalprasadsingh commented on code in PR #524: URL: https://github.com/apache/iceberg-go/pull/524#discussion_r2267808875
########## 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: Fixed ########## 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: Fixed ########## 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: 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. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org