1+ package com .alipay .oceanbase .hbase .result ;
2+
3+ import com .alipay .oceanbase .hbase .util .OHBaseFuncUtils ;
4+ import com .alipay .oceanbase .rpc .protocol .payload .impl .ObObj ;
5+ import com .alipay .oceanbase .rpc .stream .ObTableClientQueryAsyncStreamResult ;
6+ import com .alipay .oceanbase .rpc .stream .ObTableClientQueryStreamResult ;
7+ import org .apache .hadoop .hbase .Cell ;
8+ import org .apache .hadoop .hbase .KeyValue ;
9+ import org .apache .hadoop .hbase .PrivateCellUtil ;
10+ import org .apache .hadoop .hbase .client .Result ;
11+ import org .apache .hadoop .hbase .util .Threads ;
12+ import org .apache .hbase .thirdparty .com .google .common .annotations .VisibleForTesting ;
13+
14+ import java .io .IOException ;
15+ import java .io .InterruptedIOException ;
16+ import java .util .ArrayList ;
17+ import java .util .Arrays ;
18+ import java .util .List ;
19+ import java .util .Queue ;
20+ import java .util .concurrent .ConcurrentLinkedQueue ;
21+ import java .util .concurrent .atomic .AtomicLong ;
22+ import java .util .concurrent .locks .Condition ;
23+ import java .util .concurrent .locks .Lock ;
24+ import java .util .concurrent .locks .ReentrantLock ;
25+ import java .util .function .Consumer ;
26+ import java .util .concurrent .LinkedBlockingQueue ;
27+
28+ import static com .alipay .oceanbase .hbase .util .TableHBaseLoggerFactory .LCD ;
29+
30+ public class ClientAsyncStreamScanner extends ClientStreamScanner {
31+ private Queue <Result > cache ;
32+ private long maxCacheSize ;
33+ private AtomicLong cacheSizeInBytes ;
34+ long maxResultSize ;
35+ // exception queue (from prefetch to main scan execution)
36+ private Queue <Exception > exceptionsQueue ;
37+ // prefetch thread to be executed asynchronously
38+ private Thread prefetcher ;
39+ // used for testing
40+ private Consumer <Boolean > prefetchListener = null ;
41+
42+ private final Lock lock = new ReentrantLock ();
43+ private final Condition notEmpty = lock .newCondition ();
44+ private final Condition notFull = lock .newCondition ();
45+
46+ public ClientAsyncStreamScanner (ObTableClientQueryAsyncStreamResult streamResult , String tableName , byte [] family , boolean isTableGroup , long maxResultSize ) throws Exception {
47+ super (streamResult , tableName , family , isTableGroup );
48+ this .maxResultSize = maxResultSize ;
49+ initCache ();
50+ loadCache ();
51+ }
52+
53+ public ClientAsyncStreamScanner (ObTableClientQueryStreamResult streamResult , String tableName , byte [] family , boolean isTableGroup , long maxResultSize ) throws Exception {
54+ super (streamResult , tableName , family , isTableGroup );
55+ this .maxResultSize = maxResultSize ;
56+ initCache ();
57+ loadCache ();
58+ }
59+
60+ @ VisibleForTesting
61+ public void setPrefetchListener (Consumer <Boolean > prefetchListener ) {
62+ this .prefetchListener = prefetchListener ;
63+ }
64+
65+ private void initCache () {
66+ // concurrent cache
67+ maxCacheSize = maxResultSize > Long .MAX_VALUE / 2 ? maxResultSize : maxResultSize * 2 ;
68+ cache = new LinkedBlockingQueue <>();
69+ cacheSizeInBytes = new AtomicLong (0 );
70+ exceptionsQueue = new ConcurrentLinkedQueue <>();
71+ prefetcher = new Thread (new PrefetchRunnable ());
72+ Threads .setDaemonThreadRunning (prefetcher , tableName + ".asyncPrefetcher" );
73+ }
74+
75+ private void loadCache () throws Exception {
76+ if (streamResult .getRowIndex () == -1 && !streamResult .next ()) {
77+ return ;
78+ }
79+
80+ long addSize = 0 ;
81+ while (!streamResult .getCacheRows ().isEmpty ()) {
82+ try {
83+ checkStatus ();
84+
85+ List <ObObj > startRow ;
86+
87+ if (streamResult .getRowIndex () != -1 ) {
88+ startRow = streamResult .getRow ();
89+ } else if (streamResult .next ()) {
90+ startRow = streamResult .getRow ();
91+ } else {
92+ return ;
93+ }
94+
95+ byte [][] familyAndQualifier = new byte [2 ][];
96+ if (this .isTableGroup ) {
97+ // split family and qualifier
98+ familyAndQualifier = OHBaseFuncUtils .extractFamilyFromQualifier ((byte []) startRow
99+ .get (1 ).getValue ());
100+ this .family = familyAndQualifier [0 ];
101+ } else {
102+ familyAndQualifier [1 ] = (byte []) startRow .get (1 ).getValue ();
103+ }
104+
105+ byte [] sk = (byte []) startRow .get (0 ).getValue ();
106+ byte [] sq = familyAndQualifier [1 ];
107+ long st = (Long ) startRow .get (2 ).getValue ();
108+ byte [] sv = (byte []) startRow .get (3 ).getValue ();
109+
110+ KeyValue startKeyValue = new KeyValue (sk , family , sq , st , sv );
111+ List <Cell > keyValues = new ArrayList <>();
112+ keyValues .add (startKeyValue );
113+ addSize = 0 ;
114+ while (streamResult .next ()) {
115+ List <ObObj > row = streamResult .getRow ();
116+ if (this .isTableGroup ) {
117+ // split family and qualifier
118+ familyAndQualifier = OHBaseFuncUtils .extractFamilyFromQualifier ((byte []) row
119+ .get (1 ).getValue ());
120+ this .family = familyAndQualifier [0 ];
121+ } else {
122+ familyAndQualifier [1 ] = (byte []) row .get (1 ).getValue ();
123+ }
124+ byte [] k = (byte []) row .get (0 ).getValue ();
125+ byte [] q = familyAndQualifier [1 ];
126+ long t = (Long ) row .get (2 ).getValue ();
127+ byte [] v = (byte []) row .get (3 ).getValue ();
128+
129+ if (Arrays .equals (sk , k )) {
130+ // when rowKey is equal to the previous rowKey ,merge the result into the same result
131+ KeyValue kv = new KeyValue (k , family , q , t , v );
132+ addSize += PrivateCellUtil .estimatedSizeOfCell (kv );
133+ keyValues .add (kv );
134+ } else {
135+ break ;
136+ }
137+ }
138+ cache .add (Result .create (keyValues ));
139+ addEstimatedSize (addSize );
140+ } catch (Exception e ) {
141+ logger .error (LCD .convert ("01-00000" ), streamResult .getTableName (), e );
142+ throw new IOException (String .format ("get table %s stream next result error " ,
143+ streamResult .getTableName ()), e );
144+ }
145+ }
146+ }
147+
148+ @ Override
149+ public Result next () throws IOException {
150+ try {
151+ lock .lock ();
152+ while (cache .isEmpty ()) {
153+ handleException ();
154+ if (this .closed ) {
155+ return null ;
156+ }
157+ try {
158+ notEmpty .await ();
159+ } catch (InterruptedException e ) {
160+ throw new InterruptedIOException ("Interrupted when wait to load cache" );
161+ }
162+ }
163+
164+ Result result = pollCache ();
165+ if (prefetchCondition ()) {
166+ notFull .signalAll ();
167+ }
168+ return result ;
169+ } finally {
170+ lock .unlock ();
171+ handleException ();
172+ }
173+ }
174+
175+ @ Override
176+ public void close () {
177+ try {
178+ lock .lock ();
179+ super .close ();
180+ closed = true ;
181+ notFull .signalAll ();
182+ notEmpty .signalAll ();
183+ } finally {
184+ lock .unlock ();
185+ }
186+ }
187+
188+ private void addEstimatedSize (long estimatedSize ) {
189+ cacheSizeInBytes .addAndGet (estimatedSize );
190+ }
191+
192+ private void handleException () throws IOException {
193+ //The prefetch task running in the background puts any exception it
194+ //catches into this exception queue.
195+ // Rethrow the exception so the application can handle it.
196+ while (!exceptionsQueue .isEmpty ()) {
197+ Exception first = exceptionsQueue .peek ();
198+ first .printStackTrace ();
199+ if (first instanceof IOException ) {
200+ throw (IOException ) first ;
201+ }
202+ throw (RuntimeException ) first ;
203+ }
204+ }
205+
206+ private boolean prefetchCondition () {
207+ return cacheSizeInBytes .get () < maxCacheSize / 2 ;
208+ }
209+
210+ private long estimatedResultSize (Result res ) {
211+ long result_size = 0 ;
212+ for (Cell cell : res .rawCells ()) {
213+ result_size += PrivateCellUtil .estimatedSizeOfCell (cell );
214+ }
215+ return result_size ;
216+ }
217+
218+ private Result pollCache () {
219+ Result res = cache .poll ();
220+ if (null != res ) {
221+ long estimatedSize = estimatedResultSize (res );
222+ addEstimatedSize (-estimatedSize );
223+ }
224+ return res ;
225+ }
226+
227+ private class PrefetchRunnable implements Runnable {
228+ @ Override
229+ public void run () {
230+ while (!closed ) {
231+ boolean succeed = false ;
232+ try {
233+ lock .lock ();
234+ while (!prefetchCondition ()) {
235+ notFull .await ();
236+ }
237+ loadCache ();
238+ succeed = true ;
239+ } catch (Exception e ) {
240+ exceptionsQueue .add (e );
241+ } finally {
242+ notEmpty .signalAll ();
243+ lock .unlock ();
244+ if (prefetchListener != null ) {
245+ prefetchListener .accept (succeed );
246+ }
247+ }
248+ }
249+ }
250+ }
251+ }
0 commit comments