Skip to content

Commit 2ddf120

Browse files
committed
remove debug printlines in test
1 parent 053267c commit 2ddf120

File tree

2 files changed

+9
-35
lines changed

2 files changed

+9
-35
lines changed

coral-schema/src/main/java/com/linkedin/coral/schema/avro/MergeHiveSchemaWithAvro.java

Lines changed: 2 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -177,19 +177,13 @@ public Schema mapKeyPartner(Schema partner) {
177177
@Override
178178
public Schema mapValuePartner(Schema partner) {
179179
Schema schema = SchemaUtilities.extractIfOption(partner);
180-
if (schema.getType() != Schema.Type.MAP) {
181-
return null;
182-
}
183-
return schema.getValueType();
180+
return (schema.getType() == Schema.Type.MAP) ? schema.getValueType() : null;
184181
}
185182

186183
@Override
187184
public Schema listElementPartner(Schema partner) {
188185
Schema schema = SchemaUtilities.extractIfOption(partner);
189-
if (schema.getType() != Schema.Type.ARRAY) {
190-
return null;
191-
}
192-
return schema.getElementType();
186+
return (schema.getType() == Schema.Type.ARRAY) ? schema.getElementType() : null;
193187
}
194188

195189
@Override

coral-schema/src/test/java/com/linkedin/coral/schema/avro/MergeHiveSchemaWithAvroTests.java

Lines changed: 7 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -273,26 +273,18 @@ public void shouldHandleSingleElementUnionsInAllTypes() {
273273

274274
// Construct Avro schema with single-element unions for primitives, array items, and map values
275275
Schema avro = struct("test_complex_array_table", "doc-test", "com.example.test", optional("id", Schema.Type.LONG),
276-
field("name", singleElementUnion(Schema.create(Schema.Type.STRING)), null, "unknown", null),
277-
field("active", singleElementUnion(Schema.create(Schema.Type.BOOLEAN)), null, false, null),
278-
optional("items", array(singleElementUnion(seu_arrayItemConfigSchema))),
279-
optional("metadata", map(singleElementUnion(seu_mapValueMetadataSchema))),
280-
optional("tags", array(singleElementUnion(Schema.create(Schema.Type.STRING)))));
276+
field("name", SchemaUtilities.wrapInSingleElementUnion(Schema.create(Schema.Type.STRING)), null, "unknown", null),
277+
field("active", SchemaUtilities.wrapInSingleElementUnion(Schema.create(Schema.Type.BOOLEAN)), null, false, null),
278+
optional("items", array(SchemaUtilities.wrapInSingleElementUnion(seu_arrayItemConfigSchema))),
279+
optional("metadata", map(SchemaUtilities.wrapInSingleElementUnion(seu_mapValueMetadataSchema))),
280+
optional("tags", array(SchemaUtilities.wrapInSingleElementUnion(Schema.create(Schema.Type.STRING)))));
281281

282282
// Expected schema after merge: single-element unions should be preserved
283283
// The structure of the Avro schema is maintained, including single-element unions
284284
// Expected output should match the input avro schema
285285
Schema expected = avro;
286286

287287
Schema actual = merge(hive, avro);
288-
289-
System.out.println("\n=== INPUT AVRO SCHEMA ===");
290-
System.out.println(avro.toString(true));
291-
System.out.println("\n=== EXPECTED OUTPUT SCHEMA ===");
292-
System.out.println(expected.toString(true));
293-
System.out.println("\n=== ACTUAL OUTPUT SCHEMA ===");
294-
System.out.println(actual.toString(true));
295-
System.out.println("\n=== END ===\n");
296288

297289
assertSchema(expected, actual);
298290
}
@@ -322,23 +314,15 @@ public void shouldHandleSingleElementUnionsWithHiveUnionType() {
322314
// Also includes a regular union for the status field (to test Hive uniontype compatibility)
323315
Schema avro = struct("test_union_compat", "doc-test", "com.example.test", optional("id", Schema.Type.LONG),
324316
required("status", union(Schema.Type.NULL, Schema.Type.STRING, Schema.Type.INT)),
325-
optional("items", array(singleElementUnion(seu_arrayItemSchema))),
326-
optional("metadata", map(singleElementUnion(seu_mapValueMetadataSchema))));
317+
optional("items", array(SchemaUtilities.wrapInSingleElementUnion(seu_arrayItemSchema))),
318+
optional("metadata", map(SchemaUtilities.wrapInSingleElementUnion(seu_mapValueMetadataSchema))));
327319

328320
// Expected schema: single-element unions should be preserved, regular union preserved
329321
// The Avro schema structure is maintained
330322
// Expected output should match the input avro schema
331323
Schema expected = avro;
332324

333325
Schema actual = merge(hive, avro);
334-
335-
System.out.println("\n=== INPUT AVRO SCHEMA (HiveUnionType test) ===");
336-
System.out.println(avro.toString(true));
337-
System.out.println("\n=== EXPECTED OUTPUT SCHEMA (HiveUnionType test) ===");
338-
System.out.println(expected.toString(true));
339-
System.out.println("\n=== ACTUAL OUTPUT SCHEMA (HiveUnionType test) ===");
340-
System.out.println(actual.toString(true));
341-
System.out.println("\n=== END ===\n");
342326

343327
assertSchema(expected, actual);
344328
}
@@ -353,10 +337,6 @@ private Schema union(Schema.Type... types) {
353337
return Schema.createUnion(Arrays.stream(types).map(Schema::create).collect(Collectors.toList()));
354338
}
355339

356-
private Schema singleElementUnion(Schema schema) {
357-
return Schema.createUnion(Arrays.asList(schema));
358-
}
359-
360340
private void assertSchema(Schema expected, Schema actual) {
361341
assertEquals(actual.toString(true), expected.toString(true));
362342
}

0 commit comments

Comments
 (0)