File tree Expand file tree Collapse file tree 3 files changed +49
-2
lines changed
src/main/java/com/getindata/connectors/http/internal
table/lookup/querycreators Expand file tree Collapse file tree 3 files changed +49
-2
lines changed Original file line number Diff line number Diff line change 6
6
7
7
- Added support for OIDC Bearer tokens.
8
8
9
+ ### Fixed
10
+
11
+ - Ensured SerializationSchema is used in thread-safe way.
12
+
9
13
## [ 0.15.0] - 2024-07-30
10
14
11
15
### Added
Original file line number Diff line number Diff line change 15
15
import com .getindata .connectors .http .LookupQueryCreator ;
16
16
import com .getindata .connectors .http .LookupQueryCreatorFactory ;
17
17
import com .getindata .connectors .http .internal .table .lookup .LookupRow ;
18
+ import com .getindata .connectors .http .internal .utils .SynchronizedSerializationSchema ;
19
+ import static com .getindata .connectors .http .internal .table .lookup .HttpLookupConnectorOptions .ASYNC_POLLING ;
18
20
import static com .getindata .connectors .http .internal .table .lookup .HttpLookupConnectorOptions .LOOKUP_REQUEST_FORMAT ;
19
21
20
22
/**
@@ -47,8 +49,14 @@ public LookupQueryCreator createLookupQueryCreator(
47
49
queryFormatAwareConfiguration
48
50
);
49
51
50
- SerializationSchema <RowData > serializationSchema =
51
- encoder .createRuntimeEncoder (null , lookupRow .getLookupPhysicalRowDataType ());
52
+ final SerializationSchema <RowData > serializationSchema ;
53
+ if (readableConfig .get (ASYNC_POLLING )) {
54
+ serializationSchema = new SynchronizedSerializationSchema <>(
55
+ encoder .createRuntimeEncoder (null , lookupRow .getLookupPhysicalRowDataType ()));
56
+ } else {
57
+ serializationSchema =
58
+ encoder .createRuntimeEncoder (null , lookupRow .getLookupPhysicalRowDataType ());
59
+ }
52
60
53
61
return new GenericJsonQueryCreator (serializationSchema );
54
62
}
Original file line number Diff line number Diff line change
1
+ package com .getindata .connectors .http .internal .utils ;
2
+
3
+ import org .apache .flink .api .common .serialization .SerializationSchema ;
4
+
5
+ /**
6
+ * Decorator which ensures that underlying SerializationSchema is called in thread-safe way.
7
+ *
8
+ * @param <T> type
9
+ */
10
+ public class SynchronizedSerializationSchema <T > implements SerializationSchema <T > {
11
+
12
+ private final SerializationSchema <T > delegate ;
13
+
14
+ public SynchronizedSerializationSchema (SerializationSchema <T > delegate ) {
15
+ this .delegate = delegate ;
16
+ }
17
+
18
+ @ Override
19
+ public void open (InitializationContext context ) throws Exception {
20
+ doOpen (context );
21
+ }
22
+
23
+ private synchronized void doOpen (InitializationContext context ) throws Exception {
24
+ this .delegate .open (context );
25
+ }
26
+
27
+ @ Override
28
+ public byte [] serialize (T element ) {
29
+ return syncSerialize (element );
30
+ }
31
+
32
+ private synchronized byte [] syncSerialize (T element ) {
33
+ return delegate .serialize (element );
34
+ }
35
+ }
You can’t perform that action at this time.
0 commit comments