-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
parquet_test.go
192 lines (166 loc) · 6.34 KB
/
parquet_test.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
// Copyright 2022 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt
package changefeedccl
import (
"context"
"os"
"testing"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdctest"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/parquet"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)
func TestParquetRows(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
// Rangefeed reader can time out under stress.
skip.UnderStress(t)
ctx := context.Background()
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{
// TODO(#98816): cdctest.GetHydratedTableDescriptor does not work with tenant dbs.
// Once it is fixed, this flag can be removed.
DefaultTestTenant: base.TestTenantDisabled,
})
defer s.Stopper().Stop(ctx)
maxRowGroupSize := int64(2)
sqlDB := sqlutils.MakeSQLRunner(db)
for _, tc := range []struct {
testName string
createTable string
inserts []string
}{
{
testName: "mixed",
createTable: `CREATE TABLE foo (
int32Col INT4 PRIMARY KEY,
varCharCol VARCHAR(16) ,
charCol CHAR(2),
tsCol TIMESTAMP ,
stringCol STRING ,
decimalCOl DECIMAL(12,2),
uuidCol UUID
)`,
inserts: []string{
`INSERT INTO foo values (0, 'zero', 'CA', now(), 'oiwjfoijsdjif', 'inf', gen_random_uuid())`,
`INSERT INTO foo values (1, 'one', 'NY', now(), 'sdi9fu90d', '-1.90', gen_random_uuid())`,
`INSERT INTO foo values (2, 'two', 'WA', now(), 'sd9fid9fuj', '0.01', gen_random_uuid())`,
`INSERT INTO foo values (3, 'three', 'ON', now(), 'sadklfhkdlsjf', '1.2', gen_random_uuid())`,
`INSERT INTO foo values (4, 'four', 'NS', now(), '123123', '-11222221.2', gen_random_uuid())`,
`INSERT INTO foo values (5, 'five', 'BC', now(), 'sadklfhkdlsjf', '1.2', gen_random_uuid())`,
`INSERT INTO foo values (6, 'siz', 'AB', now(), '123123', '-11222221.2', gen_random_uuid())`,
},
},
} {
t.Run(tc.testName, func(t *testing.T) {
sqlDB.Exec(t, tc.createTable)
defer func() {
sqlDB.Exec(t, "DROP TABLE foo")
}()
popRow, cleanup, decoder := makeRangefeedReaderAndDecoder(t, s)
defer cleanup()
fileName := "TestParquetRows"
var writer *parquetWriter
var numCols int
f, err := os.CreateTemp(os.TempDir(), fileName)
require.NoError(t, err)
defer func() {
if t.Failed() {
t.Logf("leaving %s for inspection", f.Name())
} else {
if err := os.Remove(f.Name()); err != nil {
t.Logf("could not cleanup temp file %s: %s", f.Name(), err)
}
}
}()
numRows := len(tc.inserts)
for _, insertStmt := range tc.inserts {
sqlDB.Exec(t, insertStmt)
}
datums := make([][]tree.Datum, numRows)
for i := 0; i < numRows; i++ {
v := popRow(t)
updatedRow, err := decoder.DecodeKV(
ctx, roachpb.KeyValue{Key: v.Key, Value: v.Value}, cdcevent.CurrentRow, v.Timestamp(), false)
require.NoError(t, err)
prevRow, err := decoder.DecodeKV(
ctx, roachpb.KeyValue{Key: v.Key, Value: v.PrevValue}, cdcevent.PrevRow, v.Timestamp(), false)
require.NoError(t, err)
if writer == nil {
writer, err = newParquetWriterFromRow(updatedRow, f, &TestingKnobs{EnableParquetMetadata: true}, parquet.WithMaxRowGroupLength(maxRowGroupSize),
parquet.WithCompressionCodec(parquet.CompressionGZIP))
if err != nil {
t.Fatalf(err.Error())
}
numCols = len(updatedRow.ResultColumns()) + 1
}
err = writer.addData(updatedRow, prevRow)
require.NoError(t, err)
// Save a copy of the datums we wrote.
datumRow := make([]tree.Datum, len(updatedRow.ResultColumns())+1)
err = populateDatums(updatedRow, prevRow, datumRow)
require.NoError(t, err)
datums[i] = datumRow
}
err = writer.close()
require.NoError(t, err)
parquet.ReadFileAndVerifyDatums(t, f.Name(), numRows, numCols, writer.inner, datums)
})
}
}
func makeRangefeedReaderAndDecoder(
t *testing.T, s serverutils.TestServerInterface,
) (func(t *testing.T) *kvpb.RangeFeedValue, func(), cdcevent.Decoder) {
tableDesc := cdctest.GetHydratedTableDescriptor(t, s.ExecutorConfig(), "foo")
popRow, cleanup := cdctest.MakeRangeFeedValueReader(t, s.ExecutorConfig(), tableDesc)
targets := changefeedbase.Targets{}
targets.Add(changefeedbase.Target{
Type: jobspb.ChangefeedTargetSpecification_PRIMARY_FAMILY_ONLY,
TableID: tableDesc.GetID(),
FamilyName: "primary",
})
sqlExecCfg := s.ExecutorConfig().(sql.ExecutorConfig)
ctx := context.Background()
decoder, err := cdcevent.NewEventDecoder(ctx, &sqlExecCfg, targets, false, false)
require.NoError(t, err)
return popRow, cleanup, decoder
}
func TestParquetResolvedTimestamps(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
testFn := func(t *testing.T, s TestServer, f cdctest.TestFeedFactory) {
sqlDB := sqlutils.MakeSQLRunner(s.DB)
sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY)`)
foo := feed(t, f, `CREATE CHANGEFEED FOR foo WITH format=parquet, resolved`)
defer closeFeed(t, foo)
firstResolved, _ := expectResolvedTimestamp(t, foo)
testutils.SucceedsSoon(t, func() error {
nextResolved, _ := expectResolvedTimestamp(t, foo)
if !firstResolved.Less(nextResolved) {
return errors.AssertionFailedf(
"expected resolved timestamp %s to eventually exceed timestamp %s",
nextResolved, firstResolved)
}
return nil
})
}
cdcTest(t, testFn, feedTestForceSink("cloudstorage"))
}