Skip to content

Commit f60283e

Browse files
SaintBacchuseddyxu
andauthored
refactor: remove the queue in LanceArrowWriter to reduce memory usage for spark sink (#3110)
Remove the queue in LanceArrowWriter since it may cache all rows in queue and that will require a lot of jvm memory. Use mutex to control the write rate of sinker. Writer will wait util the reader take the batch. And more I had moved the `maven-shade-plugin` into a new profile which is diabled by default because `jar-with-dependencie` was conflict with many jars in spark dependencie --------- Co-authored-by: Lei Xu <lei@lancedb.com>
1 parent 219ebcf commit f60283e

File tree

3 files changed

+84
-71
lines changed

3 files changed

+84
-71
lines changed

.github/workflows/java-publish.yml

+1-1
Original file line numberDiff line numberDiff line change
@@ -111,7 +111,7 @@ jobs:
111111
echo "use-agent" >> ~/.gnupg/gpg.conf
112112
echo "pinentry-mode loopback" >> ~/.gnupg/gpg.conf
113113
export GPG_TTY=$(tty)
114-
mvn --batch-mode -DskipTests -Drust.release.build=true -DpushChanges=false -Dgpg.passphrase=${{ secrets.GPG_PASSPHRASE }} deploy -P deploy-to-ossrh
114+
mvn --batch-mode -DskipTests -Drust.release.build=true -DpushChanges=false -Dgpg.passphrase=${{ secrets.GPG_PASSPHRASE }} deploy -P deploy-to-ossrh -P shade-jar
115115
env:
116116
SONATYPE_USER: ${{ secrets.SONATYPE_USER }}
117117
SONATYPE_TOKEN: ${{ secrets.SONATYPE_TOKEN }}

java/spark/pom.xml

+42-37
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,48 @@
3434
<scala.compat.version>2.13</scala.compat.version>
3535
</properties>
3636
</profile>
37+
<profile>
38+
<id>shade-jar</id>
39+
<activation>
40+
<activeByDefault>false</activeByDefault>
41+
</activation>
42+
<build>
43+
<plugins>
44+
<plugin>
45+
<groupId>org.apache.maven.plugins</groupId>
46+
<artifactId>maven-shade-plugin</artifactId>
47+
<executions>
48+
<execution>
49+
<id>uber-jar</id>
50+
<goals>
51+
<goal>shade</goal>
52+
</goals>
53+
<phase>package</phase>
54+
<configuration>
55+
<finalName>${project.artifactId}-${scala.compat.version}-${project.version}-jar-with-dependencies</finalName>
56+
<transformers>
57+
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
58+
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer"/>
59+
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer"/>
60+
</transformers>
61+
<filters>
62+
<filter>
63+
<artifact>*:*</artifact>
64+
<excludes>
65+
<exclude>LICENSE</exclude>
66+
<exclude>META-INF/*.SF</exclude>
67+
<exclude>META-INF/*.DSA</exclude>
68+
<exclude>META-INF/*.RSA</exclude>
69+
</excludes>
70+
</filter>
71+
</filters>
72+
</configuration>
73+
</execution>
74+
</executions>
75+
</plugin>
76+
</plugins>
77+
</build>
78+
</profile>
3779
</profiles>
3880

3981
<dependencies>
@@ -53,41 +95,4 @@
5395
<scope>test</scope>
5496
</dependency>
5597
</dependencies>
56-
57-
<build>
58-
<plugins>
59-
<plugin>
60-
<groupId>org.apache.maven.plugins</groupId>
61-
<artifactId>maven-shade-plugin</artifactId>
62-
<executions>
63-
<execution>
64-
<id>uber-jar</id>
65-
<goals>
66-
<goal>shade</goal>
67-
</goals>
68-
<phase>package</phase>
69-
<configuration>
70-
<finalName>${project.artifactId}-${scala.compat.version}-${project.version}-jar-with-dependencies</finalName>
71-
<transformers>
72-
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
73-
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer"/>
74-
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer"/>
75-
</transformers>
76-
<filters>
77-
<filter>
78-
<artifact>*:*</artifact>
79-
<excludes>
80-
<exclude>LICENSE</exclude>
81-
<exclude>META-INF/*.SF</exclude>
82-
<exclude>META-INF/*.DSA</exclude>
83-
<exclude>META-INF/*.RSA</exclude>
84-
</excludes>
85-
</filter>
86-
</filters>
87-
</configuration>
88-
</execution>
89-
</executions>
90-
</plugin>
91-
</plugins>
92-
</build>
9398
</project>

java/spark/src/main/java/com/lancedb/lance/spark/write/LanceArrowWriter.java

+41-33
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,8 @@
2525
import java.io.IOException;
2626
import java.util.Queue;
2727
import java.util.concurrent.ConcurrentLinkedQueue;
28+
import java.util.concurrent.Semaphore;
29+
import java.util.concurrent.atomic.AtomicInteger;
2830
import java.util.concurrent.atomic.AtomicLong;
2931

3032
/**
@@ -41,6 +43,9 @@ public class LanceArrowWriter extends ArrowReader {
4143

4244
private final AtomicLong totalBytesRead = new AtomicLong();
4345
private ArrowWriter arrowWriter = null;
46+
private final AtomicInteger count = new AtomicInteger(0);
47+
private final Semaphore writeToken;
48+
private final Semaphore loadToken;
4449

4550
public LanceArrowWriter(BufferAllocator allocator, Schema schema, int batchSize) {
4651
super(allocator);
@@ -49,60 +54,63 @@ public LanceArrowWriter(BufferAllocator allocator, Schema schema, int batchSize)
4954
this.schema = schema;
5055
// TODO(lu) batch size as config?
5156
this.batchSize = batchSize;
57+
this.writeToken = new Semaphore(0);
58+
this.loadToken = new Semaphore(0);
5259
}
5360

5461
void write(InternalRow row) {
5562
Preconditions.checkNotNull(row);
56-
synchronized (monitor) {
57-
// TODO(lu) wait if too much elements in rowQueue
58-
rowQueue.offer(row);
59-
monitor.notify();
63+
try {
64+
// wait util prepareLoadNextBatch to release write token,
65+
writeToken.acquire();
66+
arrowWriter.write(row);
67+
if (count.incrementAndGet() == batchSize) {
68+
// notify loadNextBatch to take the batch
69+
loadToken.release();
70+
}
71+
} catch (InterruptedException e) {
72+
throw new RuntimeException(e);
6073
}
6174
}
6275

6376
void setFinished() {
64-
synchronized (monitor) {
65-
finished = true;
66-
monitor.notify();
67-
}
77+
loadToken.release();
78+
finished = true;
6879
}
6980

7081
@Override
71-
protected void prepareLoadNextBatch() throws IOException {
82+
public void prepareLoadNextBatch() throws IOException {
7283
super.prepareLoadNextBatch();
73-
// Do not use ArrowWriter.reset since it does not work well with Arrow JNI
7484
arrowWriter = ArrowWriter.create(this.getVectorSchemaRoot());
85+
// release batch size token for write
86+
writeToken.release(batchSize);
7587
}
7688

7789
@Override
7890
public boolean loadNextBatch() throws IOException {
7991
prepareLoadNextBatch();
80-
int rowCount = 0;
81-
synchronized (monitor) {
82-
while (rowCount < batchSize) {
83-
while (rowQueue.isEmpty() && !finished) {
84-
try {
85-
monitor.wait();
86-
} catch (InterruptedException e) {
87-
Thread.currentThread().interrupt();
88-
throw new IOException("Interrupted while waiting for data", e);
89-
}
90-
}
91-
if (rowQueue.isEmpty() && finished) {
92-
break;
93-
}
94-
InternalRow row = rowQueue.poll();
95-
if (row != null) {
96-
arrowWriter.write(row);
97-
rowCount++;
92+
try {
93+
if (finished && count.get() == 0) {
94+
return false;
95+
}
96+
// wait util batch if full or finished
97+
loadToken.acquire();
98+
arrowWriter.finish();
99+
if (!finished) {
100+
count.set(0);
101+
return true;
102+
} else {
103+
// true if it has some rows and return false if there is no record
104+
if (count.get() > 0) {
105+
count.set(0);
106+
return true;
107+
} else {
108+
return false;
98109
}
99110
}
111+
} catch (InterruptedException e) {
112+
throw new RuntimeException(e);
100113
}
101-
if (rowCount == 0) {
102-
return false;
103-
}
104-
arrowWriter.finish();
105-
return true;
106114
}
107115

108116
@Override

0 commit comments

Comments
 (0)