|
| 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 | +import org.apache.paimon.utils.ThreadPoolUtils; |
| 30 | + |
| 31 | +import org.apache.paimon.shade.guava30.com.google.common.collect.Iterators; |
| 32 | + |
| 33 | +import org.apache.arrow.vector.VectorSchemaRoot; |
| 34 | + |
| 35 | +import javax.annotation.Nullable; |
| 36 | + |
| 37 | +import java.io.ByteArrayOutputStream; |
| 38 | +import java.io.IOException; |
| 39 | +import java.util.ArrayDeque; |
| 40 | +import java.util.ArrayList; |
| 41 | +import java.util.Collection; |
| 42 | +import java.util.Iterator; |
| 43 | +import java.util.List; |
| 44 | +import java.util.Queue; |
| 45 | +import java.util.concurrent.ConcurrentLinkedQueue; |
| 46 | +import java.util.concurrent.ExecutionException; |
| 47 | +import java.util.concurrent.ExecutorService; |
| 48 | +import java.util.concurrent.Future; |
| 49 | +import java.util.concurrent.ThreadPoolExecutor; |
| 50 | +import java.util.function.Function; |
| 51 | + |
| 52 | +/** Parallely read Arrow bytes from multiple splits. */ |
| 53 | +public class ParallelBytesReader { |
| 54 | + |
| 55 | + private static final String THREAD_NAME_PREFIX = "PARALLEL_SPLITS_READER"; |
| 56 | + private static final int DEFAULT_WRITE_BATCH_SIZE = 1024; |
| 57 | + |
| 58 | + private final TableRead tableRead; |
| 59 | + private final RowType rowType; |
| 60 | + private final int threadNum; |
| 61 | + |
| 62 | + private final ConcurrentLinkedQueue<RecordReaderIterator<InternalRow>> iterators; |
| 63 | + private final ConcurrentLinkedQueue<ArrowFormatWriter> arrowFormatWriters; |
| 64 | + |
| 65 | + private ThreadPoolExecutor executor; |
| 66 | + private Iterator<byte[]> bytesIterator; |
| 67 | + |
| 68 | + public ParallelBytesReader(TableRead tableRead, RowType rowType, int threadNum) { |
| 69 | + this.tableRead = tableRead; |
| 70 | + this.rowType = rowType; |
| 71 | + this.threadNum = threadNum; |
| 72 | + this.iterators = new ConcurrentLinkedQueue<>(); |
| 73 | + this.arrowFormatWriters = new ConcurrentLinkedQueue<>(); |
| 74 | + } |
| 75 | + |
| 76 | + public void setSplits(List<Split> splits) { |
| 77 | + bytesIterator = randomlyExecute(getExecutor(), makeProcessor(), splits); |
| 78 | + } |
| 79 | + |
| 80 | + public byte[] serializeSchema() { |
| 81 | + ArrowFormatWriter arrowFormatWriter = newWriter(); |
| 82 | + VectorSchemaRoot vsr = arrowFormatWriter.getVectorSchemaRoot(); |
| 83 | + ByteArrayOutputStream out = new ByteArrayOutputStream(); |
| 84 | + ArrowUtils.serializeToIpc(vsr, out); |
| 85 | + arrowFormatWriter.close(); |
| 86 | + return out.toByteArray(); |
| 87 | + } |
| 88 | + |
| 89 | + @Nullable |
| 90 | + public byte[] next() { |
| 91 | + if (bytesIterator.hasNext()) { |
| 92 | + return bytesIterator.next(); |
| 93 | + } else { |
| 94 | + closeResources(); |
| 95 | + return null; |
| 96 | + } |
| 97 | + } |
| 98 | + |
| 99 | + private ThreadPoolExecutor getExecutor() { |
| 100 | + if (executor == null) { |
| 101 | + executor = ThreadPoolUtils.createCachedThreadPool(threadNum, THREAD_NAME_PREFIX); |
| 102 | + } |
| 103 | + return executor; |
| 104 | + } |
| 105 | + |
| 106 | + private Function<Split, Iterator<byte[]>> makeProcessor() { |
| 107 | + return split -> { |
| 108 | + try { |
| 109 | + RecordReader<InternalRow> recordReader = tableRead.createReader(split); |
| 110 | + RecordReaderIterator<InternalRow> iterator = |
| 111 | + new RecordReaderIterator<>(recordReader); |
| 112 | + iterators.add(iterator); |
| 113 | + ArrowFormatWriter arrowFormatWriter = newWriter(); |
| 114 | + arrowFormatWriters.add(arrowFormatWriter); |
| 115 | + return new RecordBytesIterator(iterator, arrowFormatWriter); |
| 116 | + } catch (IOException e) { |
| 117 | + throw new RuntimeException(e); |
| 118 | + } |
| 119 | + }; |
| 120 | + } |
| 121 | + |
| 122 | + private <U, T> Iterator<T> randomlyExecute( |
| 123 | + ExecutorService executor, Function<U, Iterator<T>> processor, Collection<U> input) { |
| 124 | + List<Future<Iterator<T>>> futures = new ArrayList<>(input.size()); |
| 125 | + for (U u : input) { |
| 126 | + futures.add(executor.submit(() -> processor.apply(u))); |
| 127 | + } |
| 128 | + return futuresToIterIter(futures); |
| 129 | + } |
| 130 | + |
| 131 | + private <T> Iterator<T> futuresToIterIter(List<Future<Iterator<T>>> futures) { |
| 132 | + final Queue<Future<Iterator<T>>> queue = new ArrayDeque<>(futures); |
| 133 | + return Iterators.concat( |
| 134 | + new Iterator<Iterator<T>>() { |
| 135 | + public boolean hasNext() { |
| 136 | + return !queue.isEmpty(); |
| 137 | + } |
| 138 | + |
| 139 | + public Iterator<T> next() { |
| 140 | + try { |
| 141 | + return queue.poll().get(); |
| 142 | + } catch (InterruptedException e) { |
| 143 | + Thread.currentThread().interrupt(); |
| 144 | + throw new RuntimeException(e); |
| 145 | + } catch (ExecutionException e) { |
| 146 | + throw new RuntimeException(e); |
| 147 | + } |
| 148 | + } |
| 149 | + }); |
| 150 | + } |
| 151 | + |
| 152 | + private void closeResources() { |
| 153 | + for (RecordReaderIterator<InternalRow> iterator : iterators) { |
| 154 | + try { |
| 155 | + iterator.close(); |
| 156 | + } catch (Exception e) { |
| 157 | + throw new RuntimeException(e); |
| 158 | + } |
| 159 | + } |
| 160 | + iterators.clear(); |
| 161 | + |
| 162 | + for (ArrowFormatWriter arrowFormatWriter : arrowFormatWriters) { |
| 163 | + arrowFormatWriter.close(); |
| 164 | + } |
| 165 | + arrowFormatWriters.clear(); |
| 166 | + } |
| 167 | + |
| 168 | + private ArrowFormatWriter newWriter() { |
| 169 | + return new ArrowFormatWriter(rowType, DEFAULT_WRITE_BATCH_SIZE, true); |
| 170 | + } |
| 171 | +} |
0 commit comments