Skip to content

Commit 73366fb

Browse files
authored
Complete table read and write (#6)
1 parent 974f8aa commit 73366fb

File tree

12 files changed

+499
-44
lines changed

12 files changed

+499
-44
lines changed

dev/dev-requirements.txt

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,4 +21,8 @@ setuptools>=18.0
2121
wheel
2222
py4j==0.10.9.7
2323
pyarrow>=5.0.0
24+
pandas>=1.3.0
25+
numpy>=1.22.4
26+
python-dateutil>=2.8.0,<3
27+
pytz>=2018.3
2428
pytest~=7.0

java_based_implementation/api_impl.py

Lines changed: 67 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -16,12 +16,15 @@
1616
# limitations under the License.
1717
################################################################################
1818

19+
import itertools
20+
1921
from java_based_implementation.java_gateway import get_gateway
2022
from java_based_implementation.util.java_utils import to_j_catalog_context
2123
from paimon_python_api import (catalog, table, read_builder, table_scan, split, table_read,
2224
write_builder, table_write, commit_message, table_commit)
23-
from pyarrow import RecordBatchReader, RecordBatch
24-
from typing import List
25+
from pyarrow import (RecordBatch, BufferOutputStream, RecordBatchStreamWriter,
26+
RecordBatchStreamReader, BufferReader, RecordBatchReader)
27+
from typing import List, Iterator
2528

2629

2730
class Catalog(catalog.Catalog):
@@ -49,18 +52,19 @@ def __init__(self, j_table):
4952
self._j_table = j_table
5053

5154
def new_read_builder(self) -> 'ReadBuilder':
52-
j_read_builder = self._j_table.newReadBuilder()
53-
return ReadBuilder(j_read_builder)
55+
j_read_builder = get_gateway().jvm.InvocationUtil.getReadBuilder(self._j_table)
56+
return ReadBuilder(j_read_builder, self._j_table.rowType())
5457

5558
def new_batch_write_builder(self) -> 'BatchWriteBuilder':
56-
j_batch_write_builder = self._j_table.newBatchWriteBuilder()
57-
return BatchWriteBuilder(j_batch_write_builder)
59+
j_batch_write_builder = get_gateway().jvm.InvocationUtil.getBatchWriteBuilder(self._j_table)
60+
return BatchWriteBuilder(j_batch_write_builder, self._j_table.rowType())
5861

5962

6063
class ReadBuilder(read_builder.ReadBuilder):
6164

62-
def __init__(self, j_read_builder):
65+
def __init__(self, j_read_builder, j_row_type):
6366
self._j_read_builder = j_read_builder
67+
self._j_row_type = j_row_type
6468

6569
def with_projection(self, projection: List[List[int]]) -> 'ReadBuilder':
6670
self._j_read_builder.withProjection(projection)
@@ -75,8 +79,8 @@ def new_scan(self) -> 'TableScan':
7579
return TableScan(j_table_scan)
7680

7781
def new_read(self) -> 'TableRead':
78-
# TODO
79-
pass
82+
j_table_read = self._j_read_builder.newRead()
83+
return TableRead(j_table_read, self._j_row_type)
8084

8185

8286
class TableScan(table_scan.TableScan):
@@ -110,23 +114,56 @@ def to_j_split(self):
110114

111115
class TableRead(table_read.TableRead):
112116

113-
def create_reader(self, split: Split) -> RecordBatchReader:
114-
# TODO
115-
pass
117+
def __init__(self, j_table_read, j_row_type):
118+
self._j_table_read = j_table_read
119+
self._j_bytes_reader = get_gateway().jvm.InvocationUtil.createBytesReader(
120+
j_table_read, j_row_type)
121+
self._arrow_schema = None
122+
123+
def create_reader(self, split: Split):
124+
self._j_bytes_reader.setSplit(split.to_j_split())
125+
batch_iterator = self._batch_generator()
126+
# to init arrow schema
127+
try:
128+
first_batch = next(batch_iterator)
129+
except StopIteration:
130+
return self._empty_batch_reader()
131+
132+
batches = itertools.chain((b for b in [first_batch]), batch_iterator)
133+
return RecordBatchReader.from_batches(self._arrow_schema, batches)
134+
135+
def _batch_generator(self) -> Iterator[RecordBatch]:
136+
while True:
137+
next_bytes = self._j_bytes_reader.next()
138+
if next_bytes is None:
139+
break
140+
else:
141+
stream_reader = RecordBatchStreamReader(BufferReader(next_bytes))
142+
if self._arrow_schema is None:
143+
self._arrow_schema = stream_reader.schema
144+
yield from stream_reader
145+
146+
def _empty_batch_reader(self):
147+
import pyarrow as pa
148+
schema = pa.schema([])
149+
empty_batch = pa.RecordBatch.from_arrays([], schema=schema)
150+
empty_reader = pa.RecordBatchReader.from_batches(schema, [empty_batch])
151+
return empty_reader
116152

117153

118154
class BatchWriteBuilder(write_builder.BatchWriteBuilder):
119155

120-
def __init__(self, j_batch_write_builder):
156+
def __init__(self, j_batch_write_builder, j_row_type):
121157
self._j_batch_write_builder = j_batch_write_builder
158+
self._j_row_type = j_row_type
122159

123160
def with_overwrite(self, static_partition: dict) -> 'BatchWriteBuilder':
124161
self._j_batch_write_builder.withOverwrite(static_partition)
125162
return self
126163

127164
def new_write(self) -> 'BatchTableWrite':
128165
j_batch_table_write = self._j_batch_write_builder.newWrite()
129-
return BatchTableWrite(j_batch_table_write)
166+
return BatchTableWrite(j_batch_table_write, self._j_row_type)
130167

131168
def new_commit(self) -> 'BatchTableCommit':
132169
j_batch_table_commit = self._j_batch_write_builder.newCommit()
@@ -135,17 +172,27 @@ def new_commit(self) -> 'BatchTableCommit':
135172

136173
class BatchTableWrite(table_write.BatchTableWrite):
137174

138-
def __init__(self, j_batch_table_write):
175+
def __init__(self, j_batch_table_write, j_row_type):
139176
self._j_batch_table_write = j_batch_table_write
177+
self._j_bytes_writer = get_gateway().jvm.InvocationUtil.createBytesWriter(
178+
j_batch_table_write, j_row_type)
140179

141180
def write(self, record_batch: RecordBatch):
142-
# TODO
143-
pass
181+
stream = BufferOutputStream()
182+
with RecordBatchStreamWriter(stream, record_batch.schema) as writer:
183+
writer.write(record_batch)
184+
writer.close()
185+
arrow_bytes = stream.getvalue().to_pybytes()
186+
self._j_bytes_writer.write(arrow_bytes)
144187

145188
def prepare_commit(self) -> List['CommitMessage']:
146189
j_commit_messages = self._j_batch_table_write.prepareCommit()
147190
return list(map(lambda cm: CommitMessage(cm), j_commit_messages))
148191

192+
def close(self):
193+
self._j_batch_table_write.close()
194+
self._j_bytes_writer.close()
195+
149196

150197
class CommitMessage(commit_message.CommitMessage):
151198

@@ -164,3 +211,6 @@ def __init__(self, j_batch_table_commit):
164211
def commit(self, commit_messages: List[CommitMessage]):
165212
j_commit_messages = list(map(lambda cm: cm.to_j_commit_message(), commit_messages))
166213
self._j_batch_table_commit.commit(j_commit_messages)
214+
215+
def close(self):
216+
self._j_batch_table_commit.close()

java_based_implementation/java_gateway.py

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -101,9 +101,14 @@ def launch_gateway():
101101
return gateway
102102

103103

104-
# TODO: import more
105104
def import_paimon_view(gateway):
106105
java_import(gateway.jvm, "org.apache.paimon.table.*")
106+
java_import(gateway.jvm, "org.apache.paimon.options.Options")
107+
java_import(gateway.jvm, "org.apache.paimon.catalog.*")
108+
java_import(gateway.jvm, "org.apache.paimon.schema.Schema*")
109+
java_import(gateway.jvm, 'org.apache.paimon.types.*')
110+
java_import(gateway.jvm, 'org.apache.paimon.python.InvocationUtil')
111+
java_import(gateway.jvm, "org.apache.paimon.data.*")
107112

108113

109114
class Watchdog(object):

java_based_implementation/paimon-python-java-bridge/pom.xml

Lines changed: 46 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -33,8 +33,10 @@
3333
<flink.shaded.hadoop.version>2.8.3-10.0</flink.shaded.hadoop.version>
3434
<py4j.version>0.10.9.7</py4j.version>
3535
<slf4j.version>1.7.32</slf4j.version>
36+
<log4j.version>2.17.1</log4j.version>
3637
<spotless.version>2.13.0</spotless.version>
3738
<spotless.delimiter>package</spotless.delimiter>
39+
<arrow.version>14.0.0</arrow.version>
3840
</properties>
3941

4042
<dependencies>
@@ -47,18 +49,48 @@
4749
<version>${paimon.version}</version>
4850
</dependency>
4951

52+
<dependency>
53+
<groupId>org.apache.paimon</groupId>
54+
<artifactId>paimon-arrow</artifactId>
55+
<version>${paimon.version}</version>
56+
</dependency>
57+
5058
<dependency>
5159
<groupId>org.slf4j</groupId>
5260
<artifactId>slf4j-api</artifactId>
5361
<version>${slf4j.version}</version>
5462
</dependency>
5563

64+
<dependency>
65+
<groupId>org.apache.logging.log4j</groupId>
66+
<artifactId>log4j-1.2-api</artifactId>
67+
<version>${log4j.version}</version>
68+
</dependency>
69+
5670
<dependency>
5771
<groupId>org.apache.flink</groupId>
5872
<artifactId>flink-shaded-hadoop-2-uber</artifactId>
5973
<version>${flink.shaded.hadoop.version}</version>
6074
</dependency>
6175

76+
<dependency>
77+
<groupId>org.apache.arrow</groupId>
78+
<artifactId>arrow-vector</artifactId>
79+
<version>${arrow.version}</version>
80+
</dependency>
81+
82+
<dependency>
83+
<groupId>org.apache.arrow</groupId>
84+
<artifactId>arrow-memory-unsafe</artifactId>
85+
<version>${arrow.version}</version>
86+
</dependency>
87+
88+
<dependency>
89+
<groupId>org.apache.arrow</groupId>
90+
<artifactId>arrow-c-data</artifactId>
91+
<version>${arrow.version}</version>
92+
</dependency>
93+
6294
<!-- Python API dependencies -->
6395

6496
<dependency>
@@ -119,11 +151,25 @@
119151
<artifactSet>
120152
<includes combine.children="append">
121153
<include>org.apache.paimon:paimon-bundle</include>
154+
<include>org.apache.paimon:paimon-arrow</include>
155+
<include>org.apache.arrow:arrow-vector</include>
156+
<include>org.apache.arrow:arrow-memory-core</include>
157+
<include>org.apache.arrow:arrow-memory-unsafe</include>
158+
<include>org.apache.arrow:arrow-c-data</include>
159+
<include>org.apache.arrow:arrow-format</include>
160+
<include>com.google.flatbuffers:flatbuffers-java</include>
122161
<include>org.slf4j:slf4j-api</include>
162+
<include>org.apache.logging.log4j:log4j-1.2-api</include>
123163
<include>org.apache.flink:flink-shaded-hadoop-2-uber</include>
124164
<include>net.sf.py4j:py4j</include>
125165
</includes>
126166
</artifactSet>
167+
<relocations>
168+
<relocation>
169+
<pattern>com.fasterxml.jackson</pattern>
170+
<shadedPattern>org.apache.paimon.shade.jackson2.com.fasterxml.jackson</shadedPattern>
171+
</relocation>
172+
</relocations>
127173
</configuration>
128174
</execution>
129175
</executions>
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,90 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.paimon.python;
20+
21+
import org.apache.paimon.arrow.ArrowUtils;
22+
import org.apache.paimon.arrow.vector.ArrowFormatWriter;
23+
import org.apache.paimon.data.InternalRow;
24+
import org.apache.paimon.reader.RecordReader;
25+
import org.apache.paimon.reader.RecordReaderIterator;
26+
import org.apache.paimon.table.source.Split;
27+
import org.apache.paimon.table.source.TableRead;
28+
import org.apache.paimon.types.RowType;
29+
30+
import org.apache.arrow.vector.VectorSchemaRoot;
31+
32+
import javax.annotation.Nullable;
33+
34+
import java.io.ByteArrayOutputStream;
35+
import java.io.IOException;
36+
37+
/** Read Arrow bytes from split. */
38+
public class BytesReader {
39+
40+
private static final int DEFAULT_WRITE_BATCH_SIZE = 2048;
41+
42+
private final TableRead tableRead;
43+
private final ArrowFormatWriter arrowFormatWriter;
44+
45+
private RecordReaderIterator<InternalRow> iterator;
46+
private InternalRow nextRow;
47+
48+
public BytesReader(TableRead tableRead, RowType rowType) {
49+
this.tableRead = tableRead;
50+
this.arrowFormatWriter = new ArrowFormatWriter(rowType, DEFAULT_WRITE_BATCH_SIZE);
51+
}
52+
53+
public void setSplit(Split split) throws IOException {
54+
RecordReader<InternalRow> recordReader = tableRead.createReader(split);
55+
iterator = new RecordReaderIterator<InternalRow>(recordReader);
56+
nextRow();
57+
}
58+
59+
@Nullable
60+
public byte[] next() throws Exception {
61+
if (nextRow == null) {
62+
return null;
63+
}
64+
65+
int rowCount = 0;
66+
while (nextRow != null && arrowFormatWriter.write(nextRow)) {
67+
nextRow();
68+
rowCount++;
69+
}
70+
71+
VectorSchemaRoot vsr = arrowFormatWriter.getVectorSchemaRoot();
72+
vsr.setRowCount(rowCount);
73+
ByteArrayOutputStream out = new ByteArrayOutputStream();
74+
ArrowUtils.serializeToIpc(vsr, out);
75+
if (nextRow == null) {
76+
// close resource
77+
arrowFormatWriter.close();
78+
iterator.close();
79+
}
80+
return out.toByteArray();
81+
}
82+
83+
private void nextRow() {
84+
if (iterator.hasNext()) {
85+
nextRow = iterator.next();
86+
} else {
87+
nextRow = null;
88+
}
89+
}
90+
}

0 commit comments

Comments
 (0)