Chesnay Schepler created FLINK-10235:
----------------------------------------

             Summary: KafkaTableSourceSinkFactoryBase#getFlinkKafkaPartitioner 
does not compile with Java 9
                 Key: FLINK-10235
                 URL: https://issues.apache.org/jira/browse/FLINK-10235
             Project: Flink
          Issue Type: Sub-task
          Components: Table API & SQL
    Affects Versions: 1.7.0
            Reporter: Chesnay Schepler
            Assignee: Chesnay Schepler
             Fix For: 1.7.0


The method compilation fails on java 9. The problem is that the 
{{CONNECTOR_SINK_PARTITIONER_VALUE_CUSTOM}} branch returns an 
{{Optional<Class<? extends FlinkKafkaPartitioner>>}}, but the method requires a 
{{Optional<FlinkKafkaPartitioner<Row>>}}.

{code}
private Optional<FlinkKafkaPartitioner<Row>> 
getFlinkKafkaPartitioner(DescriptorProperties descriptorProperties) {
        return descriptorProperties
                .getOptionalString(CONNECTOR_SINK_PARTITIONER)
                .flatMap((String partitionerString) -> {
                        switch (partitionerString) {
                                case CONNECTOR_SINK_PARTITIONER_VALUE_FIXED:
                                        return Optional.of(new 
FlinkFixedPartitioner<>());
                                case 
CONNECTOR_SINK_PARTITIONER_VALUE_ROUND_ROBIN:
                                        return Optional.empty();
                                case CONNECTOR_SINK_PARTITIONER_VALUE_CUSTOM:
                                        final Class<? extends 
FlinkKafkaPartitioner> partitionerClass =
                                                
descriptorProperties.getClass(CONNECTOR_SINK_PARTITIONER_CLASS, 
FlinkKafkaPartitioner.class);
                                        return 
Optional.of(InstantiationUtil.instantiate(partitionerClass));
                                default:
                                        throw new TableException("Unsupported 
sink partitioner. Validator should have checked that.");
                        }
                });
}
{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to