You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2017/02/15 22:54:41 UTC

[1/2] flink git commit: [FLINK-5771] [core] Fix multi-char delimiter detection in DelimitedInputFormat.

Repository: flink
Updated Branches:
  refs/heads/master 5fb267de6 -> ae0fbff76


[FLINK-5771] [core] Fix multi-char delimiter detection in DelimitedInputFormat.

- Add a test case to validate correct delimiter detection.
- Remove a couple of try-catch blocks from existing tests.

This closes #3316.


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

Branch: refs/heads/master
Commit: d6a97e480e294e4779eb320a8b57983122a6cf63
Parents: 5fb267d
Author: Fabian Hueske <fh...@apache.org>
Authored: Tue Feb 14 22:02:26 2017 +0100
Committer: Fabian Hueske <fh...@apache.org>
Committed: Wed Feb 15 23:53:09 2017 +0100

----------------------------------------------------------------------
 .../api/common/io/DelimitedInputFormat.java     |  11 +-
 .../api/common/io/DelimitedInputFormatTest.java | 313 +++++++++----------
 2 files changed, 164 insertions(+), 160 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d6a97e48/flink-core/src/main/java/org/apache/flink/api/common/io/DelimitedInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/DelimitedInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/DelimitedInputFormat.java
index 5c8dfc1..a83d45f 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/DelimitedInputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/DelimitedInputFormat.java
@@ -567,17 +567,24 @@ public abstract class DelimitedInputFormat<OT> extends FileInputFormat<OT> imple
 			int startPos = this.readPos;
 			int count;
 
