Skip to content

[SUPPORT] Enhance TypedProperties class to support running with Java 11 #4923

@stayrascal

Description

@stayrascal

Tips before filing an issue

  • Have you gone through our FAQs?

  • Join the mailing list to engage in conversations and get faster support at [email protected].

  • If you have triaged this as a bug, then file an issue directly.

Describe the problem you faced

Problem

I'm running Flink with Hudi in Java 11 environment(Even through I know Hudi might only support Java8/9/10 currently), But we cannot change the JDK version of our runtime. Based on this situation, We meet a NPE problem during HoodieWriteConfig.getIndexType(), the exception trace as follow

Caused by: java.lang.NullPointerException: Name is null
	at java.base/java.lang.Enum.valueOf(Enum.java:238)
	at org.apache.hudi.index.HoodieIndex$IndexType.valueOf(HoodieIndex.java:143)
	at org.apache.hudi.config.HoodieWriteConfig.getIndexType(HoodieWriteConfig.java:1347)
	at org.apache.hudi.index.FlinkHoodieIndexFactory.createIndex(FlinkHoodieIndexFactory.java:47)
	at org.apache.hudi.client.HoodieFlinkWriteClient.createIndex(HoodieFlinkWriteClient.java:105)
	at org.apache.hudi.client.BaseHoodieWriteClient.<init>(BaseHoodieWriteClient.java:145)
	at org.apache.hudi.client.BaseHoodieWriteClient.<init>(BaseHoodieWriteClient.java:131)
	at org.apache.hudi.client.HoodieFlinkWriteClient.<init>(HoodieFlinkWriteClient.java:96)
	at org.apache.hudi.util.StreamerUtil.createWriteClient(StreamerUtil.java:402)
	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.start(StreamWriteOperatorCoordinator.java:166)
	at org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder.start(OperatorCoordinatorHolder.java:194)
	at org.apache.flink.runtime.scheduler.DefaultOperatorCoordinatorHandler.startAllOperatorCoordinators(DefaultOperatorCoordinatorHandler.java:85)

But I test that it works in my local Java 8 environment. It seems that we implemented a TypedProperties class inherit from java.util.Properties class, and overwrite stringPropertyNames method & put method from this PR.

Source Code Analysis

There are some difference of Properties.putAll(Map<?,?> t) between Java 8 and Java 11.

Java 8: TypedProperties will use HashTable's putAll method, which will call TypedProperties.put method, and the latter will add the incoming keys in TypedProperties.keys field.

public synchronized void putAll(Map<? extends K, ? extends V> t) {
        for (Map.Entry<? extends K, ? extends V> e : t.entrySet())
            put(e.getKey(), e.getValue());
    }

Java 11: Properties overwrite putAll method, and delegate the incoming map to ConcurrentHashMap. It won't call TypedProperties.put method anymore, which lead the TypedProperties.keys fields miss some keys, and then it cause that we miss the hoodie.index.type configuration.

@Override
    public synchronized void putAll(Map<?, ?> t) {
        map.putAll(t);
    }

So we might need to overload putAll method to better support high Java version runtime.

To Reproduce

Steps to reproduce the behavior:

  1. run the follow UT case with Java 11
void testPutAllProperties() {
    Properties firstProp = new TypedProperties();
    firstProp.put("key0", "true");
    firstProp.put("key1", "false");
    firstProp.put("key2", "true");

    TypedProperties firstProperties = new TypedProperties(firstProp);
    assertTypeProperties(firstProperties, 0);

    TypedProperties secondProperties = new TypedProperties();
    secondProperties.put("key3", "true");
    secondProperties.put("key4", "false");
    secondProperties.put("key5", "true");
    assertTypeProperties(secondProperties, 3);

    TypedProperties thirdProperties = new TypedProperties();
    thirdProperties.putAll(firstProp);
    thirdProperties.putAll(secondProperties);

    assertEquals(3, firstProp.stringPropertyNames().size());
    assertEquals(3, secondProperties.stringPropertyNames().size());
    assertEquals(6, thirdProperties.stringPropertyNames().size());
  }

Expected behavior

the UT case show passed.
image

Environment Description

  • Java version: 11

  • Hudi version : Master branch

  • Spark version :

  • Hive version :

  • Hadoop version :

  • Storage (HDFS/S3/GCS..) :

  • Running on Docker? (yes/no) :
    no

Additional context

Add any other context about the problem here.

Stacktrace

Add the stacktrace of the error.

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions