|
| 1 | +// Licensed to the Apache Software Foundation (ASF) under one |
| 2 | +// or more contributor license agreements. See the NOTICE file |
| 3 | +// distributed with this work for additional information |
| 4 | +// regarding copyright ownership. The ASF licenses this file |
| 5 | +// to you under the Apache License, Version 2.0 (the |
| 6 | +// "License"); you may not use this file except in compliance |
| 7 | +// with the License. You may obtain a copy of the License at |
| 8 | +// |
| 9 | +// http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | +// |
| 11 | +// Unless required by applicable law or agreed to in writing, |
| 12 | +// software distributed under the License is distributed on an |
| 13 | +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 14 | +// KIND, either express or implied. See the License for the |
| 15 | +// specific language governing permissions and limitations |
| 16 | +// under the License. |
| 17 | + |
| 18 | +//go:build integration |
| 19 | + |
| 20 | +package table_test |
| 21 | + |
| 22 | +import ( |
| 23 | + "context" |
| 24 | + "io" |
| 25 | + "strings" |
| 26 | + "testing" |
| 27 | + "time" |
| 28 | + |
| 29 | + "github.com/apache/arrow-go/v18/arrow" |
| 30 | + "github.com/apache/arrow-go/v18/arrow/array" |
| 31 | + "github.com/apache/arrow-go/v18/arrow/memory" |
| 32 | + "github.com/apache/arrow-go/v18/parquet" |
| 33 | + "github.com/apache/arrow-go/v18/parquet/pqarrow" |
| 34 | + "github.com/apache/iceberg-go" |
| 35 | + "github.com/apache/iceberg-go/catalog" |
| 36 | + "github.com/apache/iceberg-go/catalog/rest" |
| 37 | + "github.com/apache/iceberg-go/internal/recipe" |
| 38 | + iceio "github.com/apache/iceberg-go/io" |
| 39 | + "github.com/apache/iceberg-go/table" |
| 40 | + "github.com/stretchr/testify/suite" |
| 41 | + "github.com/testcontainers/testcontainers-go/modules/compose" |
| 42 | +) |
| 43 | + |
| 44 | +type SparkIntegrationTestSuite struct { |
| 45 | + suite.Suite |
| 46 | + |
| 47 | + ctx context.Context |
| 48 | + cat catalog.Catalog |
| 49 | + props iceberg.Properties |
| 50 | + stack *compose.DockerCompose |
| 51 | +} |
| 52 | + |
| 53 | +func (s *SparkIntegrationTestSuite) SetupSuite() { |
| 54 | + var err error |
| 55 | + s.stack, err = recipe.Start(s.T()) |
| 56 | + s.Require().NoError(err) |
| 57 | + if s.stack == nil { |
| 58 | + s.T().Skip("skipping test, AWS_S3_ENDPOINT is set") |
| 59 | + } |
| 60 | +} |
| 61 | + |
| 62 | +func (s *SparkIntegrationTestSuite) SetupTest() { |
| 63 | + s.ctx = context.Background() |
| 64 | + s.props = iceberg.Properties{ |
| 65 | + iceio.S3Region: "us-east-1", |
| 66 | + iceio.S3EndpointURL: "http://localhost:9000", |
| 67 | + iceio.S3AccessKeyID: "admin", |
| 68 | + iceio.S3SecretAccessKey: "password", |
| 69 | + } |
| 70 | + |
| 71 | + cat, err := rest.NewCatalog(s.ctx, "rest", "http://localhost:8181", rest.WithAdditionalProps(s.props)) |
| 72 | + s.Require().NoError(err) |
| 73 | + s.cat = cat |
| 74 | +} |
| 75 | + |
| 76 | +func (s *SparkIntegrationTestSuite) TestAddFile() { |
| 77 | + const filename = "s3://warehouse/default/test_partitioned_by_days/data/ts_day=2023-03-13/supertest.parquet" |
| 78 | + |
| 79 | + tbl, err := s.cat.LoadTable(s.ctx, catalog.ToIdentifier("default", "test_partitioned_by_days"), nil) |
| 80 | + s.Require().NoError(err) |
| 81 | + |
| 82 | + sc, err := table.SchemaToArrowSchema(tbl.Schema(), nil, false, false) |
| 83 | + if err != nil { |
| 84 | + panic(err) |
| 85 | + } |
| 86 | + |
| 87 | + bldr := array.NewRecordBuilder(memory.DefaultAllocator, sc) |
| 88 | + defer bldr.Release() |
| 89 | + |
| 90 | + tm := time.Date(2023, 03, 13, 13, 22, 0, 0, time.UTC) |
| 91 | + ts, _ := arrow.TimestampFromTime(tm, arrow.Microsecond) |
| 92 | + bldr.Field(0).(*array.Date32Builder).Append(arrow.Date32FromTime(tm)) |
| 93 | + bldr.Field(1).(*array.TimestampBuilder).Append(ts) |
| 94 | + bldr.Field(2).(*array.Int32Builder).Append(13) |
| 95 | + bldr.Field(3).(*array.StringBuilder).Append("m") |
| 96 | + |
| 97 | + rec := bldr.NewRecord() |
| 98 | + defer rec.Release() |
| 99 | + |
| 100 | + fw, err := tbl.FS().(iceio.WriteFileIO).Create(filename) |
| 101 | + if err != nil { |
| 102 | + panic(err) |
| 103 | + } |
| 104 | + defer fw.Close() |
| 105 | + |
| 106 | + if err := pqarrow.WriteTable(array.NewTableFromRecords(sc, []arrow.Record{rec}), fw, rec.NumRows(), parquet.NewWriterProperties(), pqarrow.DefaultWriterProps()); err != nil { |
| 107 | + panic(err) |
| 108 | + } |
| 109 | + |
| 110 | + tx := tbl.NewTransaction() |
| 111 | + err = tx.AddFiles(s.ctx, []string{filename}, nil, false) |
| 112 | + s.Require().NoError(err) |
| 113 | + |
| 114 | + tbl, err = tx.Commit(s.ctx) |
| 115 | + s.Require().NoError(err) |
| 116 | + |
| 117 | + spark, err := s.stack.ServiceContainer(s.T().Context(), "spark-iceberg") |
| 118 | + s.Require().NoError(err) |
| 119 | + |
| 120 | + _, stdout, err := spark.Exec(s.ctx, []string{"ipython", "./run_spark_count_sql.py"}) |
| 121 | + s.Require().NoError(err) |
| 122 | + |
| 123 | + output, err := io.ReadAll(stdout) |
| 124 | + s.Require().NoError(err) |
| 125 | + strings.HasSuffix(string(output), ` |
| 126 | ++--------+ |
| 127 | +|count(1)| |
| 128 | ++--------+ |
| 129 | +| 13| |
| 130 | ++--------+ |
| 131 | +`) |
| 132 | +} |
| 133 | + |
| 134 | +func TestSparkIntegration(t *testing.T) { |
| 135 | + suite.Run(t, new(SparkIntegrationTestSuite)) |
| 136 | +} |
0 commit comments