+			// Search for next occurence of delimiter in read buffer.
 			while (this.readPos < this.limit && i < this.delimiter.length) {
-				if ((this.readBuffer[this.readPos++]) == this.delimiter[i]) {
+				if ((this.readBuffer[this.readPos]) == this.delimiter[i]) {
+					// Found the expected delimiter character. Continue looking for the next character of delimiter.
 					i++;
 				} else {
+					// Delimiter does not match.
+					// We have to reset the read position to the character after the first matching character
+					//   and search for the whole delimiter again.
+					readPos -= i;
 					i = 0;
 				}
+				readPos++;
 			}
 
 			// check why we dropped out
 			if (i == this.delimiter.length) {
-				// line end
+				// delimiter found
 				int totalBytesRead = this.readPos - startPos;
 				this.offset += countInWrapBuffer + totalBytesRead;
 				count = totalBytesRead - this.delimiter.length;

http://git-wip-us.apache.org/repos/asf/flink/blob/d6a97e48/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatTest.java
index 219365a..7ce0a2e 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatTest.java
@@ -173,32 +173,53 @@ public class DelimitedInputFormatTest {
 	}
 	
 	@Test
-	public void testReadCustomDelimiter() {
-		try {
-			final String myString = "my key|my val$$$my key2\n$$ctd.$$|my value2";
-			final FileInputSplit split = createTempFile(myString);
-			
-			final Configuration parameters = new Configuration();
-			
-			format.setDelimiter("$$$");
-			format.configure(parameters);
-			format.open(split);
-	
-			String first = format.nextRecord(null);
-			assertNotNull(first);
-			assertEquals("my key|my val", first);
-
-			String second = format.nextRecord(null);
-			assertNotNull(second);
-			assertEquals("my key2\n$$ctd.$$|my value2", second);
-			
-			assertNull(format.nextRecord(null));
-			assertTrue(format.reachedEnd());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+	public void testReadCustomDelimiter() throws IOException {
+		final String myString = "my key|my val$$$my key2\n$$ctd.$$|my value2";
+		final FileInputSplit split = createTempFile(myString);
+
+		final Configuration parameters = new Configuration();
+
+		format.setDelimiter("$$$");
+		format.configure(parameters);
+		format.open(split);
+
+		String first = format.nextRecord(null);
+		assertNotNull(first);
+		assertEquals("my key|my val", first);
+
+		String second = format.nextRecord(null);
+		assertNotNull(second);
+		assertEquals("my key2\n$$ctd.$$|my value2", second);
+
+		assertNull(format.nextRecord(null));
+		assertTrue(format.reachedEnd());
+	}
+
+	@Test
+	public void testMultiCharDelimiter() throws IOException {
+		final String myString = "www112xx1123yyy11123zzzzz1123";
+		final FileInputSplit split = createTempFile(myString);
+
+		final Configuration parameters = new Configuration();
+
+		format.setDelimiter("1123");
+		format.configure(parameters);
+		format.open(split);
+
+		String first = format.nextRecord(null);
+		assertNotNull(first);
+		assertEquals("www112xx", first);
+
+		String second = format.nextRecord(null);
+		assertNotNull(second);
+		assertEquals("yyy1", second);
+
+		String third = format.nextRecord(null);
+		assertNotNull(third);
+		assertEquals("zzzzz", third);
+
+		assertNull(format.nextRecord(null));
+		assertTrue(format.reachedEnd());
 	}
 
 	@Test
@@ -244,164 +265,140 @@ public class DelimitedInputFormatTest {
 	 * Tests that the records are read correctly when the split boundary is in the middle of a record.
 	 */
 	@Test
-	public void testReadOverSplitBoundariesUnaligned() {
-		try {
-			final String myString = "value1\nvalue2\nvalue3";
-			final FileInputSplit split = createTempFile(myString);
-			
-			FileInputSplit split1 = new FileInputSplit(0, split.getPath(), 0, split.getLength() / 2, split.getHostnames());
-			FileInputSplit split2 = new FileInputSplit(1, split.getPath(), split1.getLength(), split.getLength(), split.getHostnames());
-
-			final Configuration parameters = new Configuration();
-			
-			format.configure(parameters);
-			format.open(split1);
-			
-			assertEquals("value1", format.nextRecord(null));
-			assertEquals("value2", format.nextRecord(null));
-			assertNull(format.nextRecord(null));
-			assertTrue(format.reachedEnd());
-			
-			format.close();
-			format.open(split2);
+	public void testReadOverSplitBoundariesUnaligned() throws IOException {
+		final String myString = "value1\nvalue2\nvalue3";
+		final FileInputSplit split = createTempFile(myString);
 
-			assertEquals("value3", format.nextRecord(null));
-			assertNull(format.nextRecord(null));
-			assertTrue(format.reachedEnd());
-			
-			format.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		FileInputSplit split1 = new FileInputSplit(0, split.getPath(), 0, split.getLength() / 2, split.getHostnames());
+		FileInputSplit split2 = new FileInputSplit(1, split.getPath(), split1.getLength(), split.getLength(), split.getHostnames());
+
+		final Configuration parameters = new Configuration();
+
+		format.configure(parameters);
+		format.open(split1);
+
+		assertEquals("value1", format.nextRecord(null));
+		assertEquals("value2", format.nextRecord(null));
+		assertNull(format.nextRecord(null));
+		assertTrue(format.reachedEnd());
+
+		format.close();
+		format.open(split2);
+
+		assertEquals("value3", format.nextRecord(null));
+		assertNull(format.nextRecord(null));
+		assertTrue(format.reachedEnd());
+
+		format.close();
 	}
 
 	/**
 	 * Tests that the correct number of records is read when the split boundary is exact at the record boundary.
 	 */
 	@Test
-	public void testReadWithBufferSizeIsMultple() {
-		try {
-			final String myString = "aaaaaaa\nbbbbbbb\nccccccc\nddddddd\n";
-			final FileInputSplit split = createTempFile(myString);
-
-			FileInputSplit split1 = new FileInputSplit(0, split.getPath(), 0, split.getLength() / 2, split.getHostnames());
-			FileInputSplit split2 = new FileInputSplit(1, split.getPath(), split1.getLength(), split.getLength(), split.getHostnames());
+	public void testReadWithBufferSizeIsMultiple() throws IOException {
+		final String myString = "aaaaaaa\nbbbbbbb\nccccccc\nddddddd\n";
+		final FileInputSplit split = createTempFile(myString);
 
-			final Configuration parameters = new Configuration();
+		FileInputSplit split1 = new FileInputSplit(0, split.getPath(), 0, split.getLength() / 2, split.getHostnames());
+		FileInputSplit split2 = new FileInputSplit(1, split.getPath(), split1.getLength(), split.getLength(), split.getHostnames());
 
-			format.setBufferSize(2 * ((int) split1.getLength()));
-			format.configure(parameters);
+		final Configuration parameters = new Configuration();
 
-			String next;
-			int count = 0;
+		format.setBufferSize(2 * ((int) split1.getLength()));
+		format.configure(parameters);
 
-			// read split 1
-			format.open(split1);
-			while ((next = format.nextRecord(null)) != null) {
-				assertEquals(7, next.length());
-				count++;
-			}
-			assertNull(format.nextRecord(null));
-			assertTrue(format.reachedEnd());
-			format.close();
-			
-			// this one must have read one too many, because the next split will skipp the trailing remainder
-			// which happens to be one full record
-			assertEquals(3, count);
-
-			// read split 2
-			format.open(split2);
-			while ((next = format.nextRecord(null)) != null) {
-				assertEquals(7, next.length());
-				count++;
-			}
-			format.close();
+		String next;
+		int count = 0;
 
-			assertEquals(4, count);
+		// read split 1
+		format.open(split1);
+		while ((next = format.nextRecord(null)) != null) {
+			assertEquals(7, next.length());
+			count++;
 		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
+		assertNull(format.nextRecord(null));
+		assertTrue(format.reachedEnd());
+		format.close();
+
+		// this one must have read one too many, because the next split will skipp the trailing remainder
+		// which happens to be one full record
+		assertEquals(3, count);
+
+		// read split 2
+		format.open(split2);
+		while ((next = format.nextRecord(null)) != null) {
+			assertEquals(7, next.length());
+			count++;
 		}
+		format.close();
+
+		assertEquals(4, count);
 	}
 
 	@Test
-	public void testReadExactlyBufferSize() {
-		try {
-			final String myString = "aaaaaaa\nbbbbbbb\nccccccc\nddddddd\n";
-			
-			final FileInputSplit split = createTempFile(myString);
-			final Configuration parameters = new Configuration();
-			
-			format.setBufferSize((int) split.getLength());
-			format.configure(parameters);
-			format.open(split);
+	public void testReadExactlyBufferSize() throws IOException {
+		final String myString = "aaaaaaa\nbbbbbbb\nccccccc\nddddddd\n";
 
-			String next;
-			int count = 0;
-			while ((next = format.nextRecord(null)) != null) {
-				assertEquals(7, next.length());
-				count++;
-			}
-			assertNull(format.nextRecord(null));
-			assertTrue(format.reachedEnd());
+		final FileInputSplit split = createTempFile(myString);
+		final Configuration parameters = new Configuration();
 
-			format.close();
-			
-			assertEquals(4, count);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
+		format.setBufferSize((int) split.getLength());
+		format.configure(parameters);
+		format.open(split);
+
+		String next;
+		int count = 0;
+		while ((next = format.nextRecord(null)) != null) {
+			assertEquals(7, next.length());
+			count++;
 		}
+		assertNull(format.nextRecord(null));
+		assertTrue(format.reachedEnd());
+
+		format.close();
+
+		assertEquals(4, count);
 	}
 
 	@Test
-	public void testReadRecordsLargerThanBuffer() {
-		try {
-			final String myString = "aaaaaaaaaaaaaaaaaaaaa\n" +
-									"bbbbbbbbbbbbbbbbbbbbbbbbb\n" +
-									"ccccccccccccccccccc\n" +
-									"ddddddddddddddddddddddddddddddddddd\n";
-
-			final FileInputSplit split = createTempFile(myString);
-			FileInputSplit split1 = new FileInputSplit(0, split.getPath(), 0, split.getLength() / 2, split.getHostnames());
-			FileInputSplit split2 = new FileInputSplit(1, split.getPath(), split1.getLength(), split.getLength(), split.getHostnames());
-			
-			final Configuration parameters = new Configuration();
-
-			format.setBufferSize(8);
-			format.configure(parameters);
-
-			String next;
-			List<String> result = new ArrayList<String>();
-			
-			
-			format.open(split1);
-			while ((next = format.nextRecord(null)) != null) {
-				result.add(next);
-			}
-			assertNull(format.nextRecord(null));
-			assertTrue(format.reachedEnd());
-			format.close();
+	public void testReadRecordsLargerThanBuffer() throws IOException {
+		final String myString = "aaaaaaaaaaaaaaaaaaaaa\n" +
+								"bbbbbbbbbbbbbbbbbbbbbbbbb\n" +
+								"ccccccccccccccccccc\n" +
+								"ddddddddddddddddddddddddddddddddddd\n";
 
-			format.open(split2);
-			while ((next = format.nextRecord(null)) != null) {
-				result.add(next);
-			}
-			assertNull(format.nextRecord(null));
-			assertTrue(format.reachedEnd());
-			format.close();
-			
-			assertEquals(4, result.size());
-			assertEquals(Arrays.asList(myString.split("\n")), result);
+		final FileInputSplit split = createTempFile(myString);
+		FileInputSplit split1 = new FileInputSplit(0, split.getPath(), 0, split.getLength() / 2, split.getHostnames());
+		FileInputSplit split2 = new FileInputSplit(1, split.getPath(), split1.getLength(), split.getLength(), split.getHostnames());
+
+		final Configuration parameters = new Configuration();
+
+		format.setBufferSize(8);
+		format.configure(parameters);
+
+		String next;
+		List<String> result = new ArrayList<String>();
+
+
+		format.open(split1);
+		while ((next = format.nextRecord(null)) != null) {
+			result.add(next);
 		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
+		assertNull(format.nextRecord(null));
+		assertTrue(format.reachedEnd());
+		format.close();
+
+		format.open(split2);
+		while ((next = format.nextRecord(null)) != null) {
+			result.add(next);
 		}
+		assertNull(format.nextRecord(null));
+		assertTrue(format.reachedEnd());
+		format.close();
+
+		assertEquals(4, result.size());
+		assertEquals(Arrays.asList(myString.split("\n")), result);
 	}
 
 	static FileInputSplit createTempFile(String contents) throws IOException {


[2/2] flink git commit: [FLINK-5567] [table] Introduce and migrate current table statistics to FlinkStatistic.

Posted by fh...@apache.org.
[FLINK-5567] [table] Introduce and migrate current table statistics to FlinkStatistic.

This closes #3197.


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

Branch: refs/heads/master
Commit: ae0fbff76f327c008bdbf02cac0067bab507a04f
Parents: d6a97e4
Author: \u69ff\u745c <ji...@alibaba-inc.com>
Authored: Tue Jan 24 14:57:08 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Wed Feb 15 23:53:48 2017 +0100

----------------------------------------------------------------------
 .../flink/table/plan/schema/DataSetTable.scala  | 29 +------
 .../table/plan/schema/DataStreamTable.scala     |  6 +-
 .../flink/table/plan/schema/FlinkTable.scala    | 14 +++-
 .../table/plan/schema/TableSourceTable.scala    |  8 +-
 .../flink/table/plan/stats/FlinkStatistic.scala | 87 ++++++++++++++++++++
 5 files changed, 113 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ae0fbff7/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataSetTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataSetTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataSetTable.scala
index f8c6835..0ce2a87 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataSetTable.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataSetTable.scala
@@ -18,34 +18,13 @@
 
 package org.apache.flink.table.plan.schema
 
-import java.lang.Double
-import java.util
-import java.util.Collections
-
-import org.apache.calcite.rel.{RelCollation, RelDistribution}
-import org.apache.calcite.schema.Statistic
-import org.apache.calcite.util.ImmutableBitSet
 import org.apache.flink.api.java.DataSet
+import org.apache.flink.table.plan.stats.{FlinkStatistic, TableStats}
 
 class DataSetTable[T](
     val dataSet: DataSet[T],
     override val fieldIndexes: Array[Int],
-    override val fieldNames: Array[String])
-  extends FlinkTable[T](dataSet.getType, fieldIndexes, fieldNames) {
-
-  override def getStatistic: Statistic = {
-    new DefaultDataSetStatistic
-  }
-
-}
-
-class DefaultDataSetStatistic extends Statistic {
-
-  override def getRowCount: Double = 1000d
-
-  override def getCollations: util.List[RelCollation] = Collections.emptyList()
-
-  override def isKey(columns: ImmutableBitSet): Boolean = false
-
-  override def getDistribution: RelDistribution = null
+    override val fieldNames: Array[String],
+    override val statistic: FlinkStatistic = FlinkStatistic.of(TableStats(1000L)))
+  extends FlinkTable[T](dataSet.getType, fieldIndexes, fieldNames, statistic) {
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ae0fbff7/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala
index 0355fac..6ce6570 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala
@@ -19,10 +19,12 @@
 package org.apache.flink.table.plan.schema
 
 import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.plan.stats.FlinkStatistic
 
 class DataStreamTable[T](
     val dataStream: DataStream[T],
     override val fieldIndexes: Array[Int],
-    override val fieldNames: Array[String])
-  extends FlinkTable[T](dataStream.getType, fieldIndexes, fieldNames) {
+    override val fieldNames: Array[String],
+    override val statistic: FlinkStatistic = FlinkStatistic.UNKNOWN)
+  extends FlinkTable[T](dataStream.getType, fieldIndexes, fieldNames, statistic) {
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ae0fbff7/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala
index 971f54f..ea77061 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala
@@ -19,16 +19,19 @@
 package org.apache.flink.table.plan.schema
 
 import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
+import org.apache.calcite.schema.Statistic
 import org.apache.calcite.schema.impl.AbstractTable
 import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
 import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.table.api.TableException
 import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.plan.stats.FlinkStatistic
 
 abstract class FlinkTable[T](
     val typeInfo: TypeInformation[T],
     val fieldIndexes: Array[Int],
-    val fieldNames: Array[String])
+    val fieldNames: Array[String],
+    val statistic: FlinkStatistic)
   extends AbstractTable {
 
   if (fieldIndexes.length != fieldNames.length) {
@@ -64,4 +67,11 @@ abstract class FlinkTable[T](
     flinkTypeFactory.buildRowDataType(fieldNames, fieldTypes)
   }
 
+  /**
+    * Returns statistics of current table
+    *
+    * @return statistics of current table
+    */
+  override def getStatistic: Statistic = statistic
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ae0fbff7/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala
index 4f82f5e..a3851e3 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala
@@ -19,11 +19,15 @@
 package org.apache.flink.table.plan.schema
 
 import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.sources.TableSource
 
 /** Table which defines an external table via a [[TableSource]] */
-class TableSourceTable[T](val tableSource: TableSource[T])
+class TableSourceTable[T](
+    val tableSource: TableSource[T],
+    override val statistic: FlinkStatistic = FlinkStatistic.UNKNOWN)
   extends FlinkTable[T](
     typeInfo = tableSource.getReturnType,
     fieldIndexes = TableEnvironment.getFieldIndices(tableSource),
-    fieldNames = TableEnvironment.getFieldNames(tableSource))
+    fieldNames = TableEnvironment.getFieldNames(tableSource),
+    statistic)

http://git-wip-us.apache.org/repos/asf/flink/blob/ae0fbff7/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala
new file mode 100644
index 0000000..6f4ea00
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala
@@ -0,0 +1,87 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.table.plan.stats
+
+import java.lang.Double
+import java.util.{Collections, List}
+
+import org.apache.calcite.rel.{RelCollation, RelDistribution}
+import org.apache.calcite.schema.Statistic
+import org.apache.calcite.util.ImmutableBitSet
+
+/**
+  * The class provides statistics for a [[org.apache.flink.table.plan.schema.FlinkTable]].
+  *
+  * @param tableStats The table statistics.
+  */
+class FlinkStatistic(tableStats: Option[TableStats]) extends Statistic {
+
+  /**
+    * Returns the table statistics.
+    *
+    * @return The table statistics
+    */
+  def getTableStats: TableStats = tableStats.getOrElse(null)
+
+  /**
+    * Returns the stats of the specified the column.
+    *
+    * @param columnName The name of the column for which the stats are requested.
+    * @return The stats of the specified column.
+    */
+  def getColumnStats(columnName: String): ColumnStats = tableStats match {
+    case Some(tStats) => tStats.colStats.get(columnName)
+    case None => null
+  }
+
+  /**
+    * Returns the number of rows of the table.
+    *
+    * @return The number of rows of the table.
+    */
+  override def getRowCount: Double = tableStats match {
+    case Some(tStats) => tStats.rowCount.toDouble
+    case None => null
+  }
+
+  override def getCollations: List[RelCollation] = Collections.emptyList()
+
+  override def isKey(columns: ImmutableBitSet): Boolean = false
+
+  override def getDistribution: RelDistribution = null
+
+}
+
+/**
+  * Methods to create FlinkStatistic.
+  */
+object FlinkStatistic {
+
+  /** Represents a FlinkStatistic that knows nothing about a table */
+  val UNKNOWN: FlinkStatistic = new FlinkStatistic(None)
+
+  /**
+    * Returns a FlinkStatistic with given table statistics.
+    *
+    * @param tableStats The table statistics.
+    * @return The generated FlinkStatistic
+    */
+  def of(tableStats: TableStats): FlinkStatistic = new FlinkStatistic(Some(tableStats))
+
+}