Repository: flink
Updated Branches:
  refs/heads/master 130511ffc -> 6ab96a7bb


[FLINK-3729] [table] Several SQL tests fail on Windows OS

This closes #2238.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6ab96a7b
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6ab96a7b
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6ab96a7b

Branch: refs/heads/master
Commit: 6ab96a7bbe5a65db1e0aa683380fbde55883607f
Parents: 130511f
Author: zentol <ches...@apache.org>
Authored: Wed Jul 13 13:51:05 2016 +0200
Committer: twalthr <twal...@apache.org>
Committed: Wed Jul 20 16:25:53 2016 +0200

----------------------------------------------------------------------
 .../flink/api/table/BatchTableEnvironment.scala |  9 +++++---
 .../flink/api/java/batch/ExplainTest.java       | 24 ++++++++++----------
 .../flink/api/scala/batch/ExplainTest.scala     | 24 ++++++++++----------
 .../flink/api/scala/batch/TableSinkITCase.scala | 12 ++++------
 .../api/scala/stream/TableSinkITCase.scala      |  2 +-
 5 files changed, 35 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6ab96a7b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala
----------------------------------------------------------------------
diff --git 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala
 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala
index 1ba13be..eb4c819 100644
--- 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala
+++ 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala
@@ -174,9 +174,12 @@ abstract class BatchTableEnvironment(
     val jasonSqlPlan = env.getExecutionPlan
     val sqlPlan = PlanJsonParser.getSqlExecutionPlan(jasonSqlPlan, extended)
 
-    s"== Abstract Syntax Tree ==\n" +
-    s"$ast\n" +
-    s"== Physical Execution Plan ==\n" +
+    s"== Abstract Syntax Tree ==" +
+    System.lineSeparator +
+    s"$ast" +
+    System.lineSeparator +
+    s"== Physical Execution Plan ==" +
+    System.lineSeparator +
     s"$sqlPlan"
 
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/6ab96a7b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/ExplainTest.java
----------------------------------------------------------------------
diff --git 
a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/ExplainTest.java
 
b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/ExplainTest.java
index 29cc0d9..747cd92 100644
--- 
a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/ExplainTest.java
+++ 
b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/ExplainTest.java
@@ -50,10 +50,10 @@ public class ExplainTest extends MultipleProgramsTestBase {
                        .fromDataSet(input, "a, b")
                        .filter("a % 2 = 0");
 
-               String result = tableEnv.explain(table);
+               String result = tableEnv.explain(table).replaceAll("\\r\\n", 
"\n");
                try (Scanner scanner = new Scanner(new File(testFilePath +
                        "../../src/test/scala/resources/testFilter0.out"))){
-                       String source = scanner.useDelimiter("\\A").next();
+                       String source = 
scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
                        assertEquals(source, result);
                }
        }
@@ -68,10 +68,10 @@ public class ExplainTest extends MultipleProgramsTestBase {
                        .fromDataSet(input, "a, b")
                        .filter("a % 2 = 0");
 
-               String result = tableEnv.explain(table, true);
+               String result = tableEnv.explain(table, 
true).replaceAll("\\r\\n", "\n");
                try (Scanner scanner = new Scanner(new File(testFilePath +
                        "../../src/test/scala/resources/testFilter1.out"))){
-                       String source = scanner.useDelimiter("\\A").next();
+                       String source = 
scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
                        assertEquals(source, result);
                }
        }
@@ -90,10 +90,10 @@ public class ExplainTest extends MultipleProgramsTestBase {
                        .where("b = d")
                        .select("a, c");
 
-               String result = tableEnv.explain(table);
+               String result = tableEnv.explain(table).replaceAll("\\r\\n", 
"\n");
                try (Scanner scanner = new Scanner(new File(testFilePath +
                        "../../src/test/scala/resources/testJoin0.out"))){
-                       String source = scanner.useDelimiter("\\A").next();
+                       String source = 
scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
                        assertEquals(source, result);
                }
        }
@@ -112,10 +112,10 @@ public class ExplainTest extends MultipleProgramsTestBase 
{
                        .where("b = d")
                        .select("a, c");
 
-               String result = tableEnv.explain(table, true);
+               String result = tableEnv.explain(table, 
true).replaceAll("\\r\\n", "\n");
                try (Scanner scanner = new Scanner(new File(testFilePath +
                        "../../src/test/scala/resources/testJoin1.out"))){
-                       String source = scanner.useDelimiter("\\A").next();
+                       String source = 
scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
                        assertEquals(source, result);
                }
        }
@@ -131,10 +131,10 @@ public class ExplainTest extends MultipleProgramsTestBase 
{
                Table table2 = tableEnv.fromDataSet(input2, "count, word");
                Table table = table1.unionAll(table2);
 
-               String result = tableEnv.explain(table);
+               String result = tableEnv.explain(table).replaceAll("\\r\\n", 
"\n");
                try (Scanner scanner = new Scanner(new File(testFilePath +
                        "../../src/test/scala/resources/testUnion0.out"))){
-                       String source = scanner.useDelimiter("\\A").next();
+                       String source = 
scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
                        assertEquals(source, result);
                }
        }
@@ -150,10 +150,10 @@ public class ExplainTest extends MultipleProgramsTestBase 
{
                Table table2 = tableEnv.fromDataSet(input2, "count, word");
                Table table = table1.unionAll(table2);
 
-               String result = tableEnv.explain(table, true);
+               String result = tableEnv.explain(table, 
true).replaceAll("\\r\\n", "\n");
                try (Scanner scanner = new Scanner(new File(testFilePath +
                        "../../src/test/scala/resources/testUnion1.out"))){
-                       String source = scanner.useDelimiter("\\A").next();
+                       String source = 
scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
                        assertEquals(source, result);
                }
        }

http://git-wip-us.apache.org/repos/asf/flink/blob/6ab96a7b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ExplainTest.scala
----------------------------------------------------------------------
diff --git 
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ExplainTest.scala
 
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ExplainTest.scala
index 77bb471..ab70ec5 100644
--- 
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ExplainTest.scala
+++ 
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ExplainTest.scala
@@ -39,9 +39,9 @@ class ExplainTest
       .toTable(tEnv, 'a, 'b)
       .filter("a % 2 = 0")
 
-    val result = tEnv.explain(table)
+    val result = tEnv.explain(table).replaceAll("\\r\\n", "\n")
     val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testFilter0.out").mkString
+      
"../../src/test/scala/resources/testFilter0.out").mkString.replaceAll("\\r\\n", 
"\n")
     assertEquals(result, source)
   }
 
