25
25
import java .io .IOException ;
26
26
import java .util .Queue ;
27
27
import java .util .concurrent .ConcurrentLinkedQueue ;
28
+ import java .util .concurrent .Semaphore ;
29
+ import java .util .concurrent .atomic .AtomicInteger ;
28
30
import java .util .concurrent .atomic .AtomicLong ;
29
31
30
32
/**
@@ -41,6 +43,9 @@ public class LanceArrowWriter extends ArrowReader {
41
43
42
44
private final AtomicLong totalBytesRead = new AtomicLong ();
43
45
private ArrowWriter arrowWriter = null ;
46
+ private final AtomicInteger count = new AtomicInteger (0 );
47
+ private final Semaphore writeToken ;
48
+ private final Semaphore loadToken ;
44
49
45
50
public LanceArrowWriter (BufferAllocator allocator , Schema schema , int batchSize ) {
46
51
super (allocator );
@@ -49,60 +54,63 @@ public LanceArrowWriter(BufferAllocator allocator, Schema schema, int batchSize)
49
54
this .schema = schema ;
50
55
// TODO(lu) batch size as config?
51
56
this .batchSize = batchSize ;
57
+ this .writeToken = new Semaphore (0 );
58
+ this .loadToken = new Semaphore (0 );
52
59
}
53
60
54
61
void write (InternalRow row ) {
55
62
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 );
60
73
}
61
74
}
62
75
63
76
void setFinished () {
64
- synchronized (monitor ) {
65
- finished = true ;
66
- monitor .notify ();
67
- }
77
+ loadToken .release ();
78
+ finished = true ;
68
79
}
69
80
70
81
@ Override
71
- protected void prepareLoadNextBatch () throws IOException {
82
+ public void prepareLoadNextBatch () throws IOException {
72
83
super .prepareLoadNextBatch ();
73
- // Do not use ArrowWriter.reset since it does not work well with Arrow JNI
74
84
arrowWriter = ArrowWriter .create (this .getVectorSchemaRoot ());
85
+ // release batch size token for write
86
+ writeToken .release (batchSize );
75
87
}
76
88
77
89
@ Override
78
90
public boolean loadNextBatch () throws IOException {
79
91
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 ;
98
109
}
99
110
}
111
+ } catch (InterruptedException e ) {
112
+ throw new RuntimeException (e );
100
113
}
101
- if (rowCount == 0 ) {
102
- return false ;
103
- }
104
- arrowWriter .finish ();
105
- return true ;
106
114
}
107
115
108
116
@ Override
0 commit comments