提交 0385651f 编写于 作者: A Aljoscha Krettek

[scala] Add Scalastyle, use scalastyle-config.xml from Spark

上级 fd280981
.cache .cache
scalastyle-output.xml
.classpath .classpath
.idea .idea
.metadata .metadata
......
...@@ -71,7 +71,8 @@ object ConnectedComponents { ...@@ -71,7 +71,8 @@ object ConnectedComponents {
// assign the initial components (equal to the vertex id) // assign the initial components (equal to the vertex id)
val vertices = getVerticesDataSet(env).map { id => (id, id) } val vertices = getVerticesDataSet(env).map { id => (id, id) }
// undirected edges by emitting for each input edge the input edges itself and an inverted version // undirected edges by emitting for each input edge the input edges itself and an inverted
// version
val edges = getEdgesDataSet(env).flatMap { edge => Seq(edge, (edge._2, edge._1)) } val edges = getEdgesDataSet(env).flatMap { edge => Seq(edge, (edge._2, edge._1)) }
// open a delta iteration // open a delta iteration
...@@ -106,20 +107,22 @@ object ConnectedComponents { ...@@ -106,20 +107,22 @@ object ConnectedComponents {
private def parseParameters(args: Array[String]): Boolean = { private def parseParameters(args: Array[String]): Boolean = {
if (args.length > 0) { if (args.length > 0) {
fileOutput = true fileOutput = true
if (args.length == 4) { if (args.length == 4) {
verticesPath = args(0) verticesPath = args(0)
edgesPath = args(1) edgesPath = args(1)
outputPath = args(2) outputPath = args(2)
maxIterations = args(3).toInt maxIterations = args(3).toInt
} else { } else {
System.err.println("Usage: ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>") System.err.println("Usage: ConnectedComponents <vertices path> <edges path> <result path>" +
" <max number of iterations>")
false false
} }
} else { } else {
System.out.println("Executing Connected Components example with built-in default data.") System.out.println("Executing Connected Components example with built-in default data.")
System.out.println(" Provide parameters to read input data from a file.") System.out.println(" Provide parameters to read input data from a file.")
System.out.println(" Usage: ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>") System.out.println(" Usage: ConnectedComponents <vertices path> <edges path> <result path>" +
" <max number of iterations>")
} }
true true
} }
......
...@@ -65,111 +65,113 @@ import scala.collection.mutable ...@@ -65,111 +65,113 @@ import scala.collection.mutable
* *
*/ */
object EnumTrianglesBasic { object EnumTrianglesBasic {
def main(args: Array[String]) { def main(args: Array[String]) {
if (!parseParameters(args)) { if (!parseParameters(args)) {
return return
} }
// set up execution environment // set up execution environment
val env = ExecutionEnvironment.getExecutionEnvironment val env = ExecutionEnvironment.getExecutionEnvironment
// read input data // read input data
val edges = getEdgeDataSet(env) val edges = getEdgeDataSet(env)
// project edges by vertex id // project edges by vertex id
val edgesById = edges map(e => if (e.v1 < e.v2) e else Edge(e.v2, e.v1) ) val edgesById = edges map(e => if (e.v1 < e.v2) e else Edge(e.v2, e.v1) )
val triangles = edgesById val triangles = edgesById
// build triads // build triads
.groupBy("v1").sortGroup("v2", Order.ASCENDING).reduceGroup(new TriadBuilder()) .groupBy("v1").sortGroup("v2", Order.ASCENDING).reduceGroup(new TriadBuilder())
// filter triads // filter triads
.join(edgesById).where("v2", "v3").equalTo("v1", "v2") { (t, _) => t } .join(edgesById).where("v2", "v3").equalTo("v1", "v2") { (t, _) => t }
// emit result // emit result
if (fileOutput) { if (fileOutput) {
triangles.writeAsCsv(outputPath, "\n", ",") triangles.writeAsCsv(outputPath, "\n", ",")
} else { } else {
triangles.print() triangles.print()
} }
// execute program // execute program
env.execute("TriangleEnumeration Example") env.execute("TriangleEnumeration Example")
} }
// ************************************************************************* // *************************************************************************
// USER DATA TYPES // USER DATA TYPES
// ************************************************************************* // *************************************************************************
case class Edge(v1: Int, v2: Int) extends Serializable case class Edge(v1: Int, v2: Int) extends Serializable
case class Triad(v1: Int, v2: Int, v3: Int) extends Serializable case class Triad(v1: Int, v2: Int, v3: Int) extends Serializable
// ************************************************************************* // *************************************************************************
// USER FUNCTIONS // USER FUNCTIONS
// ************************************************************************* // *************************************************************************
/** /**
* Builds triads (triples of vertices) from pairs of edges that share a vertex. * Builds triads (triples of vertices) from pairs of edges that share a vertex. The first vertex
* The first vertex of a triad is the shared vertex, the second and third vertex are ordered by vertexId. * of a triad is the shared vertex, the second and third vertex are ordered by vertexId. Assumes
* Assumes that input edges share the first vertex and are in ascending order of the second vertex. * that input edges share the first vertex and are in ascending order of the second vertex.
*/ */
class TriadBuilder extends GroupReduceFunction[Edge, Triad] { class TriadBuilder extends GroupReduceFunction[Edge, Triad] {
val vertices = mutable.MutableList[Integer]() val vertices = mutable.MutableList[Integer]()
override def reduce(edges: java.lang.Iterable[Edge], out: Collector[Triad]) = { override def reduce(edges: java.lang.Iterable[Edge], out: Collector[Triad]) = {
// clear vertex list // clear vertex list
vertices.clear() vertices.clear()
// build and emit triads // build and emit triads
for(e <- edges.asScala) { for(e <- edges.asScala) {
// combine vertex with all previously read vertices // combine vertex with all previously read vertices
for(v <- vertices) { for(v <- vertices) {
out.collect(Triad(e.v1, v, e.v2)) out.collect(Triad(e.v1, v, e.v2))
} }
vertices += e.v2 vertices += e.v2
} }
} }
} }
// ************************************************************************* // *************************************************************************
// UTIL METHODS // UTIL METHODS
// ************************************************************************* // *************************************************************************
private def parseParameters(args: Array[String]): Boolean = { private def parseParameters(args: Array[String]): Boolean = {
if (args.length > 0) { if (args.length > 0) {
fileOutput = true fileOutput = true
if (args.length == 2) { if (args.length == 2) {
edgePath = args(0) edgePath = args(0)
outputPath = args(1) outputPath = args(1)
} else { } else {
System.err.println("Usage: EnumTriangleBasic <edge path> <result path>") System.err.println("Usage: EnumTriangleBasic <edge path> <result path>")
false false
} }
} else { } else {
System.out.println("Executing Enum Triangles Basic example with built-in default data.") System.out.println("Executing Enum Triangles Basic example with built-in default data.")
System.out.println(" Provide parameters to read input data from files.") System.out.println(" Provide parameters to read input data from files.")
System.out.println(" See the documentation for the correct format of input files.") System.out.println(" See the documentation for the correct format of input files.")
System.out.println(" Usage: EnumTriangleBasic <edge path> <result path>") System.out.println(" Usage: EnumTriangleBasic <edge path> <result path>")
} }
true true
} }
private def getEdgeDataSet(env: ExecutionEnvironment): DataSet[Edge] = { private def getEdgeDataSet(env: ExecutionEnvironment): DataSet[Edge] = {
if (fileOutput) { if (fileOutput) {
env.readCsvFile[Edge](edgePath, fieldDelimiter = ' ', includedFields = Array(0, 1)) env.readCsvFile[Edge](edgePath, fieldDelimiter = ' ', includedFields = Array(0, 1))
} else { } else {
val edges = EnumTrianglesData.EDGES.map{ case Array(v1, v2) => new Edge(v1.asInstanceOf[Int], v2.asInstanceOf[Int]) } val edges = EnumTrianglesData.EDGES.map {
env.fromCollection(edges) case Array(v1, v2) => new Edge(v1.asInstanceOf[Int], v2.asInstanceOf[Int])
} }
} env.fromCollection(edges)
}
}
private var fileOutput: Boolean = false
private var edgePath: String = null
private var outputPath: String = null private var fileOutput: Boolean = false
private var edgePath: String = null
private var outputPath: String = null
} }
...@@ -86,7 +86,9 @@ object PageRankBasic { ...@@ -86,7 +86,9 @@ object PageRankBasic {
// initialize lists // initialize lists
.map(e => AdjacencyList(e.sourceId, Array(e.targetId))) .map(e => AdjacencyList(e.sourceId, Array(e.targetId)))
// concatenate lists // concatenate lists
.groupBy("sourceId").reduce((l1, l2) => AdjacencyList(l1.sourceId, l1.targetIds ++ l2.targetIds)) .groupBy("sourceId").reduce {
(l1, l2) => AdjacencyList(l1.sourceId, l1.targetIds ++ l2.targetIds)
}
// start iteration // start iteration
val finalRanks = pagesWithRanks.iterateWithTermination(maxIterations) { val finalRanks = pagesWithRanks.iterateWithTermination(maxIterations) {
......
...@@ -23,90 +23,93 @@ import org.apache.flink.util.Collector ...@@ -23,90 +23,93 @@ import org.apache.flink.util.Collector
object TransitiveClosureNaive { object TransitiveClosureNaive {
def main (args: Array[String]): Unit = { def main (args: Array[String]): Unit = {
if (!parseParameters(args)) { if (!parseParameters(args)) {
return return
} }
val env = ExecutionEnvironment.getExecutionEnvironment val env = ExecutionEnvironment.getExecutionEnvironment
val edges = getEdgesDataSet(env) val edges = getEdgesDataSet(env)
val paths = edges.iterateWithTermination(maxIterations) { prevPaths: DataSet[(Long, Long)] => val paths = edges.iterateWithTermination(maxIterations) { prevPaths: DataSet[(Long, Long)] =>
val nextPaths = prevPaths val nextPaths = prevPaths
.join(edges) .join(edges)
.where(1).equalTo(0) { .where(1).equalTo(0) {
(left, right) => (left._1,right._2) (left, right) => (left._1,right._2)
} }
.union(prevPaths) .union(prevPaths)
.groupBy(0, 1) .groupBy(0, 1)
.reduce((l, r) => l) .reduce((l, r) => l)
val terminate = prevPaths val terminate = prevPaths
.coGroup(nextPaths) .coGroup(nextPaths)
.where(0).equalTo(0) { .where(0).equalTo(0) {
(prev, next, out: Collector[(Long, Long)]) => { (prev, next, out: Collector[(Long, Long)]) => {
val prevPaths = prev.toSet val prevPaths = prev.toSet
for (n <- next) for (n <- next)
if (!prevPaths.contains(n)) if (!prevPaths.contains(n)) out.collect(n)
out.collect(n) }
} }
} (nextPaths, terminate)
(nextPaths, terminate) }
}
if (fileOutput) {
if (fileOutput) paths.writeAsCsv(outputPath, "\n", " ")
paths.writeAsCsv(outputPath, "\n", " ") } else {
else paths.print()
paths.print() }
env.execute("Scala Transitive Closure Example") env.execute("Scala Transitive Closure Example")
} }
private var fileOutput: Boolean = false private var fileOutput: Boolean = false
private var edgesPath: String = null private var edgesPath: String = null
private var outputPath: String = null private var outputPath: String = null
private var maxIterations: Int = 10 private var maxIterations: Int = 10
private def parseParameters(programArguments: Array[String]): Boolean = { private def parseParameters(programArguments: Array[String]): Boolean = {
if (programArguments.length > 0) { if (programArguments.length > 0) {
fileOutput = true fileOutput = true
if (programArguments.length == 3) { if (programArguments.length == 3) {
edgesPath = programArguments(0) edgesPath = programArguments(0)
outputPath = programArguments(1) outputPath = programArguments(1)
maxIterations = Integer.parseInt(programArguments(2)) maxIterations = Integer.parseInt(programArguments(2))
} }
else { else {
System.err.println("Usage: TransitiveClosure <edges path> <result path> <max number of iterations>") System.err.println("Usage: TransitiveClosure <edges path> <result path> <max number of " +
return false "iterations>")
} return false
} }
else { }
System.out.println("Executing TransitiveClosure example with default parameters and built-in default data.") else {
System.out.println(" Provide parameters to read input data from files.") System.out.println("Executing TransitiveClosure example with default parameters and " +
System.out.println(" See the documentation for the correct format of input files.") "built-in default data.")
System.out.println(" Usage: TransitiveClosure <edges path> <result path> <max number of iterations>") System.out.println(" Provide parameters to read input data from files.")
} System.out.println(" See the documentation for the correct format of input files.")
true System.out.println(" Usage: TransitiveClosure <edges path> <result path> <max number of " +
} "iterations>")
}
private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[(Long, Long)] = { true
if (fileOutput) { }
env.readCsvFile[(Long, Long)](
edgesPath, private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[(Long, Long)] = {
fieldDelimiter = ' ', if (fileOutput) {
includedFields = Array(0, 1)) env.readCsvFile[(Long, Long)](
.map { x => (x._1, x._2)} edgesPath,
} fieldDelimiter = ' ',
else { includedFields = Array(0, 1))
val edgeData = ConnectedComponentsData.EDGES map { .map { x => (x._1, x._2)}
case Array(x, y) => (x.asInstanceOf[Long], y.asInstanceOf[Long]) }
} else {
env.fromCollection(edgeData) val edgeData = ConnectedComponentsData.EDGES map {
} case Array(x, y) => (x.asInstanceOf[Long], y.asInstanceOf[Long])
} }
} env.fromCollection(edgeData)
\ No newline at end of file }
}
}
...@@ -37,7 +37,7 @@ object PiEstimation { ...@@ -37,7 +37,7 @@ object PiEstimation {
val y = Math.random() val y = Math.random()
if (x * x + y * y < 1) 1L else 0L if (x * x + y * y < 1) 1L else 0L
} }
.reduce(_+_) .reduce(_ + _)
// ratio of samples in upper right quadrant vs total samples gives surface of upper // ratio of samples in upper right quadrant vs total samples gives surface of upper
// right quadrant, times 4 gives surface of whole unit circle, i.e. PI // right quadrant, times 4 gives surface of whole unit circle, i.e. PI
......
...@@ -62,101 +62,101 @@ import scala.collection.JavaConverters._ ...@@ -62,101 +62,101 @@ import scala.collection.JavaConverters._
*/ */
object LinearRegression { object LinearRegression {
def main(args: Array[String]) { def main(args: Array[String]) {
if (!parseParameters(args)) { if (!parseParameters(args)) {
return return
} }
val env = ExecutionEnvironment.getExecutionEnvironment val env = ExecutionEnvironment.getExecutionEnvironment
val data = getDataSet(env) val data = getDataSet(env)
val parameters = getParamsDataSet(env) val parameters = getParamsDataSet(env)
val result = parameters.iterate(numIterations) { currentParameters => val result = parameters.iterate(numIterations) { currentParameters =>
val newParameters = data val newParameters = data
.map(new SubUpdate).withBroadcastSet(currentParameters, "parameters") .map(new SubUpdate).withBroadcastSet(currentParameters, "parameters")
.reduce { (p1, p2) => .reduce { (p1, p2) =>
val result = p1._1 + p2._1 val result = p1._1 + p2._1
(result, p1._2 + p2._2) (result, p1._2 + p2._2)
} }
.map { x => x._1.div(x._2) } .map { x => x._1.div(x._2) }
newParameters newParameters
} }
if (fileOutput) { if (fileOutput) {
result.writeAsText(outputPath) result.writeAsText(outputPath)
} }
else { else {
result.print() result.print()
} }
env.execute("Scala Linear Regression example") env.execute("Scala Linear Regression example")
} }
/** /**
* A simple data sample, x means the input, and y means the target. * A simple data sample, x means the input, and y means the target.
*/ */
case class Data(var x: Double, var y: Double) case class Data(var x: Double, var y: Double)
/** /**
* A set of parameters -- theta0, theta1. * A set of parameters -- theta0, theta1.
*/ */
case class Params(theta0: Double, theta1: Double) { case class Params(theta0: Double, theta1: Double) {
def div(a: Int): Params = { def div(a: Int): Params = {
Params(theta0 / a, theta1 / a) Params(theta0 / a, theta1 / a)
} }
def +(other: Params) = { def + (other: Params) = {
Params(theta0 + other.theta0, theta1 + other.theta1) Params(theta0 + other.theta0, theta1 + other.theta1)
} }
} }
// ************************************************************************* // *************************************************************************
// USER FUNCTIONS // USER FUNCTIONS
// ************************************************************************* // *************************************************************************
/** /**
* Compute a single BGD type update for every parameters. * Compute a single BGD type update for every parameters.
*/ */
class SubUpdate extends RichMapFunction[Data, (Params, Int)] { class SubUpdate extends RichMapFunction[Data, (Params, Int)] {
private var parameter: Params = null private var parameter: Params = null
/** Reads the parameters from a broadcast variable into a collection. */ /** Reads the parameters from a broadcast variable into a collection. */
override def open(parameters: Configuration) { override def open(parameters: Configuration) {
val parameters = getRuntimeContext.getBroadcastVariable[Params]("parameters").asScala val parameters = getRuntimeContext.getBroadcastVariable[Params]("parameters").asScala
parameter = parameters.head parameter = parameters.head
} }
def map(in: Data): (Params, Int) = { def map(in: Data): (Params, Int) = {
val theta0 = val theta0 =
parameter.theta0 - 0.01 * ((parameter.theta0 + (parameter.theta1 * in.x)) - in.y) parameter.theta0 - 0.01 * ((parameter.theta0 + (parameter.theta1 * in.x)) - in.y)
val theta1 = val theta1 =
parameter.theta1 - 0.01 * (((parameter.theta0 + (parameter.theta1 * in.x)) - in.y) * in.x) parameter.theta1 - 0.01 * (((parameter.theta0 + (parameter.theta1 * in.x)) - in.y) * in.x)
(Params(theta0, theta1), 1) (Params(theta0, theta1), 1)
} }
} }
// ************************************************************************* // *************************************************************************
// UTIL METHODS // UTIL METHODS
// ************************************************************************* // *************************************************************************
private var fileOutput: Boolean = false private var fileOutput: Boolean = false
private var dataPath: String = null private var dataPath: String = null
private var outputPath: String = null private var outputPath: String = null
private var numIterations: Int = 10 private var numIterations: Int = 10
private def parseParameters(programArguments: Array[String]): Boolean = { private def parseParameters(programArguments: Array[String]): Boolean = {
if (programArguments.length > 0) { if (programArguments.length > 0) {
fileOutput = true fileOutput = true
if (programArguments.length == 3) { if (programArguments.length == 3) {
dataPath = programArguments(0) dataPath = programArguments(0)
outputPath = programArguments(1) outputPath = programArguments(1)
numIterations = programArguments(2).toInt numIterations = programArguments(2).toInt
} }
else { else {
System.err.println("Usage: LinearRegression <data path> <result path> <num iterations>") System.err.println("Usage: LinearRegression <data path> <result path> <num iterations>")
false false
} }
} }
else { else {
System.out.println("Executing Linear Regression example with default parameters and " + System.out.println("Executing Linear Regression example with default parameters and " +
"built-in default data.") "built-in default data.")
System.out.println(" Provide parameters to read input data from files.") System.out.println(" Provide parameters to read input data from files.")
...@@ -164,30 +164,30 @@ object LinearRegression { ...@@ -164,30 +164,30 @@ object LinearRegression {
System.out.println(" We provide a data generator to create synthetic input files for this " + System.out.println(" We provide a data generator to create synthetic input files for this " +
"program.") "program.")
System.out.println(" Usage: LinearRegression <data path> <result path> <num iterations>") System.out.println(" Usage: LinearRegression <data path> <result path> <num iterations>")
} }
true true
} }
private def getDataSet(env: ExecutionEnvironment): DataSet[Data] = { private def getDataSet(env: ExecutionEnvironment): DataSet[Data] = {
if (fileOutput) { if (fileOutput) {
env.readCsvFile[(Double, Double)]( env.readCsvFile[(Double, Double)](
dataPath, dataPath,
fieldDelimiter = ' ', fieldDelimiter = ' ',
includedFields = Array(0, 1)) includedFields = Array(0, 1))
.map { t => new Data(t._1, t._2) } .map { t => new Data(t._1, t._2) }
} }
else { else {
val data = LinearRegressionData.DATA map { val data = LinearRegressionData.DATA map {
case Array(x, y) => Data(x.asInstanceOf[Double], y.asInstanceOf[Double]) case Array(x, y) => Data(x.asInstanceOf[Double], y.asInstanceOf[Double])
} }
env.fromCollection(data) env.fromCollection(data)
} }
} }
private def getParamsDataSet(env: ExecutionEnvironment): DataSet[Params] = { private def getParamsDataSet(env: ExecutionEnvironment): DataSet[Params] = {
val params = LinearRegressionData.PARAMS map { val params = LinearRegressionData.PARAMS map {
case Array(x, y) => Params(x.asInstanceOf[Double], y.asInstanceOf[Double]) case Array(x, y) => Params(x.asInstanceOf[Double], y.asInstanceOf[Double])
} }
env.fromCollection(params) env.fromCollection(params)
} }
} }
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
...@@ -15,7 +15,6 @@ ...@@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.api.scala package org.apache.flink.api.scala
import org.apache.commons.lang3.Validate import org.apache.commons.lang3.Validate
...@@ -610,9 +609,9 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) { ...@@ -610,9 +609,9 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
new Keys.FieldPositionKeys[T](fieldIndices, set.getType,false)) new Keys.FieldPositionKeys[T](fieldIndices, set.getType,false))
} }
// public UnsortedGrouping<T> groupBy(String... fields) { // public UnsortedGrouping<T> groupBy(String... fields) {
// new UnsortedGrouping<T>(this, new Keys.ExpressionKeys<T>(fields, getType())); // new UnsortedGrouping<T>(this, new Keys.ExpressionKeys<T>(fields, getType()));
// } // }
// -------------------------------------------------------------------------------------------- // --------------------------------------------------------------------------------------------
// Joining // Joining
...@@ -807,7 +806,7 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) { ...@@ -807,7 +806,7 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
/** /**
* Creates a new DataSet by performing delta (or workset) iterations using the given step * Creates a new DataSet by performing delta (or workset) iterations using the given step
* function. At the beginning `this` DataSet is the solution set and `workset` is the Workset. * function. At the beginning `this` DataSet is the solution set and `workset` is the Workset.
* The iteration step function gets the current solution set and workset and must output the * The iteration step function gets the current solution set and workset and must output the
* delta for the solution set and the workset for the next iteration. * delta for the solution set and the workset for the next iteration.
* *
...@@ -825,6 +824,28 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) { ...@@ -825,6 +824,28 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
wrap(result) wrap(result)
} }
/**
* Creates a new DataSet by performing delta (or workset) iterations using the given step
* function. At the beginning `this` DataSet is the solution set and `workset` is the Workset.
* The iteration step function gets the current solution set and workset and must output the
* delta for the solution set and the workset for the next iteration.
*
* Note: The syntax of delta iterations are very likely going to change soon.
*/
def iterateDelta[R: ClassTag](workset: DataSet[R], maxIterations: Int, keyFields: Array[String])(
stepFunction: (DataSet[T], DataSet[R]) => (DataSet[T], DataSet[R])) = {
val fieldIndices = fieldNames2Indices(set.getType, keyFields)
val key = new FieldPositionKeys[T](fieldIndices, set.getType, false)
val iterativeSet = new DeltaIteration[T, R](
set.getExecutionEnvironment, set.getType, set, workset.set, key, maxIterations)
val (newSolution, newWorkset) = stepFunction(
wrap(iterativeSet.getSolutionSet),
wrap(iterativeSet.getWorkset))
val result = iterativeSet.closeWith(newSolution.set, newWorkset.set)
wrap(result)
}
// ------------------------------------------------------------------------------------------- // -------------------------------------------------------------------------------------------
// Custom Operators // Custom Operators
// ------------------------------------------------------------------------------------------- // -------------------------------------------------------------------------------------------
...@@ -919,4 +940,4 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) { ...@@ -919,4 +940,4 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
def printToErr(): DataSink[T] = { def printToErr(): DataSink[T] = {
output(new PrintingOutputFormat[T](true)) output(new PrintingOutputFormat[T](true))
} }
} }
\ No newline at end of file
...@@ -22,7 +22,7 @@ import java.util.UUID ...@@ -22,7 +22,7 @@ import java.util.UUID
import org.apache.commons.lang3.Validate import org.apache.commons.lang3.Validate
import org.apache.flink.api.common.JobExecutionResult import org.apache.flink.api.common.JobExecutionResult
import org.apache.flink.api.java.io._ import org.apache.flink.api.java.io._
import org.apache.flink.api.java.typeutils.{TupleTypeInfoBase, BasicTypeInfo} import org.apache.flink.api.java.typeutils.{ValueTypeInfo, TupleTypeInfoBase, BasicTypeInfo}
import org.apache.flink.api.scala.operators.ScalaCsvInputFormat import org.apache.flink.api.scala.operators.ScalaCsvInputFormat
import org.apache.flink.core.fs.Path import org.apache.flink.core.fs.Path
...@@ -30,7 +30,7 @@ import org.apache.flink.api.java.{ExecutionEnvironment => JavaEnv} ...@@ -30,7 +30,7 @@ import org.apache.flink.api.java.{ExecutionEnvironment => JavaEnv}
import org.apache.flink.api.common.io.{InputFormat, FileInputFormat} import org.apache.flink.api.common.io.{InputFormat, FileInputFormat}
import org.apache.flink.api.java.operators.DataSource import org.apache.flink.api.java.operators.DataSource
import org.apache.flink.types.TypeInformation import org.apache.flink.types.{StringValue, TypeInformation}
import org.apache.flink.util.{NumberSequenceIterator, SplittableIterator} import org.apache.flink.util.{NumberSequenceIterator, SplittableIterator}
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
...@@ -104,6 +104,27 @@ class ExecutionEnvironment(javaEnv: JavaEnv) { ...@@ -104,6 +104,27 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
wrap(source) wrap(source)
} }
/**
* Creates a DataSet of Strings produced by reading the given file line wise.
* This method is similar to [[readTextFile]], but it produces a DataSet with mutable
* [[StringValue]] objects, rather than Java Strings. StringValues can be used to tune
* implementations to be less object and garbage collection heavy.
*
* @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or
* "hdfs://host:port/file/path").
* @param charsetName The name of the character set used to read the file. Default is UTF-0
*/
def readTextFileWithValue(
filePath: String,
charsetName: String = "UTF-8"): DataSet[StringValue] = {
Validate.notNull(filePath, "The file path may not be null.")
val format = new TextValueInputFormat(new Path(filePath))
format.setCharsetName(charsetName)
val source = new DataSource[StringValue](
javaEnv, format, new ValueTypeInfo[StringValue](classOf[StringValue]))
wrap(source)
}
/** /**
* Creates a DataSet by reading the given CSV file. The type parameter must be used to specify * Creates a DataSet by reading the given CSV file. The type parameter must be used to specify
* a Tuple type that has the same number of fields as there are fields in the CSV file. If the * a Tuple type that has the same number of fields as there are fields in the CSV file. If the
...@@ -337,8 +358,9 @@ class ExecutionEnvironment(javaEnv: JavaEnv) { ...@@ -337,8 +358,9 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
def createProgramPlan(jobName: String = "") = { def createProgramPlan(jobName: String = "") = {
if (jobName.isEmpty) { if (jobName.isEmpty) {
javaEnv.createProgramPlan() javaEnv.createProgramPlan()
} else } else {
javaEnv.createProgramPlan(jobName) javaEnv.createProgramPlan(jobName)
}
} }
} }
...@@ -360,7 +382,8 @@ object ExecutionEnvironment { ...@@ -360,7 +382,8 @@ object ExecutionEnvironment {
* of parallelism of the local environment is the number of hardware contexts (CPU cores/threads). * of parallelism of the local environment is the number of hardware contexts (CPU cores/threads).
*/ */
def createLocalEnvironment( def createLocalEnvironment(
degreeOfParallelism: Int = Runtime.getRuntime.availableProcessors()) : ExecutionEnvironment = { degreeOfParallelism: Int = Runtime.getRuntime.availableProcessors())
: ExecutionEnvironment = {
val javaEnv = JavaEnv.createLocalEnvironment() val javaEnv = JavaEnv.createLocalEnvironment()
javaEnv.setDegreeOfParallelism(degreeOfParallelism) javaEnv.setDegreeOfParallelism(degreeOfParallelism)
new ExecutionEnvironment(javaEnv) new ExecutionEnvironment(javaEnv)
......
...@@ -267,7 +267,7 @@ private[flink] class GroupedDataSetImpl[T: ClassTag]( ...@@ -267,7 +267,7 @@ private[flink] class GroupedDataSetImpl[T: ClassTag](
} }
def reduceGroup[R: TypeInformation: ClassTag]( def reduceGroup[R: TypeInformation: ClassTag](
fun: (TraversableOnce[T], Collector[R]) => Unit): DataSet[R] = { fun: (TraversableOnce[T], Collector[R]) => Unit): DataSet[R] = {
Validate.notNull(fun, "Group reduce function must not be null.") Validate.notNull(fun, "Group reduce function must not be null.")
val reducer = new GroupReduceFunction[T, R] { val reducer = new GroupReduceFunction[T, R] {
def reduce(in: java.lang.Iterable[T], out: Collector[R]) { def reduce(in: java.lang.Iterable[T], out: Collector[R]) {
......
...@@ -33,8 +33,8 @@ import scala.reflect.ClassTag ...@@ -33,8 +33,8 @@ import scala.reflect.ClassTag
/** /**
* A specific [[DataSet]] that results from a `coGroup` operation. The result of a default coGroup is * A specific [[DataSet]] that results from a `coGroup` operation. The result of a default coGroup
* a tuple containing two arrays of values from the two sides of the coGroup. The result of the * is a tuple containing two arrays of values from the two sides of the coGroup. The result of the
* coGroup can be changed by specifying a custom coGroup function using the `apply` method or by * coGroup can be changed by specifying a custom coGroup function using the `apply` method or by
* providing a [[RichCoGroupFunction]]. * providing a [[RichCoGroupFunction]].
* *
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
...@@ -15,8 +15,6 @@ ...@@ -15,8 +15,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.api.scala.codegen package org.apache.flink.api.scala.codegen
private[flink] class Counter { private[flink] class Counter {
...@@ -29,4 +27,4 @@ private[flink] class Counter { ...@@ -29,4 +27,4 @@ private[flink] class Counter {
current current
} }
} }
} }
\ No newline at end of file
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
...@@ -24,8 +24,8 @@ private[flink] class MacroContextHolder[C <: Context](val c: C) ...@@ -24,8 +24,8 @@ private[flink] class MacroContextHolder[C <: Context](val c: C)
private[flink] object MacroContextHolder { private[flink] object MacroContextHolder {
def newMacroHelper[C <: Context](c: C) = new MacroContextHolder[c.type](c) def newMacroHelper[C <: Context](c: C) = new MacroContextHolder[c.type](c)
with TypeDescriptors[c.type] with TypeDescriptors[c.type]
with TypeAnalyzer[c.type] with TypeAnalyzer[c.type]
with TreeGen[c.type] with TreeGen[c.type]
with TypeInformationGen[c.type] with TypeInformationGen[c.type]
} }
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
...@@ -15,8 +15,6 @@ ...@@ -15,8 +15,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.api.scala.codegen package org.apache.flink.api.scala.codegen
import scala.language.implicitConversions import scala.language.implicitConversions
...@@ -50,10 +48,11 @@ private[flink] trait TreeGen[C <: Context] { this: MacroContextHolder[C] with Ty ...@@ -50,10 +48,11 @@ private[flink] trait TreeGen[C <: Context] { this: MacroContextHolder[C] with Ty
reify(c.Expr(source).splice.asInstanceOf[T]).tree reify(c.Expr(source).splice.asInstanceOf[T]).tree
def maybeMkAsInstanceOf[S: c.WeakTypeTag, T: c.WeakTypeTag](source: Tree): Tree = { def maybeMkAsInstanceOf[S: c.WeakTypeTag, T: c.WeakTypeTag](source: Tree): Tree = {
if (weakTypeOf[S] <:< weakTypeOf[T]) if (weakTypeOf[S] <:< weakTypeOf[T]) {
source source
else } else {
mkAsInstanceOf[T](source) mkAsInstanceOf[T](source)
}
} }
// def mkIdent(target: Symbol): Tree = Ident(target) setType target.tpe // def mkIdent(target: Symbol): Tree = Ident(target) setType target.tpe
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
...@@ -15,8 +15,6 @@ ...@@ -15,8 +15,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.api.scala.codegen package org.apache.flink.api.scala.codegen
import scala.Option.option2Iterable import scala.Option.option2Iterable
...@@ -107,10 +105,11 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] ...@@ -107,10 +105,11 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C]
appliedType(d.asType.toType, dArgs) appliedType(d.asType.toType, dArgs)
} }
if (dTpe <:< tpe) if (dTpe <:< tpe) {
Some(analyze(dTpe)) Some(analyze(dTpe))
else } else {
None None
}
} }
val errors = subTypes flatMap { _.findByType[UnsupportedDescriptor] } val errors = subTypes flatMap { _.findByType[UnsupportedDescriptor] }
...@@ -150,7 +149,11 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] ...@@ -150,7 +149,11 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C]
case true => case true =>
Some( Some(
FieldAccessor( FieldAccessor(
bGetter, bSetter, bTpe, isBaseField = true, analyze(bTpe.termSymbol.asMethod.returnType))) bGetter,
bSetter,
bTpe,
isBaseField = true,
analyze(bTpe.termSymbol.asMethod.returnType)))
case false => None case false => None
} }
} }
...@@ -167,7 +170,9 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] ...@@ -167,7 +170,9 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C]
desc match { desc match {
case desc @ BaseClassDescriptor(_, _, getters, baseSubTypes) => case desc @ BaseClassDescriptor(_, _, getters, baseSubTypes) =>
desc.copy(getters = getters map updateField, subTypes = baseSubTypes map wireBaseFields) desc.copy(
getters = getters map updateField,
subTypes = baseSubTypes map wireBaseFields)
case desc @ CaseClassDescriptor(_, _, _, _, getters) => case desc @ CaseClassDescriptor(_, _, _, _, getters) =>
desc.copy(getters = getters map updateField) desc.copy(getters = getters map updateField)
case _ => desc case _ => desc
...@@ -221,7 +226,7 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] ...@@ -221,7 +226,7 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C]
case errs @ _ :: _ => case errs @ _ :: _ =>
val msgs = errs flatMap { f => val msgs = errs flatMap { f =>
(f: @unchecked) match { (f: @unchecked) match {
case FieldAccessor(fgetter, _, _, _, UnsupportedDescriptor(_, fTpe, errors)) => case FieldAccessor(fgetter, _,_,_, UnsupportedDescriptor(_, fTpe, errors)) =>
errors map { err => "Field " + fgetter.name + ": " + fTpe + " - " + err } errors map { err => "Field " + fgetter.name + ": " + fTpe + " - " + err }
} }
} }
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
...@@ -15,8 +15,6 @@ ...@@ -15,8 +15,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.api.scala.codegen package org.apache.flink.api.scala.codegen
import scala.language.postfixOps import scala.language.postfixOps
...@@ -122,7 +120,8 @@ private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C] ...@@ -122,7 +120,8 @@ private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C]
id: Int, tpe: Type, override val getters: Seq[FieldAccessor], subTypes: Seq[UDTDescriptor]) id: Int, tpe: Type, override val getters: Seq[FieldAccessor], subTypes: Seq[UDTDescriptor])
extends UDTDescriptor { extends UDTDescriptor {
override def flatten = this +: ((getters flatMap { _.desc.flatten }) ++ (subTypes flatMap { _.flatten })) override def flatten =
this +: ((getters flatMap { _.desc.flatten }) ++ (subTypes flatMap { _.flatten }))
override def canBeKey = flatten forall { f => f.canBeKey } override def canBeKey = flatten forall { f => f.canBeKey }
override def select(path: List[String]): Seq[Option[UDTDescriptor]] = path match { override def select(path: List[String]): Seq[Option[UDTDescriptor]] = path match {
...@@ -151,7 +150,8 @@ private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C] ...@@ -151,7 +150,8 @@ private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C]
override def hashCode = (id, tpe, ctor, getters).hashCode override def hashCode = (id, tpe, ctor, getters).hashCode
override def equals(that: Any) = that match { override def equals(that: Any) = that match {
case CaseClassDescriptor(thatId, thatTpe, thatMutable, thatCtor, thatGetters) => case CaseClassDescriptor(thatId, thatTpe, thatMutable, thatCtor, thatGetters) =>
(id, tpe, mutable, ctor, getters).equals(thatId, thatTpe, thatMutable, thatCtor, thatGetters) (id, tpe, mutable, ctor, getters).equals(
thatId, thatTpe, thatMutable, thatCtor, thatGetters)
case _ => false case _ => false
} }
...@@ -164,7 +164,12 @@ private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C] ...@@ -164,7 +164,12 @@ private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C]
} }
} }
case class FieldAccessor(getter: Symbol, setter: Symbol, tpe: Type, isBaseField: Boolean, desc: UDTDescriptor) case class FieldAccessor(
getter: Symbol,
setter: Symbol,
tpe: Type,
isBaseField: Boolean,
desc: UDTDescriptor)
case class RecursiveDescriptor(id: Int, tpe: Type, refId: Int) extends UDTDescriptor { case class RecursiveDescriptor(id: Int, tpe: Type, refId: Int) extends UDTDescriptor {
override def flatten = Seq(this) override def flatten = Seq(this)
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
...@@ -15,8 +15,6 @@ ...@@ -15,8 +15,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.api.scala.codegen package org.apache.flink.api.scala.codegen
import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.common.typeutils.TypeSerializer
...@@ -116,14 +114,16 @@ private[flink] trait TypeInformationGen[C <: Context] { ...@@ -116,14 +114,16 @@ private[flink] trait TypeInformationGen[C <: Context] {
} }
} }
def mkValueTypeInfo[T <: Value : c.WeakTypeTag](desc: UDTDescriptor): c.Expr[TypeInformation[T]] = { def mkValueTypeInfo[T <: Value : c.WeakTypeTag](
desc: UDTDescriptor): c.Expr[TypeInformation[T]] = {
val tpeClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe))) val tpeClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe)))
reify { reify {
new ValueTypeInfo[T](tpeClazz.splice) new ValueTypeInfo[T](tpeClazz.splice)
} }
} }
def mkWritableTypeInfo[T <: Writable : c.WeakTypeTag](desc: UDTDescriptor): c.Expr[TypeInformation[T]] = { def mkWritableTypeInfo[T <: Writable : c.WeakTypeTag](
desc: UDTDescriptor): c.Expr[TypeInformation[T]] = {
val tpeClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe))) val tpeClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe)))
reify { reify {
new WritableTypeInfo[T](tpeClazz.splice) new WritableTypeInfo[T](tpeClazz.splice)
...@@ -153,7 +153,8 @@ private[flink] trait TypeInformationGen[C <: Context] { ...@@ -153,7 +153,8 @@ private[flink] trait TypeInformationGen[C <: Context] {
c.Expr[T](result) c.Expr[T](result)
} }
// def mkCaseClassTypeInfo[T: c.WeakTypeTag](desc: CaseClassDescriptor): c.Expr[TypeInformation[T]] = { // def mkCaseClassTypeInfo[T: c.WeakTypeTag](
// desc: CaseClassDescriptor): c.Expr[TypeInformation[T]] = {
// val tpeClazz = c.Expr[Class[_]](Literal(Constant(desc.tpe))) // val tpeClazz = c.Expr[Class[_]](Literal(Constant(desc.tpe)))
// val caseFields = mkCaseFields(desc) // val caseFields = mkCaseFields(desc)
// reify { // reify {
...@@ -178,10 +179,12 @@ private[flink] trait TypeInformationGen[C <: Context] { ...@@ -178,10 +179,12 @@ private[flink] trait TypeInformationGen[C <: Context] {
// c.Expr(mkMap(fields)) // c.Expr(mkMap(fields))
// } // }
// //
// protected def getFields(name: String, desc: UDTDescriptor): Seq[(String, UDTDescriptor)] = desc match { // protected def getFields(name: String, desc: UDTDescriptor): Seq[(String, UDTDescriptor)] =
// desc match {
// // Flatten product types // // Flatten product types
// case CaseClassDescriptor(_, _, _, _, getters) => // case CaseClassDescriptor(_, _, _, _, getters) =>
// getters filterNot { _.isBaseField } flatMap { f => getFields(name + "." + f.getter.name, f.desc) } // getters filterNot { _.isBaseField } flatMap {
// f => getFields(name + "." + f.getter.name, f.desc) }
// case _ => Seq((name, desc)) // case _ => Seq((name, desc))
// } // }
} }
\ No newline at end of file
...@@ -129,4 +129,4 @@ private[flink] object CrossDataSetImpl { ...@@ -129,4 +129,4 @@ private[flink] object CrossDataSetImpl {
new CrossDataSetImpl(crossOperator, leftSet, rightSet) new CrossDataSetImpl(crossOperator, leftSet, rightSet)
} }
} }
\ No newline at end of file
...@@ -228,4 +228,4 @@ private[flink] class UnfinishedJoinOperationImpl[T, O]( ...@@ -228,4 +228,4 @@ private[flink] class UnfinishedJoinOperationImpl[T, O](
new JoinDataSetImpl(joinOperator, leftSet.set, rightSet.set, leftKey, rightKey) new JoinDataSetImpl(joinOperator, leftSet.set, rightSet.set, leftKey, rightKey)
} }
} }
\ No newline at end of file
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
...@@ -44,4 +44,4 @@ package object scala { ...@@ -44,4 +44,4 @@ package object scala {
"supported on Case Classes (for now).") "supported on Case Classes (for now).")
} }
} }
} }
\ No newline at end of file
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
...@@ -15,7 +15,6 @@ ...@@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.api.scala.typeutils package org.apache.flink.api.scala.typeutils
import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer} import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
...@@ -15,7 +15,6 @@ ...@@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.api.scala.typeutils package org.apache.flink.api.scala.typeutils
import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.common.typeutils.TypeSerializer
......
...@@ -15,7 +15,6 @@ ...@@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.api.scala.typeutils package org.apache.flink.api.scala.typeutils
import org.apache.flink.api.java.typeutils.{AtomicType, TupleTypeInfoBase} import org.apache.flink.api.java.typeutils.{AtomicType, TupleTypeInfoBase}
...@@ -79,8 +78,8 @@ abstract class ScalaTupleTypeInfo[T <: Product]( ...@@ -79,8 +78,8 @@ abstract class ScalaTupleTypeInfo[T <: Product](
def getFieldIndices(fields: Array[String]): Array[Int] = { def getFieldIndices(fields: Array[String]): Array[Int] = {
val result = fields map { x => fieldNames.indexOf(x) } val result = fields map { x => fieldNames.indexOf(x) }
if (result.contains(-1)) { if (result.contains(-1)) {
throw new IllegalArgumentException("Fields '" + fields.mkString(", ") + "' are not valid for" + throw new IllegalArgumentException("Fields '" + fields.mkString(", ") +
" " + tupleClass + " with fields '" + fieldNames.mkString(", ") + "'.") "' are not valid for " + tupleClass + " with fields '" + fieldNames.mkString(", ") + "'.")
} }
result result
} }
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
...@@ -15,7 +15,6 @@ ...@@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.api.scala.typeutils package org.apache.flink.api.scala.typeutils
import scala.reflect.macros.Context import scala.reflect.macros.Context
......
...@@ -71,7 +71,9 @@ private[flink] abstract class UnfinishedKeyPairOperation[T, O, R]( ...@@ -71,7 +71,9 @@ private[flink] abstract class UnfinishedKeyPairOperation[T, O, R](
* This only works on a CaseClass [[DataSet]]. * This only works on a CaseClass [[DataSet]].
*/ */
def where(firstLeftField: String, otherLeftFields: String*) = { def where(firstLeftField: String, otherLeftFields: String*) = {
val fieldIndices = fieldNames2Indices(leftSet.set.getType, firstLeftField +: otherLeftFields.toArray) val fieldIndices = fieldNames2Indices(
leftSet.set.getType,
firstLeftField +: otherLeftFields.toArray)
val leftKey = new FieldPositionKeys[T](fieldIndices, leftSet.set.getType) val leftKey = new FieldPositionKeys[T](fieldIndices, leftSet.set.getType)
new HalfUnfinishedKeyPairOperation[T, O, R](this, leftKey) new HalfUnfinishedKeyPairOperation[T, O, R](this, leftKey)
......
/* /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one
* contributor license agreements. See the NOTICE file distributed with * or more contributor license agreements. See the NOTICE file
* this work for additional information regarding copyright ownership. * distributed with this work for additional information
* The ASF licenses this file to You under the Apache License, Version 2.0 * regarding copyright ownership. The ASF licenses this file
* (the "License"); you may not use this file except in compliance with * to you under the Apache License, Version 2.0 (the
* the License. You may obtain a copy of the License at * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
...@@ -14,7 +15,6 @@ ...@@ -14,7 +15,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.api.scala package org.apache.flink.api.scala
import java.lang.reflect.Method import java.lang.reflect.Method
...@@ -149,7 +149,8 @@ class ScalaAPICompletenessTest { ...@@ -149,7 +149,8 @@ class ScalaAPICompletenessTest {
checkMethods("SingleInputOperator", "DataSet", checkMethods("SingleInputOperator", "DataSet",
classOf[SingleInputOperator[_, _, _]], classOf[DataSet[_]]) classOf[SingleInputOperator[_, _, _]], classOf[DataSet[_]])
checkMethods("TwoInputOperator", "DataSet", classOf[TwoInputOperator[_, _, _, _]], classOf[DataSet[_]]) checkMethods("TwoInputOperator", "DataSet",
classOf[TwoInputOperator[_, _, _, _]], classOf[DataSet[_]])
checkMethods("SingleInputUdfOperator", "DataSet", checkMethods("SingleInputUdfOperator", "DataSet",
classOf[SingleInputUdfOperator[_, _, _]], classOf[DataSet[_]]) classOf[SingleInputUdfOperator[_, _, _]], classOf[DataSet[_]])
......
...@@ -46,7 +46,7 @@ class SemanticPropertiesTranslationTest { ...@@ -46,7 +46,7 @@ class SemanticPropertiesTranslationTest {
try { try {
val env = ExecutionEnvironment.getExecutionEnvironment val env = ExecutionEnvironment.getExecutionEnvironment
val input = env.fromElements((3l, "test", 42)) val input = env.fromElements((3L, "test", 42))
input.map(new WildcardConstantMapper[(Long, String, Int)]).print() input.map(new WildcardConstantMapper[(Long, String, Int)]).print()
val plan = env.createProgramPlan() val plan = env.createProgramPlan()
...@@ -83,7 +83,7 @@ class SemanticPropertiesTranslationTest { ...@@ -83,7 +83,7 @@ class SemanticPropertiesTranslationTest {
try { try {
val env = ExecutionEnvironment.getExecutionEnvironment val env = ExecutionEnvironment.getExecutionEnvironment
val input = env.fromElements((3l, "test", 42)) val input = env.fromElements((3L, "test", 42))
input.map(new IndividualConstantMapper[Long, String, Int]).print() input.map(new IndividualConstantMapper[Long, String, Int]).print()
val plan = env.createProgramPlan() val plan = env.createProgramPlan()
...@@ -120,8 +120,8 @@ class SemanticPropertiesTranslationTest { ...@@ -120,8 +120,8 @@ class SemanticPropertiesTranslationTest {
try { try {
val env = ExecutionEnvironment.getExecutionEnvironment val env = ExecutionEnvironment.getExecutionEnvironment
val input1 = env.fromElements((3l, "test")) val input1 = env.fromElements((3L, "test"))
val input2 = env.fromElements((3l, 3.1415)) val input2 = env.fromElements((3L, 3.1415))
input1.join(input2).where(0).equalTo(0)( input1.join(input2).where(0).equalTo(0)(
new ForwardingTupleJoin[Long, String, Long, Double]).print() new ForwardingTupleJoin[Long, String, Long, Double]).print()
......
...@@ -100,7 +100,9 @@ class DeltaIterationTranslationTest { ...@@ -100,7 +100,9 @@ class DeltaIterationTranslationTest {
assertEquals(classOf[IdentityMapper[_]], worksetMapper.getUserCodeWrapper.getUserCodeClass) assertEquals(classOf[IdentityMapper[_]], worksetMapper.getUserCodeWrapper.getUserCodeClass)
assertEquals(classOf[NextWorksetMapper], nextWorksetMapper.getUserCodeWrapper.getUserCodeClass) assertEquals(
classOf[NextWorksetMapper],
nextWorksetMapper.getUserCodeWrapper.getUserCodeClass)
if (solutionSetJoin.getUserCodeWrapper.getUserCodeObject.isInstanceOf[WrappingFunction[_]]) { if (solutionSetJoin.getUserCodeWrapper.getUserCodeObject.isInstanceOf[WrappingFunction[_]]) {
...@@ -203,7 +205,8 @@ class DeltaIterationTranslationTest { ...@@ -203,7 +205,8 @@ class DeltaIterationTranslationTest {
// val iteration: DeltaIteration[Tuple3[Double, Long, String], Tuple2[Double, // val iteration: DeltaIteration[Tuple3[Double, Long, String], Tuple2[Double,
// String]] = initialSolutionSet.iterateDelta(initialWorkSet, 10, 1) // String]] = initialSolutionSet.iterateDelta(initialWorkSet, 10, 1)
// try { // try {
// iteration.getWorkset.coGroup(iteration.getSolutionSet).where(1).equalTo(2).`with`(new DeltaIterationTranslationTest.SolutionWorksetCoGroup1) // iteration.getWorkset.coGroup(iteration.getSolutionSet).where(1).equalTo(2).`with`(
// new DeltaIterationTranslationTest.SolutionWorksetCoGroup1)
// fail("Accepted invalid program.") // fail("Accepted invalid program.")
// } // }
// catch { // catch {
...@@ -211,7 +214,8 @@ class DeltaIterationTranslationTest { ...@@ -211,7 +214,8 @@ class DeltaIterationTranslationTest {
// } // }
// } // }
// try { // try {
// iteration.getSolutionSet.coGroup(iteration.getWorkset).where(2).equalTo(1).`with`(new DeltaIterationTranslationTest.SolutionWorksetCoGroup2) // iteration.getSolutionSet.coGroup(iteration.getWorkset).where(2).equalTo(1).`with`(
// new DeltaIterationTranslationTest.SolutionWorksetCoGroup2)
// fail("Accepted invalid program.") // fail("Accepted invalid program.")
// } // }
// catch { // catch {
......
...@@ -121,7 +121,9 @@ class ReduceTranslationTest { ...@@ -121,7 +121,9 @@ class ReduceTranslationTest {
assertEquals(keyValueInfo, reducer.getOperatorInfo.getOutputType) assertEquals(keyValueInfo, reducer.getOperatorInfo.getOutputType)
assertEquals(keyValueInfo, keyProjector.getOperatorInfo.getInputType) assertEquals(keyValueInfo, keyProjector.getOperatorInfo.getInputType)
assertEquals(initialData.set.getType, keyProjector.getOperatorInfo.getOutputType) assertEquals(initialData.set.getType, keyProjector.getOperatorInfo.getOutputType)
assertEquals(classOf[KeyExtractingMapper[_, _]], keyExtractor.getUserCodeWrapper.getUserCodeClass) assertEquals(
classOf[KeyExtractingMapper[_, _]],
keyExtractor.getUserCodeWrapper.getUserCodeClass)
assertTrue(keyExtractor.getInput.isInstanceOf[GenericDataSourceBase[_, _]]) assertTrue(keyExtractor.getInput.isInstanceOf[GenericDataSourceBase[_, _]])
} }
catch { catch {
......
...@@ -34,8 +34,8 @@ class TupleSerializerTest { ...@@ -34,8 +34,8 @@ class TupleSerializerTest {
@Test @Test
def testTuple1Int(): Unit = { def testTuple1Int(): Unit = {
val testTuples = val testTuples = Array(Tuple1(42), Tuple1(1), Tuple1(0), Tuple1(-1), Tuple1(Int.MaxValue),
Array(Tuple1(42), Tuple1(1), Tuple1(0), Tuple1(-1), Tuple1(Int.MaxValue), Tuple1(Int.MinValue)) Tuple1(Int.MinValue))
runTests(testTuples) runTests(testTuples)
} }
......
...@@ -522,8 +522,10 @@ under the License. ...@@ -522,8 +522,10 @@ under the License.
<exclude>**/*.creole</exclude> <exclude>**/*.creole</exclude>
<exclude>CONTRIBUTORS</exclude> <exclude>CONTRIBUTORS</exclude>
<exclude>DEPENDENCIES</exclude> <exclude>DEPENDENCIES</exclude>
<!-- Build fiels --> <!-- Build files -->
<exclude>tools/maven/checkstyle.xml</exclude> <exclude>tools/maven/checkstyle.xml</exclude>
<exclude>tools/maven/scalastyle-config.xml</exclude>
<exclude>**/scalastyle-output.xml</exclude>
<exclude>tools/maven/suppressions.xml</exclude> <exclude>tools/maven/suppressions.xml</exclude>
<exclude>**/pom.xml</exclude> <exclude>**/pom.xml</exclude>
<exclude>**/pom.hadoop2.xml</exclude> <exclude>**/pom.hadoop2.xml</exclude>
...@@ -556,6 +558,29 @@ under the License. ...@@ -556,6 +558,29 @@ under the License.
<logViolationsToConsole>true</logViolationsToConsole> <logViolationsToConsole>true</logViolationsToConsole>
</configuration> </configuration>
</plugin> </plugin>
<plugin>
<groupId>org.scalastyle</groupId>
<artifactId>scalastyle-maven-plugin</artifactId>
<version>0.5.0</version>
<configuration>
<verbose>false</verbose>
<failOnViolation>true</failOnViolation>
<includeTestSourceDirectory>true</includeTestSourceDirectory>
<failOnWarning>false</failOnWarning>
<sourceDirectory>${basedir}/src/main/scala</sourceDirectory>
<testSourceDirectory>${basedir}/src/test/scala</testSourceDirectory>
<configLocation>tools/maven/scalastyle-config.xml</configLocation>
<outputFile>${project.basedir}/scalastyle-output.xml</outputFile>
<outputEncoding>UTF-8</outputEncoding>
</configuration>
<executions>
<execution>
<goals>
<goal>check</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin> <plugin>
<!-- just define the Java version to be used for compiling and plugins --> <!-- just define the Java version to be used for compiling and plugins -->
<groupId>org.apache.maven.plugins</groupId> <groupId>org.apache.maven.plugins</groupId>
......
<!--
~ Licensed to the Apache Software Foundation (ASF) under one or more
~ contributor license agreements. See the NOTICE file distributed with
~ this work for additional information regarding copyright ownership.
~ The ASF licenses this file to You under the Apache License, Version 2.0
~ (the "License"); you may not use this file except in compliance with
~ the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing, software
~ distributed under the License is distributed on an "AS IS" BASIS,
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
~ See the License for the specific language governing permissions and
~ limitations under the License.
-->
<!-- NOTE: This was taken and adapted from Apache Spark. -->
<!-- If you wish to turn off checking for a section of code, you can put a comment in the source
before and after the section, with the following syntax: -->
<!-- // scalastyle:off -->
<!-- ... -->
<!-- // naughty stuff -->
<!-- ... -->
<!-- // scalastyle:on -->
<scalastyle>
<name>Scalastyle standard configuration</name>
<check level="error" class="org.scalastyle.file.FileTabChecker" enabled="true"></check>
<!-- <check level="error" class="org.scalastyle.file.FileLengthChecker" enabled="true"> -->
<!-- <parameters> -->
<!-- <parameter name="maxFileLength"><![CDATA[800]]></parameter> -->
<!-- </parameters> -->
<!-- </check> -->
<check level="error" class="org.scalastyle.file.HeaderMatchesChecker" enabled="true">
<parameters>
<parameter name="header"><![CDATA[/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/]]></parameter>
</parameters>
</check>
<check level="error" class="org.scalastyle.scalariform.SpacesAfterPlusChecker" enabled="true"></check>
<check level="error" class="org.scalastyle.file.WhitespaceEndOfLineChecker" enabled="false"></check>
<check level="error" class="org.scalastyle.scalariform.SpacesBeforePlusChecker" enabled="true"></check>
<check level="error" class="org.scalastyle.file.FileLineLengthChecker" enabled="true">
<parameters>
<parameter name="maxLineLength"><![CDATA[100]]></parameter>
<parameter name="tabSize"><![CDATA[2]]></parameter>
<parameter name="ignoreImports">true</parameter>
</parameters>
</check>
<check level="error" class="org.scalastyle.scalariform.ClassNamesChecker" enabled="true">
<parameters>
<parameter name="regex"><![CDATA[[A-Z][A-Za-z]*]]></parameter>
</parameters>
</check>
<check level="error" class="org.scalastyle.scalariform.ObjectNamesChecker" enabled="true">
<parameters>
<parameter name="regex"><![CDATA[[A-Z][A-Za-z]*]]></parameter>
</parameters>
</check>
<check level="error" class="org.scalastyle.scalariform.PackageObjectNamesChecker" enabled="true">
<parameters>
<parameter name="regex"><![CDATA[^[a-z][A-Za-z]*$]]></parameter>
</parameters>
</check>
<check level="error" class="org.scalastyle.scalariform.EqualsHashCodeChecker" enabled="false"></check>
<!-- <check level="error" class="org.scalastyle.scalariform.IllegalImportsChecker" enabled="true"> -->
<!-- <parameters> -->
<!-- <parameter name="illegalImports"><![CDATA[sun._,java.awt._]]></parameter> -->
<!-- </parameters> -->
<!-- </check> -->
<check level="error" class="org.scalastyle.scalariform.ParameterNumberChecker" enabled="true">
<parameters>
<parameter name="maxParameters"><![CDATA[10]]></parameter>
</parameters>
</check>
<!-- <check level="error" class="org.scalastyle.scalariform.MagicNumberChecker" enabled="true"> -->
<!-- <parameters> -->
<!-- <parameter name="ignore"><![CDATA[-1,0,1,2,3]]></parameter> -->
<!-- </parameters> -->
<!-- </check> -->
<check level="error" class="org.scalastyle.scalariform.NoWhitespaceBeforeLeftBracketChecker" enabled="false"></check>
<check level="error" class="org.scalastyle.scalariform.NoWhitespaceAfterLeftBracketChecker" enabled="false"></check>
<!-- <check level="error" class="org.scalastyle.scalariform.ReturnChecker" enabled="true"></check> -->
<!-- <check level="error" class="org.scalastyle.scalariform.NullChecker" enabled="true"></check> -->
<!-- <check level="error" class="org.scalastyle.scalariform.NoCloneChecker" enabled="true"></check> -->
<!-- <check level="error" class="org.scalastyle.scalariform.NoFinalizeChecker" enabled="true"></check> -->
<!-- <check level="error" class="org.scalastyle.scalariform.CovariantEqualsChecker" enabled="true"></check> -->
<!-- <check level="error" class="org.scalastyle.scalariform.StructuralTypeChecker" enabled="true"></check> -->
<!-- <check level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> -->
<!-- <parameters> -->
<!-- <parameter name="regex"><![CDATA[println]]></parameter> -->
<!-- </parameters> -->
<!-- </check> -->
<!-- <check level="error" class="org.scalastyle.scalariform.NumberOfTypesChecker" enabled="true"> -->
<!-- <parameters> -->
<!-- <parameter name="maxTypes"><![CDATA[30]]></parameter> -->
<!-- </parameters> -->
<!-- </check> -->
<!-- <check level="error" class="org.scalastyle.scalariform.CyclomaticComplexityChecker" enabled="true"> -->
<!-- <parameters> -->
<!-- <parameter name="maximum"><![CDATA[10]]></parameter> -->
<!-- </parameters> -->
<!-- </check> -->
<check level="error" class="org.scalastyle.scalariform.UppercaseLChecker" enabled="true"></check>
<check level="error" class="org.scalastyle.scalariform.SimplifyBooleanExpressionChecker" enabled="false"></check>
<check level="error" class="org.scalastyle.scalariform.IfBraceChecker" enabled="true">
<parameters>
<parameter name="singleLineAllowed"><![CDATA[true]]></parameter>
<parameter name="doubleLineAllowed"><![CDATA[true]]></parameter>
</parameters>
</check>
<!-- <check level="error" class="org.scalastyle.scalariform.MethodLengthChecker" enabled="true"> -->
<!-- <parameters> -->
<!-- <parameter name="maxLength"><![CDATA[50]]></parameter> -->
<!-- </parameters> -->
<!-- </check> -->
<!-- <check level="error" class="org.scalastyle.scalariform.MethodNamesChecker" enabled="true"> -->
<!-- <parameters> -->
<!-- <parameter name="regex"><![CDATA[^[a-z][A-Za-z0-9]*$]]></parameter> -->
<!-- </parameters> -->
<!-- </check> -->
<!-- <check level="error" class="org.scalastyle.scalariform.NumberOfMethodsInTypeChecker" enabled="true"> -->
<!-- <parameters> -->
<!-- <parameter name="maxMethods"><![CDATA[30]]></parameter> -->
<!-- </parameters> -->
<!-- </check> -->
<!-- <check level="error" class="org.scalastyle.scalariform.PublicMethodsHaveTypeChecker" enabled="true"></check> -->
<check level="error" class="org.scalastyle.file.NewLineAtEofChecker" enabled="true"></check>
<check level="error" class="org.scalastyle.file.NoNewLineAtEofChecker" enabled="false"></check>
</scalastyle>
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册