@@ -54,9 +54,9 @@ class ExplainTest
       .toTable(tEnv, 'a, 'b)
       .filter("a % 2 = 0")
 
-    val result = tEnv.explain(table, true)
+    val result = tEnv.explain(table, true).replaceAll("\\r\\n", "\n")
     val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testFilter1.out").mkString
+      
"../../src/test/scala/resources/testFilter1.out").mkString.replaceAll("\\r\\n", 
"\n")
     assertEquals(result, source)
   }
 
@@ -69,9 +69,9 @@ class ExplainTest
     val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'c, 'd)
     val table = table1.join(table2).where("b = d").select("a, c")
 
-    val result = tEnv.explain(table)
+    val result = tEnv.explain(table).replaceAll("\\r\\n", "\n")
     val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testJoin0.out").mkString
+      
"../../src/test/scala/resources/testJoin0.out").mkString.replaceAll("\\r\\n", 
"\n")
     assertEquals(result, source)
   }
 
@@ -84,9 +84,9 @@ class ExplainTest
     val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'c, 'd)
     val table = table1.join(table2).where("b = d").select("a, c")
 
-    val result = tEnv.explain(table, true)
+    val result = tEnv.explain(table, true).replaceAll("\\r\\n", "\n")
     val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testJoin1.out").mkString
+      
"../../src/test/scala/resources/testJoin1.out").mkString.replaceAll("\\r\\n", 
"\n")
     assertEquals(result, source)
   }
 
@@ -99,9 +99,9 @@ class ExplainTest
     val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
     val table = table1.unionAll(table2)
 
-    val result = tEnv.explain(table)
+    val result = tEnv.explain(table).replaceAll("\\r\\n", "\n")
     val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testUnion0.out").mkString
+      
"../../src/test/scala/resources/testUnion0.out").mkString.replaceAll("\\r\\n", 
"\n")
     assertEquals(result, source)
   }
 
@@ -114,9 +114,9 @@ class ExplainTest
     val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
     val table = table1.unionAll(table2)
 
-    val result = tEnv.explain(table, true)
+    val result = tEnv.explain(table, true).replaceAll("\\r\\n", "\n")
     val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testUnion1.out").mkString
+      
"../../src/test/scala/resources/testUnion1.out").mkString.replaceAll("\\r\\n", 
"\n")
     assertEquals(result, source)
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6ab96a7b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala
----------------------------------------------------------------------
diff --git 
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala
 
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala
index dd0668c..407fa4c 100644
--- 
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala
+++ 
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala
@@ -20,17 +20,13 @@ package org.apache.flink.api.scala.batch
 
 import java.io.File
 
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
 import org.apache.flink.api.scala._
 import org.apache.flink.api.scala.table._
 import org.apache.flink.api.scala.ExecutionEnvironment
 import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.table.typeutils.RowTypeInfo
-import org.apache.flink.api.table.{Row, TableEnvironment}
-import org.apache.flink.api.table.sinks.{CsvTableSink, TableSink, 
BatchTableSink}
-import org.apache.flink.test.util.{TestBaseUtils, MultipleProgramsTestBase}
+import org.apache.flink.api.table.TableEnvironment
+import org.apache.flink.api.table.sinks.CsvTableSink
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
 import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
 import org.junit.Test
 import org.junit.runner.RunWith
@@ -47,7 +43,7 @@ class TableSinkITCase(
 
     val tmpFile = File.createTempFile("flink-table-sink-test", ".tmp")
     tmpFile.deleteOnExit()
-    val path = "file:///" + tmpFile.getAbsolutePath
+    val path = tmpFile.toURI.toString
 
     val env = ExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)

http://git-wip-us.apache.org/repos/asf/flink/blob/6ab96a7b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala
----------------------------------------------------------------------
diff --git 
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala
 
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala
index 9dfee11..f3eb87c 100644
--- 
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala
+++ 
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala
@@ -38,7 +38,7 @@ class TableSinkITCase extends 
StreamingMultipleProgramsTestBase {
 
     val tmpFile = File.createTempFile("flink-table-sink-test", ".tmp")
     tmpFile.deleteOnExit()
-    val path = "file:///" + tmpFile.getAbsolutePath
+    val path = tmpFile.toURI.toString
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)

Reply via email to