-
Notifications
You must be signed in to change notification settings - Fork 6
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
Showing
12 changed files
with
499 additions
and
44 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
90 changes: 90 additions & 0 deletions
90
...ntation/paimon-python-java-bridge/src/main/java/org/apache/paimon/python/BytesReader.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,90 @@ | ||
/* | ||
* 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 org.apache.paimon.python; | ||
|
||
import org.apache.paimon.arrow.ArrowUtils; | ||
import org.apache.paimon.arrow.vector.ArrowFormatWriter; | ||
import org.apache.paimon.data.InternalRow; | ||
import org.apache.paimon.reader.RecordReader; | ||
import org.apache.paimon.reader.RecordReaderIterator; | ||
import org.apache.paimon.table.source.Split; | ||
import org.apache.paimon.table.source.TableRead; | ||
import org.apache.paimon.types.RowType; | ||
|
||
import org.apache.arrow.vector.VectorSchemaRoot; | ||
|
||
import javax.annotation.Nullable; | ||
|
||
import java.io.ByteArrayOutputStream; | ||
import java.io.IOException; | ||
|
||
/** Read Arrow bytes from split. */ | ||
public class BytesReader { | ||
|
||
private static final int DEFAULT_WRITE_BATCH_SIZE = 2048; | ||
|
||
private final TableRead tableRead; | ||
private final ArrowFormatWriter arrowFormatWriter; | ||
|
||
private RecordReaderIterator<InternalRow> iterator; | ||
private InternalRow nextRow; | ||
|
||
public BytesReader(TableRead tableRead, RowType rowType) { | ||
this.tableRead = tableRead; | ||
this.arrowFormatWriter = new ArrowFormatWriter(rowType, DEFAULT_WRITE_BATCH_SIZE); | ||
} | ||
|
||
public void setSplit(Split split) throws IOException { | ||
RecordReader<InternalRow> recordReader = tableRead.createReader(split); | ||
iterator = new RecordReaderIterator<InternalRow>(recordReader); | ||
nextRow(); | ||
} | ||
|
||
@Nullable | ||
public byte[] next() throws Exception { | ||
if (nextRow == null) { | ||
return null; | ||
} | ||
|
||
int rowCount = 0; | ||
while (nextRow != null && arrowFormatWriter.write(nextRow)) { | ||
nextRow(); | ||
rowCount++; | ||
} | ||
|
||
VectorSchemaRoot vsr = arrowFormatWriter.getVectorSchemaRoot(); | ||
vsr.setRowCount(rowCount); | ||
ByteArrayOutputStream out = new ByteArrayOutputStream(); | ||
ArrowUtils.serializeToIpc(vsr, out); | ||
if (nextRow == null) { | ||
// close resource | ||
arrowFormatWriter.close(); | ||
iterator.close(); | ||
} | ||
return out.toByteArray(); | ||
} | ||
|
||
private void nextRow() { | ||
if (iterator.hasNext()) { | ||
nextRow = iterator.next(); | ||
} else { | ||
nextRow = null; | ||
} | ||
} | ||
} |
Oops, something went wrong.