Skip to content

Commit

Permalink
[FLINK-8649] [scala api] Pass on TypeInfo in StreamExecutionEnvironme…
Browse files Browse the repository at this point in the history
…nt.createInput

This closes apache#5478.
  • Loading branch information
ggevay authored and zentol committed Mar 21, 2018
1 parent 129e215 commit 7e43f81
Show file tree
Hide file tree
Showing 2 changed files with 13 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import org.apache.flink.annotation.{Internal, Public, PublicEvolving}
import org.apache.flink.api.common.io.{FileInputFormat, FilePathFilter, InputFormat}
import org.apache.flink.api.common.restartstrategy.RestartStrategies.RestartStrategyConfiguration
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.java.typeutils.ResultTypeQueryable
import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer
import org.apache.flink.api.scala.ClosureCleaner
import org.apache.flink.configuration.Configuration
Expand Down Expand Up @@ -594,7 +595,11 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
*/
@PublicEvolving
def createInput[T: TypeInformation](inputFormat: InputFormat[T, _]): DataStream[T] =
asScalaStream(javaEnv.createInput(inputFormat))
if (inputFormat.isInstanceOf[ResultTypeQueryable[_]]) {
asScalaStream(javaEnv.createInput(inputFormat))
} else {
asScalaStream(javaEnv.createInput(inputFormat, implicitly[TypeInformation[T]]))
}

/**
* Create a DataStream using a user defined source function for arbitrary
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ package org.apache.flink.streaming.api.scala
import java.lang

import org.apache.flink.api.common.functions._
import org.apache.flink.api.java.io.ParallelIteratorInputFormat
import org.apache.flink.api.java.typeutils.TypeExtractor
import org.apache.flink.streaming.api.collector.selector.OutputSelector
import org.apache.flink.streaming.api.functions.{KeyedProcessFunction, ProcessFunction}
Expand Down Expand Up @@ -673,6 +674,12 @@ class DataStreamTest extends AbstractTestBase {
assert(sg.getIterationSourceSinkPairs.size() == 2)
}

@Test
def testCreateInputPassesOnTypeInfo(): Unit = {
StreamExecutionEnvironment.getExecutionEnvironment.createInput[Tuple1[Integer]](
new ParallelIteratorInputFormat[Tuple1[Integer]](null))
}

/////////////////////////////////////////////////////////////
// Utilities
/////////////////////////////////////////////////////////////
Expand Down

0 comments on commit 7e43f81

Please sign in to comment.