1919import com .datastax .oss .cdc .NativeSchemaWrapper ;
2020import com .datastax .oss .driver .api .core .CqlIdentifier ;
2121import com .datastax .oss .driver .api .core .cql .Row ;
22+ import com .datastax .oss .driver .api .core .data .TupleValue ;
2223import com .datastax .oss .driver .api .core .metadata .schema .ColumnMetadata ;
2324import com .datastax .oss .driver .api .core .metadata .schema .KeyspaceMetadata ;
2425import com .datastax .oss .driver .api .core .metadata .schema .TableMetadata ;
2728import com .datastax .oss .driver .api .core .type .ListType ;
2829import com .datastax .oss .driver .api .core .type .MapType ;
2930import com .datastax .oss .driver .api .core .type .SetType ;
31+ import com .datastax .oss .driver .api .core .type .TupleType ;
3032import com .datastax .oss .driver .api .core .type .UserDefinedType ;
3133import com .datastax .oss .protocol .internal .ProtocolConstants ;
3234import com .datastax .oss .pulsar .source .Converter ;
3335import lombok .extern .slf4j .Slf4j ;
3436import org .apache .avro .Schema ;
3537import org .apache .avro .SchemaBuilder ;
38+ import org .apache .avro .generic .GenericData ;
3639import org .apache .avro .generic .GenericRecord ;
3740import org .apache .pulsar .common .schema .SchemaType ;
3841
@@ -83,6 +86,11 @@ public AbstractNativeConverter(KeyspaceMetadata ksm, TableMetadata tm, List<Colu
8386 log .info ("Add vector schema {}={}" , field .name (), vectorSchema );
8487 }
8588 break ;
89+ case ProtocolConstants .DataType .TUPLE :
90+ Schema tupleSchema = dataTypeSchema (ksm , cm .getType ());
91+ subSchemas .put (field .name (), tupleSchema );
92+ log .info ("Add tuple schema {}={}" , field .name (), tupleSchema );
93+ break ;
8694 }
8795 }
8896 }
@@ -132,6 +140,8 @@ boolean isSupportedCqlType(DataType dataType) {
132140 return true ;
133141 case ProtocolConstants .DataType .CUSTOM :
134142 return dataType instanceof CqlVectorType ;
143+ case ProtocolConstants .DataType .TUPLE :
144+ return true ;
135145 }
136146 return false ;
137147 }
@@ -205,6 +215,9 @@ Schema dataTypeSchema(KeyspaceMetadata ksm, DataType dataType) {
205215 CqlVectorType vectorType = (CqlVectorType ) dataType ;
206216 return org .apache .avro .Schema .createArray (dataTypeSchema (ksm , vectorType .getSubtype ()));
207217 }
218+ case ProtocolConstants .DataType .TUPLE :
219+ TupleType tupleType = (TupleType ) dataType ;
220+ return buildTupleSchema (ksm , dataType .asCql (false , true ), tupleType , true );
208221 default :
209222 throw new UnsupportedOperationException ("Ignoring unsupported type=" + dataType .asCql (false , true ));
210223 }
@@ -229,6 +242,26 @@ Schema buildUDTSchema(KeyspaceMetadata ksm, String typeName, boolean optional) {
229242 return udtSchema ;
230243 }
231244
245+ Schema buildTupleSchema (KeyspaceMetadata ksm , String typeName , TupleType tupleType , boolean optional ) {
246+ List <Schema .Field > fieldSchemas = new ArrayList <>();
247+ int i = 0 ;
248+ for (DataType componentType : tupleType .getComponentTypes ()) {
249+ String fieldName = "index_" + i ;
250+ Schema .Field fieldSchema = fieldSchema (ksm , fieldName , componentType , optional );
251+ if (fieldSchema != null ) {
252+ fieldSchemas .add (fieldSchema );
253+ String path = typeName + "." + fieldName ;
254+ subSchemas .put (path , dataTypeSchema (ksm , componentType ));
255+ }
256+ i ++;
257+ }
258+ Schema tupleSchema = Schema .createRecord ("Tuple_" + Integer .toHexString (
259+ tupleType .asCql (false , true ).hashCode ()
260+ ), "CQL type " + typeName , ksm .getName ().asInternal (), false , fieldSchemas );
261+ subSchemas .put (typeName , tupleSchema );
262+ return tupleSchema ;
263+ }
264+
232265 String stringify (DataType dataType , Object value ) {
233266 switch (dataType .getProtocolCode ()) {
234267 case ProtocolConstants .DataType .ASCII :
@@ -270,8 +303,11 @@ String stringify(DataType dataType, Object value) {
270303 * @return the marshaled value; an epoch millisecond representation if the input is an {@link Instant}, or the original value otherwise
271304 */
272305 Object marshalCollectionValue (Object collectionValue ) {
273- if (collectionValue instanceof Instant ) {
274- return ((Instant )collectionValue ).toEpochMilli ();
306+ if (collectionValue instanceof Instant ) {
307+ return ((Instant ) collectionValue ).toEpochMilli ();
308+ }
309+ if (collectionValue instanceof TupleValue ) {
310+ return buildTupleValue ((TupleValue ) collectionValue );
275311 }
276312 return collectionValue ;
277313 }
@@ -286,9 +322,25 @@ Object marshalCollectionValue(Object collectionValue) {
286322 */
287323 Object marshalCollectionValue (Map .Entry <? super Object , ? super Object > entry ) {
288324 Object collectionValue = entry .getValue ();
289- if (collectionValue instanceof Instant ) {
290- return ((Instant )collectionValue ).toEpochMilli ();
325+ if (collectionValue instanceof Instant ) {
326+ return ((Instant ) collectionValue ).toEpochMilli ();
327+ }
328+ if (collectionValue instanceof TupleValue ) {
329+ return buildTupleValue ((TupleValue ) collectionValue );
291330 }
292331 return collectionValue ;
293332 }
333+
334+ GenericRecord buildTupleValue (TupleValue tupleValue ) {
335+ String typeName = tupleValue .getType ().asCql (false , true );
336+ Schema tupleSchema = subSchemas .get (typeName );
337+ if (tupleSchema == null ) {
338+ throw new IllegalStateException ("Missing tuple schema for " + typeName );
339+ }
340+ GenericRecord record = new GenericData .Record (tupleSchema );
341+ for (int i = 0 ; i < tupleValue .getType ().getComponentTypes ().size (); i ++) {
342+ record .put ("index_" + i , marshalCollectionValue (tupleValue .getObject (i )));
343+ }
344+ return record ;
345+ }
294346}
0 commit comments