forked from lakesoul-io/LakeSoul
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathNativeMetadataJavaClient.java
More file actions
531 lines (463 loc) · 21 KB
/
NativeMetadataJavaClient.java
File metadata and controls
531 lines (463 loc) · 21 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
// SPDX-FileCopyrightText: 2023 LakeSoul Contributors
//
// SPDX-License-Identifier: Apache-2.0
package com.dmetasoul.lakesoul.meta.jnr;
import com.dmetasoul.lakesoul.meta.DBUtil;
import com.dmetasoul.lakesoul.meta.DataBaseProperty;
import com.dmetasoul.lakesoul.meta.entity.JniWrapper;
import com.google.protobuf.InvalidProtocolBufferException;
import jnr.ffi.ObjectReferenceManager;
import jnr.ffi.Pointer;
import jnr.ffi.Runtime;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.sql.SQLException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.BiConsumer;
import java.util.stream.Collectors;
import static com.dmetasoul.lakesoul.meta.jnr.NativeUtils.*;
public class NativeMetadataJavaClient implements AutoCloseable {
private static final Logger LOG = LoggerFactory.getLogger(NativeMetadataJavaClient.class);
private long timeout;
private long bufferSize;
private long largeBufferSize;
private Pointer tokioPostgresClient = null;
private Pointer sharedBuffer = null;
private Pointer largeSharedBuffer = null;
private Pointer tokioRuntime = null;
private Pointer preparedStatement = null;
protected final LibLakeSoulMetaData libLakeSoulMetaData;
protected final ObjectReferenceManager<LibLakeSoulMetaData.BooleanCallback> booleanCallbackObjectReferenceManager;
protected final ObjectReferenceManager<LibLakeSoulMetaData.StringCallback> stringCallbackObjectReferenceManager;
protected final ObjectReferenceManager<LibLakeSoulMetaData.IntegerCallback> integerCallbackObjectReferenceManager;
private static NativeMetadataJavaClient instance = null;
private final ReentrantReadWriteLock lock;
private static DataBaseProperty dataBaseProperty = null;
public static void setDataBaseProperty(DataBaseProperty dataBaseProperty) {
NativeMetadataJavaClient.dataBaseProperty = dataBaseProperty;
}
public NativeMetadataJavaClient() {
this(5000L, 1 << 12, 1 << 16);
}
public NativeMetadataJavaClient(long timeout, int bufferSize, int largeBufferSize) {
this.timeout = timeout;
libLakeSoulMetaData = JnrLoader.get();
booleanCallbackObjectReferenceManager = Runtime.getRuntime(libLakeSoulMetaData).newObjectReferenceManager();
stringCallbackObjectReferenceManager = Runtime.getRuntime(libLakeSoulMetaData).newObjectReferenceManager();
integerCallbackObjectReferenceManager = Runtime.getRuntime(libLakeSoulMetaData).newObjectReferenceManager();
this.bufferSize = bufferSize;
this.largeBufferSize = largeBufferSize;
sharedBuffer = Runtime.getRuntime(libLakeSoulMetaData).getMemoryManager().allocateDirect(bufferSize);
largeSharedBuffer = Runtime.getRuntime(libLakeSoulMetaData).getMemoryManager().allocateDirect(largeBufferSize);
lock = new ReentrantReadWriteLock();
initialize();
}
public static NativeMetadataJavaClient getInstance() {
if (instance == null) {
instance = new NativeMetadataJavaClient();
}
return instance;
}
public Pointer getTokioPostgresClient() {
return tokioPostgresClient;
}
public ObjectReferenceManager<LibLakeSoulMetaData.BooleanCallback> getbooleanCallbackObjectReferenceManager() {
return booleanCallbackObjectReferenceManager;
}
public ObjectReferenceManager<LibLakeSoulMetaData.StringCallback> getStringCallbackObjectReferenceManager() {
return stringCallbackObjectReferenceManager;
}
public ObjectReferenceManager<LibLakeSoulMetaData.IntegerCallback> getIntegerCallbackObjectReferenceManager() {
return integerCallbackObjectReferenceManager;
}
public Runtime getRuntime() {
return Runtime.getRuntime(libLakeSoulMetaData);
}
public LibLakeSoulMetaData getLibLakeSoulMetaData() {
return libLakeSoulMetaData;
}
static class ReferencedBooleanCallback implements LibLakeSoulMetaData.BooleanCallback {
public final BiConsumer<Boolean, String> callback;
private final Pointer key;
private final ObjectReferenceManager<LibLakeSoulMetaData.BooleanCallback> referenceManager;
public ReferencedBooleanCallback(BiConsumer<Boolean, String> callback, ObjectReferenceManager<LibLakeSoulMetaData.BooleanCallback> referenceManager) {
this.callback = callback;
this.referenceManager = referenceManager;
key = this.referenceManager.add(this);
}
@Override
public void invoke(Boolean result, String msg) {
callback.accept(result, msg);
close();
}
public void close() {
if (key != null) {
referenceManager.remove(key);
}
}
}
static class ReferencedIntegerCallback implements LibLakeSoulMetaData.IntegerCallback {
public final BiConsumer<Integer, String> callback;
private final Pointer key;
private final ObjectReferenceManager<LibLakeSoulMetaData.IntegerCallback> referenceManager;
public ReferencedIntegerCallback(BiConsumer<Integer, String> callback, ObjectReferenceManager<LibLakeSoulMetaData.IntegerCallback> referenceManager) {
this.callback = callback;
this.referenceManager = referenceManager;
key = this.referenceManager.add(this);
}
@Override
public void invoke(Integer result, String msg) {
callback.accept(result, msg);
close();
}
public void close() {
if (key != null) {
referenceManager.remove(key);
}
}
}
static class ReferencedStringCallback implements LibLakeSoulMetaData.StringCallback, AutoCloseable {
public final BiConsumer<String, String> callback;
private final Pointer key;
private final ObjectReferenceManager<LibLakeSoulMetaData.StringCallback> referenceManager;
public ReferencedStringCallback(BiConsumer<String, String> callback, ObjectReferenceManager<LibLakeSoulMetaData.StringCallback> referenceManager) {
this.callback = callback;
this.referenceManager = referenceManager;
key = this.referenceManager.add(this);
}
@Override
public void invoke(String result, String msg) {
callback.accept(result, msg);
close();
}
@Override
public void close() {
if (key != null) {
referenceManager.remove(key);
}
}
}
private void initialize() {
DataBaseProperty dataBaseProperty = NativeMetadataJavaClient.dataBaseProperty;
if (dataBaseProperty == null) {
dataBaseProperty = DBUtil.getDBInfo();
}
tokioRuntime = libLakeSoulMetaData.create_tokio_runtime();
String config = String.format(
"host=%s port=%s dbname=%s user=%s password=%s",
dataBaseProperty.getHost(),
dataBaseProperty.getPort(),
dataBaseProperty.getDbName(),
dataBaseProperty.getUsername(),
dataBaseProperty.getPassword());
final CompletableFuture<Boolean> future = new CompletableFuture<>();
tokioPostgresClient = libLakeSoulMetaData.create_tokio_postgres_client(
new ReferencedBooleanCallback((bool, msg) -> {
if (msg.isEmpty()) {
future.complete(bool);
} else {
System.err.println(msg);
future.completeExceptionally(new IOException(msg));
}
}, getbooleanCallbackObjectReferenceManager()),
config,
tokioRuntime
);
preparedStatement = libLakeSoulMetaData.create_prepared_statement();
try {
future.get(timeout, TimeUnit.MILLISECONDS);
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
} catch (TimeoutException e) {
LOG.error("Configuring postgres with {} timeout", dataBaseProperty);
throw new RuntimeException(e);
}
}
public JniWrapper executeQuery(Integer queryType, List<String> params) {
try {
getReadLock();
int retryCounter = NATIVE_METADATA_MAX_RETRY_ATTEMPTS;
while (retryCounter >= 0) {
try {
final CompletableFuture<Integer> future = new CompletableFuture<>();
getLibLakeSoulMetaData().execute_query(
new ReferencedIntegerCallback((result, msg) -> {
if (msg.isEmpty()) {
future.complete(result);
} else {
future.completeExceptionally(new SQLException(msg));
}
}, getIntegerCallbackObjectReferenceManager()),
tokioRuntime,
tokioPostgresClient,
preparedStatement,
queryType,
String.join(PARAM_DELIM, params),
queryType < DAO_TYPE_QUERY_LIST_OFFSET ? sharedBuffer.address() : largeSharedBuffer.address()
);
Integer len = future.get(timeout, TimeUnit.MILLISECONDS);
if (len < 0) return null;
byte[] bytes = new byte[len];
if (queryType < DAO_TYPE_QUERY_LIST_OFFSET)
sharedBuffer.get(0, bytes, 0, len);
else
largeSharedBuffer.get(0, bytes, 0, len);
return JniWrapper.parseFrom(bytes);
} catch (InvalidProtocolBufferException | InterruptedException | ExecutionException e) {
if (retryCounter == 0) {
throw new RuntimeException(e);
} else {
enlargeBufferAndTimeout();
retryCounter--;
}
} catch (TimeoutException e) {
if (retryCounter == 0) {
LOG.error("Execute Query {} with {} timeout", queryType, params);
throw new RuntimeException(e);
} else {
enlargeBufferAndTimeout();
retryCounter--;
}
}
}
} finally {
unlockReadLock();
}
return null;
}
private void enlargeBufferAndTimeout() {
bufferSize *= 2;
largeBufferSize *= 2;
sharedBuffer = Runtime.getRuntime(libLakeSoulMetaData).getMemoryManager().allocateDirect(bufferSize);
largeSharedBuffer = Runtime.getRuntime(libLakeSoulMetaData).getMemoryManager().allocateDirect(largeBufferSize);
timeout += 5000L;
}
private void getReadLock() {
lock.readLock().lock();
}
private void unlockReadLock() {
lock.readLock().unlock();
}
private void getWriteLock() {
lock.writeLock().lock();
}
private void unlockWriteLock() {
lock.writeLock().unlock();
}
public Integer executeInsert(Integer insertType, JniWrapper jniWrapper) {
try {
getWriteLock();
int retryCounter = NATIVE_METADATA_MAX_RETRY_ATTEMPTS;
while (retryCounter >= 0) {
try {
final CompletableFuture<Integer> future = new CompletableFuture<>();
byte[] bytes = jniWrapper.toByteArray();
if (insertType < DAO_TYPE_TRANSACTION_INSERT_LIST_OFFSET)
sharedBuffer.put(0, bytes, 0, bytes.length);
else
largeSharedBuffer.put(0, bytes, 0, bytes.length);
getLibLakeSoulMetaData().execute_insert(
new ReferencedIntegerCallback((result, msg) -> {
if (msg.isEmpty()) {
future.complete(result);
} else {
future.completeExceptionally(new SQLException(msg));
}
}, getIntegerCallbackObjectReferenceManager()),
tokioRuntime,
tokioPostgresClient,
preparedStatement,
insertType,
insertType < DAO_TYPE_TRANSACTION_INSERT_LIST_OFFSET ? sharedBuffer.address() : largeSharedBuffer.address(),
bytes.length
);
return future.get(timeout, TimeUnit.MILLISECONDS);
} catch (InterruptedException | ExecutionException e) {
if (retryCounter == 0) {
throw new RuntimeException(e);
} else {
enlargeBufferAndTimeout();
retryCounter--;
}
} catch (TimeoutException e) {
if (retryCounter == 0) {
LOG.error("Execute Insert {} with {} timeout", insertType, jniWrapper);
throw new RuntimeException(e);
} else {
enlargeBufferAndTimeout();
retryCounter--;
}
}
}
} finally {
unlockWriteLock();
}
return -1;
}
public Integer executeUpdate(Integer updateType, List<String> params) {
try {
getWriteLock();
int retryCounter = NATIVE_METADATA_MAX_RETRY_ATTEMPTS;
while (retryCounter >= 0) {
try {
final CompletableFuture<Integer> future = new CompletableFuture<>();
getLibLakeSoulMetaData().execute_update(
new ReferencedIntegerCallback((result, msg) -> {
if (msg.isEmpty()) {
future.complete(result);
} else {
future.completeExceptionally(new SQLException(msg));
}
}, getIntegerCallbackObjectReferenceManager()),
tokioRuntime,
tokioPostgresClient,
preparedStatement,
updateType,
String.join(PARAM_DELIM, params)
);
return future.get(timeout, TimeUnit.MILLISECONDS);
} catch (InterruptedException | ExecutionException e) {
if (retryCounter == 0) {
throw new RuntimeException(e);
} else {
enlargeBufferAndTimeout();
retryCounter--;
}
} catch (TimeoutException e) {
if (retryCounter == 0) {
LOG.error("Execute Update {} with {} timeout", updateType, params);
throw new RuntimeException(e);
} else {
enlargeBufferAndTimeout();
retryCounter--;
}
}
}
} finally {
unlockWriteLock();
}
return -1;
}
public List<String> executeQueryScalar(Integer queryScalarType, List<String> params) {
try {
getReadLock();
int retryCounter = NATIVE_METADATA_MAX_RETRY_ATTEMPTS;
while (retryCounter >= 0) {
try {
final CompletableFuture<String> future = new CompletableFuture<>();
getLibLakeSoulMetaData().execute_query_scalar(
new ReferencedStringCallback((result, msg) -> {
if (msg.isEmpty()) {
future.complete(result);
} else {
future.completeExceptionally(new SQLException(msg));
}
}, getStringCallbackObjectReferenceManager()),
tokioRuntime,
tokioPostgresClient,
preparedStatement,
queryScalarType,
String.join(PARAM_DELIM, params)
);
String result = future.get(timeout, TimeUnit.MILLISECONDS);
if (result.isEmpty()) return Collections.emptyList();
return Arrays.stream(result.split(PARAM_DELIM)).collect(Collectors.toList());
} catch (InterruptedException | ExecutionException e) {
if (retryCounter == 0) {
throw new RuntimeException(e);
} else {
enlargeBufferAndTimeout();
retryCounter--;
}
} catch (TimeoutException e) {
if (retryCounter == 0) {
LOG.error("Execute QueryScalar {} with {} timeout", queryScalarType, params);
throw new RuntimeException(e);
} else {
enlargeBufferAndTimeout();
retryCounter--;
}
}
}
} finally {
unlockReadLock();
}
return Collections.emptyList();
}
public static Integer insert(NativeUtils.CodedDaoType insertType, JniWrapper jniWrapper) {
return getInstance().executeInsert(insertType.getCode(), jniWrapper);
}
public static JniWrapper query(NativeUtils.CodedDaoType queryType, List<String> params) {
if (params.size() != queryType.getParamsNum()) {
throw new RuntimeException("Params Num mismatch for " + queryType.name() + ", params=" + params + " paramsNum=" + params.size());
}
return getInstance().executeQuery(queryType.getCode(), params);
}
public static Integer update(NativeUtils.CodedDaoType updateType, List<String> params) {
if (params.size() != updateType.getParamsNum()) {
throw new RuntimeException("Params Num mismatch for " + updateType.name() + ", params=" + params + " paramsNum=" + params.size());
}
return getInstance().executeUpdate(updateType.getCode(), params);
}
public static List<String> queryScalar(NativeUtils.CodedDaoType queryScalarType, List<String> params) {
if (params.size() != queryScalarType.getParamsNum()) {
throw new RuntimeException("Params Num mismatch for " + queryScalarType.name() + ", params=" + params + " paramsNum=" + params.size());
}
return getInstance().executeQueryScalar(queryScalarType.getCode(), params);
}
public static int cleanMeta() {
final CompletableFuture<Integer> future = new CompletableFuture<>();
NativeMetadataJavaClient instance = getInstance();
instance.getWriteLock();
try {
instance.getLibLakeSoulMetaData().clean_meta_for_test(
new ReferencedIntegerCallback((result, msg) -> {
if (msg.isEmpty()) {
future.complete(result);
} else {
future.completeExceptionally(new SQLException(msg));
}
}, instance.getIntegerCallbackObjectReferenceManager()),
instance.tokioRuntime,
instance.tokioPostgresClient
);
return future.get(instance.timeout, TimeUnit.MILLISECONDS);
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
} catch (TimeoutException e) {
LOG.error("Clean Meta timeout");
throw new RuntimeException(e);
} finally {
instance.unlockWriteLock();
}
}
@Override
public void close() {
if (tokioRuntime != null) {
libLakeSoulMetaData.free_tokio_runtime(tokioRuntime);
tokioRuntime = null;
}
if (tokioPostgresClient != null) {
libLakeSoulMetaData.free_tokio_postgres_client(tokioPostgresClient);
tokioPostgresClient = null;
}
if (preparedStatement != null) {
libLakeSoulMetaData.free_prepared_statement(preparedStatement);
preparedStatement = null;
}
}
public static void closeAll() {
if (instance != null) {
instance.close();
instance = null;
}
}
}