You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by bl...@apache.org on 2019/08/12 18:21:20 UTC

[flink] branch master updated: [FLINK-13517][docs][hive] Restructure Hive Catalog documentation

This is an automated email from the ASF dual-hosted git repository.

bli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 6a63b34  [FLINK-13517][docs][hive] Restructure Hive Catalog documentation
6a63b34 is described below

commit 6a63b34463ce9fd1f66b0a26a599423d6500ee17
Author: Seth Wiesman <sj...@gmail.com>
AuthorDate: Wed Jul 31 12:30:36 2019 -0500

    [FLINK-13517][docs][hive] Restructure Hive Catalog documentation
    
    Hive documentation is currently spread across a number of pages and fragmented. In particular:
    
    - An example was added to getting-started/examples, however, this section is being removed
    - There is a dedicated page on hive integration but also a lot of hive specific information is on the catalog page
    
    This closes #9308.
---
 docs/dev/table/catalog.md                  | 366 -----------------------------
 docs/dev/table/catalogs.md                 | 181 ++++++++++++++
 docs/dev/table/catalogs.zh.md              | 181 ++++++++++++++
 docs/dev/table/hive/index.md               | 265 +++++++++++++++++++++
 docs/dev/table/hive/index.zh.md            | 265 +++++++++++++++++++++
 docs/dev/table/hive/read_write_hive.md     | 123 ++++++++++
 docs/dev/table/hive/read_write_hive.zh.md  | 123 ++++++++++
 docs/dev/table/hive_integration.md         |  70 ------
 docs/dev/table/hive_integration_example.md | 277 ----------------------
 9 files changed, 1138 insertions(+), 713 deletions(-)

diff --git a/docs/dev/table/catalog.md b/docs/dev/table/catalog.md
deleted file mode 100644
index acccd83..0000000
--- a/docs/dev/table/catalog.md
+++ /dev/null
@@ -1,366 +0,0 @@
----
-title: "Catalog"
-is_beta: true
-nav-parent_id: tableapi
-nav-pos: 100
----
-<!--
-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.
--->
-
-Catalogs provide metadata, such as names, schemas, statistics of tables, and information about how to access data stored in a database or other external systems. Once a catalog is registered within a `TableEnvironment`, all its meta-objects are accessible from the Table API and SQL queries.
-
-
-* This will be replaced by the TOC
-{:toc}
-
-
-Catalog Interface
------------------
-
-APIs are defined in `Catalog` interface. The interface defines a set of APIs to read and write catalog meta-objects such as database, tables, partitions, views, and functions.
-
-
-Catalog Meta-Objects Naming Structure
--------------------------------------
-
-Flink's catalogs use a strict two-level structure, that is, catalogs contain databases, and databases contain meta-objects. Thus, the full name of a meta-object is always structured as `catalogName`.`databaseName`.`objectName`.
-
-Each `TableEnvironment` has a `CatalogManager` to manager all registered catalogs. To ease access to meta-objects, `CatalogManager` has a concept of current catalog and current database. By setting current catalog and current database, users can use just the meta-object's name in their queries. This greatly simplifies user experience.
-
-For example, a previous query as
-
-```sql
-select * from mycatalog.mydb.myTable;
-```
-
-can be shortened to
-
-```sql
-select * from myTable;
-```
-
-To querying tables in a different database under the current catalog, users don't need to specify the catalog name. In our example, it would be
-
-```
-select * from mydb2.myTable2
-```
-
-`CatalogManager` always has a built-in `GenericInMemoryCatalog` named `default_catalog`, which has a built-in default database named `default_database`. If no other catalog and database are explicitly set, they will be the current catalog and current database by default. All temp meta-objects, such as those defined by `TableEnvironment#registerTable`  are registered to this catalog. 
-
-Users can set current catalog and database via `TableEnvironment.useCatalog(...)` and 
-`TableEnvironment.useDatabase(...)` in Table API, or `USE CATALOG ...` and `USE ...` in Flink SQL
- Client.
-
-
-Catalog Types
--------------
-
-## GenericInMemoryCatalog
-
-The default catalog; all meta-objects in this catalog are stored in memory, and be will be lost once the session shuts down.
-
-Its config entry value in SQL CLI yaml file is "generic_in_memory".
-
-## HiveCatalog
-
-Flink's `HiveCatalog` can read and write both Flink and Hive meta-objects using Hive Metastore as persistent storage.
-
-Its config entry value in SQL CLI yaml file is "hive".
-
-### Persist Flink meta-objects
-
-Historically, Flink meta-objects are only stored in memory and are per session based. That means users have to recreate all the meta-objects every time they start a new session.
-
-To maintain meta-objects across sessions, users can choose to use `HiveCatalog` to persist all of users' Flink streaming (unbounded-stream) and batch (bounded-stream) meta-objects. Because Hive Metastore is only used for storage, Hive itself may not understand Flink's meta-objects stored in the metastore.
-
-### Integrate Flink with Hive metadata
-
-The ultimate goal for integrating Flink with Hive metadata is that:
-
-1. Existing meta-objects, like tables, views, and functions, created by Hive or other Hive-compatible applications can be used by Flink
-
-2. Meta-objects created by `HiveCatalog` can be written back to Hive metastore such that Hive and other Hive-compatible applications can consume.
-
-### Supported Hive Versions
-
-Flink's `HiveCatalog` officially supports Hive 2.3.4 and 1.2.1.
-
-The Hive version is explicitly specified as a String, either by passing it to the constructor when creating `HiveCatalog` instances directly in Table API or specifying it in yaml config file in SQL CLI. The Hive version string are `2.3.4` and `1.2.1`.
-
-### Case Insensitive to Meta-Object Names
-
-Note that Hive Metastore stores meta-object names in lower cases. Thus, unlike `GenericInMemoryCatalog`, `HiveCatalog` is case-insensitive to meta-object names, and users need to be cautious on that.
-
-### Dependencies
-
-To use `HiveCatalog`, users need to include the following dependency jars.
-
-For Hive 2.3.4, users need:
-
-```
-// Hive dependencies
-
-- hive-exec-2.3.4.jar // contains hive-metastore-2.3.4
-
-
-// Hadoop dependencies
-- flink-shaded-hadoop-2-uber-2.7.5-1.8.0.jar
-- flink-hadoop-compatibility-{{site.version}}.jar
-
-```
-
-For Hive 1.2.1, users need:
-
-```
-// Hive dependencies
-
-- hive-metastore-1.2.1.jar
-- hive-exec-1.2.1.jar
-- libfb303-0.9.3.jar
-
-
-// Hadoop dependencies
-- flink-shaded-hadoop-2-uber-2.6.5-1.8.0.jar
-- flink-hadoop-compatibility-{{site.version}}.jar
-
-```
-
-If you don't have Hive dependencies at hand, they can be found at [mvnrepostory.com](https://mvnrepository.com):
-
-- [hive-exec](https://mvnrepository.com/artifact/org.apache.hive/hive-exec)
-- [hive-metastore](https://mvnrepository.com/artifact/org.apache.hive/hive-metastore)
-
-Note that users need to make sure the compatibility between their Hive versions and Hadoop versions. Otherwise, there may be potential problem, for example, Hive 2.3.4 is compiled against Hadoop 2.7.2, you may run into problems when using Hive 2.3.4 with Hadoop 2.4.
-
-
-### Data Type Mapping
-
-For both Flink and Hive tables, `HiveCatalog` stores table schemas by mapping them to Hive table schemas with Hive data types. Types are dynamically mapped back on read.
-
-Currently `HiveCatalog` supports most Flink data types with the following mapping:
-
-|  Flink Data Type  |  Hive Data Type  |
-|---|---|
-| CHAR(p)       |  CHAR(p)* |
-| VARCHAR(p)    |  VARCHAR(p)** |
-| STRING        |  STRING |
-| BOOLEAN       |  BOOLEAN |
-| TINYINT       |  TINYINT |
-| SMALLINT      |  SMALLINT |
-| INT           |  INT |
-| BIGINT        |  LONG |
-| FLOAT         |  FLOAT |
-| DOUBLE        |  DOUBLE |
-| DECIMAL(p, s) |  DECIMAL(p, s) |
-| DATE          |  DATE |
-| TIMESTAMP_WITHOUT_TIME_ZONE |  TIMESTAMP |
-| TIMESTAMP_WITH_TIME_ZONE |  N/A |
-| TIMESTAMP_WITH_LOCAL_TIME_ZONE |  N/A |
-| INTERVAL      |   N/A*** |
-| BINARY        |   N/A |
-| VARBINARY(p)  |   N/A |
-| BYTES         |   BINARY |
-| ARRAY\<E>     |  ARRAY\<E> |
-| MAP<K, V>     |  MAP<K, V> ****|
-| ROW           |  STRUCT |
-| MULTISET      |  N/A |
-
-
-Note that we only cover most commonly used data types for now.
-
-The following limitations in Hive's data types impact the mapping between Flink and Hive:
-
-\* maximum length is 255
-
-\** maximum length is 65535
-
-\*** `INTERVAL` type can not be mapped to hive `INTERVAL` for now.
-
-\**** Hive map key type only allows primitive types, while Flink map key can be any data type.
-
-## User-configured Catalog
-
-Catalogs are pluggable. Users can develop custom catalogs by implementing the `Catalog` interface, which defines a set of APIs for reading and writing catalog meta-objects such as database, tables, partitions, views, and functions.
-
-Catalog Registration
---------------------
-
-## Register Catalog in Table API
-
-To register a catalog in Table API, users can create a catalog instance and register it through `TableEnvironment.registerCatalog(name, catalog)`.
-
-## Register Catalog in SQL CLI
-
-To use pre-defined catalogs (`GenericInMemoryCatalog` and `HiveCatalog`) in SQL CLI, please refer to [SQL Clinet]({{ site.baseurl }}/dev/table/sqlClient.html)
-
-To use custom catalogs in SQL CLI, users should develop both a catalog and its corresponding catalog factory by implementing `Catalog` and `CatalogFactory` interfaces respectively.
-
-The catalog factory defines a set of properties for configuring the catalog when SQL CLI bootstraps. The set of properties will be passed to a discovery service where the service tries to match the properties to a `CatalogFactory` and initiate an corresponding catalog instance.
-
-
-{% top %}
-
-Catalog Modules
----------------
-
-`GenericInMemoryCatalog` is built into Flink's Table API.
-
-To use `HiveCatalog` in Flink Table API and SQL, users need to include `flink-connector-hive` jar in their projects.
-
-{% highlight xml %}
-<dependency>
-	<groupId>org.apache.flink</groupId>
-	<artifactId>flink-connector-hive_{{ site.scala_version_suffix }}</artifactId>
-	<version>{{site.version}}</version>
-</dependency>
-{% endhighlight %}
-
-
-Use Catalog
------------
-
-## Use HiveCatalog in Table API
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-//  ------ Register HiveCatalog ------
-TableEnvironment tEnv = ...
-
-// Register with Hive conf dir
-tEnv.registerCatalog(new HiveCatalog("myHive1", hiveConfDir));
-
-tEnv.listCatalogs();
-
-// ------ Set default catalog and database ------
-
-tEnv.useCatalog("myHive1")
-tEnv.useDatabase("myDb");
-
-// ------ Access Hive meta-objects ------
-
-// First get the catalog
-Catalog myHive1 = tEnv.getCatalog("myHive1");
-
-// Then read Hive meta-objects
-myHive1.listDatabases();
-myHive1.listTables("myDb");
-myHive1.listViews("myDb");
-
-ObjectPath myTablePath = new ObjectPath("myDb", "myTable");
-myHive1.getTable(myTablePath);
-myHive1.listPartitions(myTablePath);
-
-......
-
-{% endhighlight %}
-</div>
-
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-//  ------ Register HiveCatalog ------
-val tEnv = ...
-
-// Register with Hive conf dir
-tEnv.registerCatalog(new HiveCatalog("myHive1", hiveConfDir));
-
-tEnv.listCatalogs();
-
-// ------ Set default catalog and database ------
-
-tEnv.useCatalog("myHive1")
-tEnv.useDatabase("myDb");
-
-// ------ Access Hive meta-objects ------
-
-// First get the catalog
-val myHive1 = tEnv.getCatalog("myHive1");
-
-// Then read Hive meta-objects
-myHive1.listDatabases();
-myHive1.listTables("myDb");
-myHive1.listViews("myDb");
-
-val myTablePath = ew ObjectPath("myDb", "myTable");
-myHive1.getTable(myTablePath);
-myHive1.listPartitions(myTablePath);
-
-......
-
-
-{% endhighlight %}
-</div>
-</div>
-
-
-
-## Use HiveCatalog in Flink SQL Client
-
-Users can specify catalogs in the yaml config file of Flink SQL CLI. See [SQL Client]({{ site.baseurl }}/dev/table/sqlClient.html) for more details.
-
-```yaml
-catalogs:
-   - name: myHive1
-     type: hive
-     hive-conf-dir: ...
-   - name: myHive2
-     type: hive
-     hive-conf-dir: ...
-     default-database: ...
-```
-
-And below are a few example SQL commands accessing a Hive table.
-
-```sql
-Flink SQL> show catalogs;
-myHive1
-myHive2
-default_catalog
-
-# ------ Set default catalog and database ------
-
-Flink SQL> use catalog myHive1;
-Flink SQL> use myDb;
-
-# ------ Access Hive metadata ------
-
-Flink SQL> show databases;
-myDb
-
-Flink SQL> show tables;
-myTable
-
-Flink SQL> describe myTable;
-root
- |--    name: name
-    |-- type: StringType
-    |-- isNullable: true
- |--    name: value
-    |-- type: DoubleType
-    |-- isNullable: true
-
-Flink SQL> ......
-
-
-```
-
-For a full list of Flink SQL commands to access Hive meta-objects, see [FLINK SQL]({{ site.baseurl }}/dev/table/sql.html)
-
-
-{% top %}
diff --git a/docs/dev/table/catalogs.md b/docs/dev/table/catalogs.md
new file mode 100644
index 0000000..97c07e7
--- /dev/null
+++ b/docs/dev/table/catalogs.md
@@ -0,0 +1,181 @@
+---
+title: "Catalogs"
+is_beta: true
+nav-parent_id: tableapi
+nav-pos: 100
+---
+<!--
+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.
+-->
+
+Catalogs provide metadata, such as database, tables, partitions, views, and functions and information needed to access data stored in a database or other external systems.
+
+One of the most crucial aspects of data processing is managing metadata.
+It may be transient metadata like temporary tables, or UDF's registered against the table environment.
+Or permanent metadata, like a Hive Metastore. Catalogs provide a unified API for managing metadata and making it accessible from the Table API and SQL Queries. 
+
+* This will be replaced by the TOC
+{:toc}
+
+## Catalog Types
+
+### GenericInMemoryCatalog
+
+Flink sessions always have a built-in `GenericInMemoryCatalog` named `default_catalog`, which has a built-in default database named `default_database`.
+All temporary metadata, such tables defined using `TableEnvironment#registerTable` is registered to this catalog. 
+
+### HiveCatalog
+
+The `HiveCatalog` serves two purposes; as persistent storage for pure Flink metadata, and as an interface for reading and writing existing Hive metadata. 
+Flink's [Hive documentation]({{ site.baseurl }}/dev/table/hive/index.html) provides full details on setting up the catalog and interfacing with an existing Hive installation.
+
+
+{% warn %} The Hive Metastore stores all meta-object names in lower case. This is unlike `GenericInMemoryCatalog` which is case-sensitive
+
+### User-Defined Catalog
+
+Catalogs are pluggable and users can develop custom catalogs by implementing the `Catalog` interface.
+To use custom catalogs in SQL CLI, users should develop both a catalog and its corresponding catalog factory by implementing the `CatalogFactory` interface.
+
+The catalog factory defines a set of properties for configuring the catalog when the SQL CLI bootstraps.
+The set of properties will be passed to a discovery service where the service tries to match the properties to a `CatalogFactory` and initiate a corresponding catalog instance.
+
+## Catalog API
+
+### Registering a Catalog
+
+Users can register additional catalogs into an existing Flink session.
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java/Scala" markdown="1">
+{% highlight java %}
+tableEnv.registerCatalog(new CustomCatalog("myCatalog"));
+{% endhighlight %}
+</div>
+<div data-lang="YAML" markdown="1">
+
+All catalogs defined using YAML must provide a `type` property that specifies the type of catalog. 
+The following types are supported out of the box.
+
+<table class="table table-bordered">
+  <thead>
+    <tr>
+      <th class="text-center" style="width: 25%">Catalog</th>
+      <th class="text-center">Type Value</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+        <td class="text-center">GenericInMemory</td>
+        <td class="text-center">generic_in_memory</td>
+    </tr>
+    <tr>
+        <td class="text-center">Hive</td>
+        <td class="text-center">hive</td>
+    </tr>
+  </tbody>
+</table>
+
+{% highlight yaml %}
+catalogs:
+   - name: myCatalog
+     type: custom_catalog
+     hive-conf-dir: ...
+{% endhighlight %}
+</div>
+</div>
+
+### Changing the Current Catalog And Database
+
+Flink will always search for tables, views, and UDF's in the current catalog and database. 
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java/Scala" markdown="1">
+{% highlight java %}
+tableEnv.useCatalog("myCatalog");
+tableEnv.useDatabase("myDb");
+{% endhighlight %}
+</div>
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+Flink SQL> USE CATALOG myCatalog;
+Flink SQL> USE myDB;
+{% endhighlight %}
+</div>
+</div>
+
+Metadata from catalogs that are not the current catalog are accessible by providing fully qualified names in the form `catalog.database.object`.
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java/Scala" markdown="1">
+{% highlight java %}
+tableEnv.scan("not_the_current_catalog", "not_the_current_db", "my_table");
+{% endhighlight %}
+</div>
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+Flink SQL> SELECT * FROM not_the_current_catalog.not_the_current_db.my_table;
+{% endhighlight %}
+</div>
+</div>
+
+### List Available Catalogs
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java/Scala" markdown="1">
+{% highlight java %}
+tableEnv.listCatalogs();
+{% endhighlight %}
+</div>
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+Flink SQL> show catalogs;
+{% endhighlight %}
+</div>
+</div>
+
+
+### List Available Databases
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java/Scala" markdown="1">
+{% highlight java %}
+tableEnv.listDatabases();
+{% endhighlight %}
+</div>
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+Flink SQL> show databases;
+{% endhighlight %}
+</div>
+</div>
+
+### List Available Tables
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java/Scala" markdown="1">
+{% highlight java %}
+tableEnv.listTables();
+{% endhighlight %}
+</div>
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+Flink SQL> show tables;
+{% endhighlight %}
+</div>
+</div>
diff --git a/docs/dev/table/catalogs.zh.md b/docs/dev/table/catalogs.zh.md
new file mode 100644
index 0000000..97c07e7
--- /dev/null
+++ b/docs/dev/table/catalogs.zh.md
@@ -0,0 +1,181 @@
+---
+title: "Catalogs"
+is_beta: true
+nav-parent_id: tableapi
+nav-pos: 100
+---
+<!--
+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.
+-->
+
+Catalogs provide metadata, such as database, tables, partitions, views, and functions and information needed to access data stored in a database or other external systems.
+
+One of the most crucial aspects of data processing is managing metadata.
+It may be transient metadata like temporary tables, or UDF's registered against the table environment.
+Or permanent metadata, like a Hive Metastore. Catalogs provide a unified API for managing metadata and making it accessible from the Table API and SQL Queries. 
+
+* This will be replaced by the TOC
+{:toc}
+
+## Catalog Types
+
+### GenericInMemoryCatalog
+
+Flink sessions always have a built-in `GenericInMemoryCatalog` named `default_catalog`, which has a built-in default database named `default_database`.
+All temporary metadata, such tables defined using `TableEnvironment#registerTable` is registered to this catalog. 
+
+### HiveCatalog
+
+The `HiveCatalog` serves two purposes; as persistent storage for pure Flink metadata, and as an interface for reading and writing existing Hive metadata. 
+Flink's [Hive documentation]({{ site.baseurl }}/dev/table/hive/index.html) provides full details on setting up the catalog and interfacing with an existing Hive installation.
+
+
+{% warn %} The Hive Metastore stores all meta-object names in lower case. This is unlike `GenericInMemoryCatalog` which is case-sensitive
+
+### User-Defined Catalog
+
+Catalogs are pluggable and users can develop custom catalogs by implementing the `Catalog` interface.
+To use custom catalogs in SQL CLI, users should develop both a catalog and its corresponding catalog factory by implementing the `CatalogFactory` interface.
+
+The catalog factory defines a set of properties for configuring the catalog when the SQL CLI bootstraps.
+The set of properties will be passed to a discovery service where the service tries to match the properties to a `CatalogFactory` and initiate a corresponding catalog instance.
+
+## Catalog API
+
+### Registering a Catalog
+
+Users can register additional catalogs into an existing Flink session.
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java/Scala" markdown="1">
+{% highlight java %}
+tableEnv.registerCatalog(new CustomCatalog("myCatalog"));
+{% endhighlight %}
+</div>
+<div data-lang="YAML" markdown="1">
+
+All catalogs defined using YAML must provide a `type` property that specifies the type of catalog. 
+The following types are supported out of the box.
+
+<table class="table table-bordered">
+  <thead>
+    <tr>
+      <th class="text-center" style="width: 25%">Catalog</th>
+      <th class="text-center">Type Value</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+        <td class="text-center">GenericInMemory</td>
+        <td class="text-center">generic_in_memory</td>
+    </tr>
+    <tr>
+        <td class="text-center">Hive</td>
+        <td class="text-center">hive</td>
+    </tr>
+  </tbody>
+</table>
+
+{% highlight yaml %}
+catalogs:
+   - name: myCatalog
+     type: custom_catalog
+     hive-conf-dir: ...
+{% endhighlight %}
+</div>
+</div>
+
+### Changing the Current Catalog And Database
+
+Flink will always search for tables, views, and UDF's in the current catalog and database. 
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java/Scala" markdown="1">
+{% highlight java %}
+tableEnv.useCatalog("myCatalog");
+tableEnv.useDatabase("myDb");
+{% endhighlight %}
+</div>
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+Flink SQL> USE CATALOG myCatalog;
+Flink SQL> USE myDB;
+{% endhighlight %}
+</div>
+</div>
+
+Metadata from catalogs that are not the current catalog are accessible by providing fully qualified names in the form `catalog.database.object`.
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java/Scala" markdown="1">
+{% highlight java %}
+tableEnv.scan("not_the_current_catalog", "not_the_current_db", "my_table");
+{% endhighlight %}
+</div>
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+Flink SQL> SELECT * FROM not_the_current_catalog.not_the_current_db.my_table;
+{% endhighlight %}
+</div>
+</div>
+
+### List Available Catalogs
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java/Scala" markdown="1">
+{% highlight java %}
+tableEnv.listCatalogs();
+{% endhighlight %}
+</div>
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+Flink SQL> show catalogs;
+{% endhighlight %}
+</div>
+</div>
+
+
+### List Available Databases
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java/Scala" markdown="1">
+{% highlight java %}
+tableEnv.listDatabases();
+{% endhighlight %}
+</div>
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+Flink SQL> show databases;
+{% endhighlight %}
+</div>
+</div>
+
+### List Available Tables
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java/Scala" markdown="1">
+{% highlight java %}
+tableEnv.listTables();
+{% endhighlight %}
+</div>
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+Flink SQL> show tables;
+{% endhighlight %}
+</div>
+</div>
diff --git a/docs/dev/table/hive/index.md b/docs/dev/table/hive/index.md
new file mode 100644
index 0000000..a6dbefa
--- /dev/null
+++ b/docs/dev/table/hive/index.md
@@ -0,0 +1,265 @@
+---
+title: "Hive"
+nav-id: hive_tableapi
+nav-parent_id: tableapi
+nav-pos: 100
+is_beta: true
+nav-show_overview: true
+---
+<!--
+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.
+-->
+
+[Apache Hive](https://hive.apache.org/) has established itself as a focal point of the data warehousing ecosystem.
+It serves only not only a SQL engine for big data analytics and ETL, but also a data management platform, where data is discovered, defined, and evolved.
+
+Flink offers a two-fold integration with Hive.
+The first is to leverage Hive's Metastore as a persistent catalog for storing Flink specific metadata across sessions.
+The second is to offer Flink as an alternative engine for reading and writing Hive tables.
+
+The hive catalog is designed to be “out of the box” compatible with existing Hive installations.
+You do not need to modify your existing Hive Metastore or change the data placement or partitioning of your tables.
+
+* This will be replaced by the TOC
+{:toc}
+
+## Supported Hive Version's
+
+Flink supports Hive `2.3.4` and `1.2.1` and relies on Hive's compatibility guarantee's for other versions.
+
+### Depedencies 
+
+To integrate with Hive users need the following dependencies in their project.
+
+<div class="codetabs" markdown="1">
+<div data-lang="Hive 2.3.4" markdown="1">
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-hive_{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version}}</version>
+  <scope>provided</scope>
+</dependency>
+
+<!-- Hadoop Dependencies -->
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-hadoop-compatibility-{{site.version}}</artifactId>
+  <version>{{site.version}}</version>
+  <scope>provided</scope>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-shaded-hadoop-2-uber-2.7.5-{{site.version}}</artifactId>
+  <version>{{site.version}}</version>
+  <scope>provided</scope>
+</dependency>
+
+<!-- Hive Metastore -->
+<dependency>
+    <groupId>org.apache.hive</groupId>
+    <artifactId>hive-exec</artifactId>
+    <version>2.3.4</version>
+</dependency>
+{% endhighlight %}
+</div>
+
+<div data-lang="Hive 1.2.1" markdown="1">
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-hive_{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version}}</version>
+  <scope>provided</scope>
+</dependency>
+
+<!-- Hadoop Dependencies -->
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-hadoop-compatibility-{{site.version}}</artifactId>
+  <version>{{site.version}}</version>
+  <scope>provided</scope>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-shaded-hadoop-2-uber-2.7.5-{{site.version}}</artifactId>
+  <version>{{site.version}}</version>
+  <scope>provided</scope>
+</dependency>
+
+<!-- Hive Metastore -->
+<dependency>
+    <groupId>org.apache.hive</groupId>
+    <artifactId>hive-metastore</artifactId>
+    <version>1.2.1</version>
+</dependency>
+
+<dependency>
+    <groupId>org.apache.hive</groupId>
+    <artifactId>hive-exec</artifactId>
+    <version>1.2.1</version>
+</dependency>
+
+<dependency>
+    <groupId>org.apache.thrift</groupId>
+    <artifactId>libfb303</artifactId>
+    <version>0.9.3</version>
+</dependency>
+{% endhighlight %}
+</div>
+</div>
+
+## Connecting To Hive
+
+Connect to an existing Hive installation using the Hive [Catalog]({{ site.baseurl }}/dev/table/catalog.html) through the table environment or YAML configuration.
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java" markdown="1">
+{% highlight java %}
+
+String name            = "myhive";
+String defaultDatabase = "mydatabase";
+String hiveConfDir     = "/opt/hive-conf";
+String version         = "2.3.2"; // or 1.2.1
+
+HiveCatalog hive new HiveCatalog(name, defaultDatabase, hiveConfDir, version);
+tableEnv.registerCatalog(hive);
+{% endhighlight %}
+</div>
+<div data-lang="Scala" markdown="1">
+{% highlight scala %}
+
+val name            = "myhive"
+val defaultDatabase = "mydatabase"
+val hiveConfDir     = "/opt/hive-conf"
+val version         = "2.3.2" // or 1.2.1
+
+val hive new HiveCatalog(name, defaultDatabase, hiveConfDir, version)
+tableEnv.registerCatalog(hive)
+{% endhighlight %}
+</div>
+<div data-lang="YAML" markdown="1">
+{% highlight yaml %}
+catalogs:
+   - name: myhive
+     type: hive
+     property-version: 1
+     hive-conf-dir: /opt/hive-conf
+     hive-version: 2.3.4 # or 1.2.1
+{% endhighlight %}
+</div>
+</div>
+
+## Supported Types
+
+Currently `HiveCatalog` supports most Flink data types with the following mapping:
+
+<table class="table table-bordered">
+  <thead>
+    <tr>
+      <th class="text-center" style="width: 25%">Flink Data Type</th>
+      <th class="text-center">Hive Data Type</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+        <td class="text-center">CHAR(p)</td>
+        <td class="text-center">CHAR(p)</td>
+    </tr>
+    <tr>
+        <td class="text-center">VARCHAR(p)</td>
+        <td class="text-center">VARCHAR(p)</td>
+    </tr>
+        <tr>
+        <td class="text-center">STRING</td>
+        <td class="text-center">STRING</td>
+    </tr>
+    <tr>
+        <td class="text-center">BOOLEAN</td>
+        <td class="text-center">BOOLEAN</td>
+    </tr>
+    <tr>
+        <td class="text-center">TINYINT</td>
+        <td class="text-center">TINYINT</td>
+    </tr>
+    <tr>
+        <td class="text-center">SMALLINT</td>
+        <td class="text-center">SMALLINT</td>
+    </tr>
+    <tr>
+        <td class="text-center">INT</td>
+        <td class="text-center">INT</td>
+    </tr>
+    <tr>
+        <td class="text-center">BIGINT</td>
+        <td class="text-center">LONG</td>
+    </tr>
+    <tr>
+        <td class="text-center">FLOAT</td>
+        <td class="text-center">FLOAT</td>
+    </tr>
+    <tr>
+        <td class="text-center">DOUBLE</td>
+        <td class="text-center">DOUBLE</td>
+    </tr>
+    <tr>
+        <td class="text-center">DECIMAL(p, s)</td>
+        <td class="text-center">DECIMAL(p, s)</td>
+    </tr>
+    <tr>
+        <td class="text-center">DATE</td>
+        <td class="text-center">DATE</td>
+    </tr>
+    <tr>
+        <td class="text-center">TIMESTAMP_WITHOUT_TIME_ZONE</td>
+        <td class="text-center">TIMESTAMP</td>
+    </tr>
+    <tr>
+        <td class="text-center">BYTES</td>
+        <td class="text-center">BINARY</td>
+    </tr>
+    <tr>
+        <td class="text-center">ARRAY\<T\></td>
+        <td class="text-center">LIST\<T\></td>
+    </tr>
+    <tr>
+        <td class="text-center">MAP<K, V></td>
+        <td class="text-center">MAP<K, V></td>
+    </tr>
+    <tr>
+        <td class="text-center">ROW</td>
+        <td class="text-center">STRUCT</td>
+    </tr>
+  </tbody>
+</table>
+
+### Limitations
+
+The following limitations in Hive's data types impact the mapping between Flink and Hive:
+
+* `CHAR(p)` has a maximum length of 255
+* `VARCHAR(p)` has a maximum length of 65535
+* Hive's `MAP` only supports primitive key types while Flink's `MAP` can be any data type
+* Flink's `INTERVAL` type cannot be mapped to Hive `INTERVAL` type
+* Flink's `TIMESTAMP_WITH_TIME_ZONE` is not supported by Hive
+* Flink's `TIMESTAMP_WITH_LOCAL_TIME_ZONE` is not supported by Hive
+* Flink's `MULTISET` is not supported by Hive
diff --git a/docs/dev/table/hive/index.zh.md b/docs/dev/table/hive/index.zh.md
new file mode 100644
index 0000000..a6dbefa
--- /dev/null
+++ b/docs/dev/table/hive/index.zh.md
@@ -0,0 +1,265 @@
+---
+title: "Hive"
+nav-id: hive_tableapi
+nav-parent_id: tableapi
+nav-pos: 100
+is_beta: true
+nav-show_overview: true
+---
+<!--
+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.
+-->
+
+[Apache Hive](https://hive.apache.org/) has established itself as a focal point of the data warehousing ecosystem.
+It serves only not only a SQL engine for big data analytics and ETL, but also a data management platform, where data is discovered, defined, and evolved.
+
+Flink offers a two-fold integration with Hive.
+The first is to leverage Hive's Metastore as a persistent catalog for storing Flink specific metadata across sessions.
+The second is to offer Flink as an alternative engine for reading and writing Hive tables.
+
+The hive catalog is designed to be “out of the box” compatible with existing Hive installations.
+You do not need to modify your existing Hive Metastore or change the data placement or partitioning of your tables.
+
+* This will be replaced by the TOC
+{:toc}
+
+## Supported Hive Version's
+
+Flink supports Hive `2.3.4` and `1.2.1` and relies on Hive's compatibility guarantee's for other versions.
+
+### Depedencies 
+
+To integrate with Hive users need the following dependencies in their project.
+
+<div class="codetabs" markdown="1">
+<div data-lang="Hive 2.3.4" markdown="1">
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-hive_{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version}}</version>
+  <scope>provided</scope>
+</dependency>
+
+<!-- Hadoop Dependencies -->
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-hadoop-compatibility-{{site.version}}</artifactId>
+  <version>{{site.version}}</version>
+  <scope>provided</scope>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-shaded-hadoop-2-uber-2.7.5-{{site.version}}</artifactId>
+  <version>{{site.version}}</version>
+  <scope>provided</scope>
+</dependency>
+
+<!-- Hive Metastore -->
+<dependency>
+    <groupId>org.apache.hive</groupId>
+    <artifactId>hive-exec</artifactId>
+    <version>2.3.4</version>
+</dependency>
+{% endhighlight %}
+</div>
+
+<div data-lang="Hive 1.2.1" markdown="1">
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-hive_{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version}}</version>
+  <scope>provided</scope>
+</dependency>
+
+<!-- Hadoop Dependencies -->
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-hadoop-compatibility-{{site.version}}</artifactId>
+  <version>{{site.version}}</version>
+  <scope>provided</scope>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-shaded-hadoop-2-uber-2.7.5-{{site.version}}</artifactId>
+  <version>{{site.version}}</version>
+  <scope>provided</scope>
+</dependency>
+
+<!-- Hive Metastore -->
+<dependency>
+    <groupId>org.apache.hive</groupId>
+    <artifactId>hive-metastore</artifactId>
+    <version>1.2.1</version>
+</dependency>
+
+<dependency>
+    <groupId>org.apache.hive</groupId>
+    <artifactId>hive-exec</artifactId>
+    <version>1.2.1</version>
+</dependency>
+
+<dependency>
+    <groupId>org.apache.thrift</groupId>
+    <artifactId>libfb303</artifactId>
+    <version>0.9.3</version>
+</dependency>
+{% endhighlight %}
+</div>
+</div>
+
+## Connecting To Hive
+
+Connect to an existing Hive installation using the Hive [Catalog]({{ site.baseurl }}/dev/table/catalog.html) through the table environment or YAML configuration.
+
+<div class="codetabs" markdown="1">
+<div data-lang="Java" markdown="1">
+{% highlight java %}
+
+String name            = "myhive";
+String defaultDatabase = "mydatabase";
+String hiveConfDir     = "/opt/hive-conf";
+String version         = "2.3.2"; // or 1.2.1
+
+HiveCatalog hive new HiveCatalog(name, defaultDatabase, hiveConfDir, version);
+tableEnv.registerCatalog(hive);
+{% endhighlight %}
+</div>
+<div data-lang="Scala" markdown="1">
+{% highlight scala %}
+
+val name            = "myhive"
+val defaultDatabase = "mydatabase"
+val hiveConfDir     = "/opt/hive-conf"
+val version         = "2.3.2" // or 1.2.1
+
+val hive new HiveCatalog(name, defaultDatabase, hiveConfDir, version)
+tableEnv.registerCatalog(hive)
+{% endhighlight %}
+</div>
+<div data-lang="YAML" markdown="1">
+{% highlight yaml %}
+catalogs:
+   - name: myhive
+     type: hive
+     property-version: 1
+     hive-conf-dir: /opt/hive-conf
+     hive-version: 2.3.4 # or 1.2.1
+{% endhighlight %}
+</div>
+</div>
+
+## Supported Types
+
+Currently `HiveCatalog` supports most Flink data types with the following mapping:
+
+<table class="table table-bordered">
+  <thead>
+    <tr>
+      <th class="text-center" style="width: 25%">Flink Data Type</th>
+      <th class="text-center">Hive Data Type</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+        <td class="text-center">CHAR(p)</td>
+        <td class="text-center">CHAR(p)</td>
+    </tr>
+    <tr>
+        <td class="text-center">VARCHAR(p)</td>
+        <td class="text-center">VARCHAR(p)</td>
+    </tr>
+        <tr>
+        <td class="text-center">STRING</td>
+        <td class="text-center">STRING</td>
+    </tr>
+    <tr>
+        <td class="text-center">BOOLEAN</td>
+        <td class="text-center">BOOLEAN</td>
+    </tr>
+    <tr>
+        <td class="text-center">TINYINT</td>
+        <td class="text-center">TINYINT</td>
+    </tr>
+    <tr>
+        <td class="text-center">SMALLINT</td>
+        <td class="text-center">SMALLINT</td>
+    </tr>
+    <tr>
+        <td class="text-center">INT</td>
+        <td class="text-center">INT</td>
+    </tr>
+    <tr>
+        <td class="text-center">BIGINT</td>
+        <td class="text-center">LONG</td>
+    </tr>
+    <tr>
+        <td class="text-center">FLOAT</td>
+        <td class="text-center">FLOAT</td>
+    </tr>
+    <tr>
+        <td class="text-center">DOUBLE</td>
+        <td class="text-center">DOUBLE</td>
+    </tr>
+    <tr>
+        <td class="text-center">DECIMAL(p, s)</td>
+        <td class="text-center">DECIMAL(p, s)</td>
+    </tr>
+    <tr>
+        <td class="text-center">DATE</td>
+        <td class="text-center">DATE</td>
+    </tr>
+    <tr>
+        <td class="text-center">TIMESTAMP_WITHOUT_TIME_ZONE</td>
+        <td class="text-center">TIMESTAMP</td>
+    </tr>
+    <tr>
+        <td class="text-center">BYTES</td>
+        <td class="text-center">BINARY</td>
+    </tr>
+    <tr>
+        <td class="text-center">ARRAY\<T\></td>
+        <td class="text-center">LIST\<T\></td>
+    </tr>
+    <tr>
+        <td class="text-center">MAP<K, V></td>
+        <td class="text-center">MAP<K, V></td>
+    </tr>
+    <tr>
+        <td class="text-center">ROW</td>
+        <td class="text-center">STRUCT</td>
+    </tr>
+  </tbody>
+</table>
+
+### Limitations
+
+The following limitations in Hive's data types impact the mapping between Flink and Hive:
+
+* `CHAR(p)` has a maximum length of 255
+* `VARCHAR(p)` has a maximum length of 65535
+* Hive's `MAP` only supports primitive key types while Flink's `MAP` can be any data type
+* Flink's `INTERVAL` type cannot be mapped to Hive `INTERVAL` type
+* Flink's `TIMESTAMP_WITH_TIME_ZONE` is not supported by Hive
+* Flink's `TIMESTAMP_WITH_LOCAL_TIME_ZONE` is not supported by Hive
+* Flink's `MULTISET` is not supported by Hive
diff --git a/docs/dev/table/hive/read_write_hive.md b/docs/dev/table/hive/read_write_hive.md
new file mode 100644
index 0000000..b2e072f
--- /dev/null
+++ b/docs/dev/table/hive/read_write_hive.md
@@ -0,0 +1,123 @@
+---
+title: "Reading & Writing Hive Tables"
+nav-parent_id: hive_tableapi
+nav-pos: 2
+---
+<!--
+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.
+-->
+
+Using the `HiveCatalog` and Flink's connector to Hive, Flink can read and write from Hive data as an alternative to Hive's batch engine. Be sure to follow the instructions to include the correct [dependencies]({{ site.baseurl }}/dev/table/hive/#depedencies) in your application.  
+
+
+* This will be replaced by the TOC
+{:toc}
+
+## Reading From Hive
+
+Assume Hive contains a single table in its `default` database, named people that contains several rows.
+
+{% highlight bash %}
+hive> show databases;
+OK
+default
+Time taken: 0.841 seconds, Fetched: 1 row(s)
+
+hive> show tables;
+OK
+Time taken: 0.087 seconds
+
+hive> CREATE TABLE mytable(name string, value double);
+OK
+Time taken: 0.127 seconds
+
+hive> SELECT * FROM mytable;
+OK
+Tom   4.72
+John  8.0
+Tom   24.2
+Bob   3.14
+Bob   4.72
+Tom   34.9
+Mary  4.79
+Tiff  2.72
+Bill  4.33
+Mary  77.7
+Time taken: 0.097 seconds, Fetched: 10 row(s)
+{% endhighlight %}
+
+With the data ready your can connect to Hive [connect to an existing Hive installation]({{ site.baseurl }}/dev/table/hive/#connecting-to-hive) and begin querying. 
+
+{% highlight bash %}
+
+Flink SQL> show catalogs;
+myhive
+default_catalog
+
+# ------ Set the current catalog to be 'myhive' catalog if you haven't set it in the yaml file ------
+
+Flink SQL> use catalog myhive;
+
+# ------ See all registered database in catalog 'mytable' ------
+
+Flink SQL> show databases;
+default
+
+# ------ See the previously registered table 'mytable' ------
+
+Flink SQL> show tables;
+mytable
+
+# ------ The table schema that Flink sees is the same that we created in Hive, two columns - name as string and value as double ------ 
+Flink SQL> describe mytable;
+root
+ |-- name: name
+ |-- type: STRING
+ |-- name: value
+ |-- type: DOUBLE
+
+
+Flink SQL> SELECT * FROM mytable;
+
+   name      value
+__________ __________
+
+    Tom      4.72
+    John     8.0
+    Tom      24.2
+    Bob      3.14
+    Bob      4.72
+    Tom      34.9
+    Mary     4.79
+    Tiff     2.72
+    Bill     4.33
+    Mary     77.7
+
+{% endhighlight %}
+
+## Writing To Hive
+
+Similarly, data can be written into hive using an `INSERT INTO` clause. 
+
+{% highlight bash %}
+Flink SQL> INSERT INTO mytable (name, value) VALUES ('Tom', 4.72);
+{% endhighlight %}
+
+### Limitations
+
+Currently Flink's Hive data connector does not support writing into partitions. This feature is under active development.
diff --git a/docs/dev/table/hive/read_write_hive.zh.md b/docs/dev/table/hive/read_write_hive.zh.md
new file mode 100644
index 0000000..b2e072f
--- /dev/null
+++ b/docs/dev/table/hive/read_write_hive.zh.md
@@ -0,0 +1,123 @@
+---
+title: "Reading & Writing Hive Tables"
+nav-parent_id: hive_tableapi
+nav-pos: 2
+---
+<!--
+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.
+-->
+
+Using the `HiveCatalog` and Flink's connector to Hive, Flink can read and write from Hive data as an alternative to Hive's batch engine. Be sure to follow the instructions to include the correct [dependencies]({{ site.baseurl }}/dev/table/hive/#depedencies) in your application.  
+
+
+* This will be replaced by the TOC
+{:toc}
+
+## Reading From Hive
+
+Assume Hive contains a single table in its `default` database, named people that contains several rows.
+
+{% highlight bash %}
+hive> show databases;
+OK
+default
+Time taken: 0.841 seconds, Fetched: 1 row(s)
+
+hive> show tables;
+OK
+Time taken: 0.087 seconds
+
+hive> CREATE TABLE mytable(name string, value double);
+OK
+Time taken: 0.127 seconds
+
+hive> SELECT * FROM mytable;
+OK
+Tom   4.72
+John  8.0
+Tom   24.2
+Bob   3.14
+Bob   4.72
+Tom   34.9
+Mary  4.79
+Tiff  2.72
+Bill  4.33
+Mary  77.7
+Time taken: 0.097 seconds, Fetched: 10 row(s)
+{% endhighlight %}
+
+With the data ready your can connect to Hive [connect to an existing Hive installation]({{ site.baseurl }}/dev/table/hive/#connecting-to-hive) and begin querying. 
+
+{% highlight bash %}
+
+Flink SQL> show catalogs;
+myhive
+default_catalog
+
+# ------ Set the current catalog to be 'myhive' catalog if you haven't set it in the yaml file ------
+
+Flink SQL> use catalog myhive;
+
+# ------ See all registered database in catalog 'mytable' ------
+
+Flink SQL> show databases;
+default
+
+# ------ See the previously registered table 'mytable' ------
+
+Flink SQL> show tables;
+mytable
+
+# ------ The table schema that Flink sees is the same that we created in Hive, two columns - name as string and value as double ------ 
+Flink SQL> describe mytable;
+root
+ |-- name: name
+ |-- type: STRING
+ |-- name: value
+ |-- type: DOUBLE
+
+
+Flink SQL> SELECT * FROM mytable;
+
+   name      value
+__________ __________
+
+    Tom      4.72
+    John     8.0
+    Tom      24.2
+    Bob      3.14
+    Bob      4.72
+    Tom      34.9
+    Mary     4.79
+    Tiff     2.72
+    Bill     4.33
+    Mary     77.7
+
+{% endhighlight %}
+
+## Writing To Hive
+
+Similarly, data can be written into hive using an `INSERT INTO` clause. 
+
+{% highlight bash %}
+Flink SQL> INSERT INTO mytable (name, value) VALUES ('Tom', 4.72);
+{% endhighlight %}
+
+### Limitations
+
+Currently Flink's Hive data connector does not support writing into partitions. This feature is under active development.
diff --git a/docs/dev/table/hive_integration.md b/docs/dev/table/hive_integration.md
deleted file mode 100644
index 465a5f6..0000000
--- a/docs/dev/table/hive_integration.md
+++ /dev/null
@@ -1,70 +0,0 @@
----
-title: "Hive Integration"
-is_beta: true
-nav-parent_id: tableapi
-nav-pos: 9
----
-<!--
-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.
--->
-
-With its wide adoption in streaming processing, Flink has also shown its potentials in batch processing. Improving Flink’s batch processing, especially in terms of SQL, would offer user a complete set of solutions for both their streaming and batch processing needs.
-
-On the other hand, Hive has established its focal point in big data technology and its complete ecosystem. For most of big data users, Hive is not only a SQL engine for big data analytics and ETL, but also a data management platform, on which data are discovered, defined, and evolved. In another words, Hive is a de facto standard for big data on Hadoop.
-
-Therefore, it’s imperative for Flink to integrate with Hive ecosystem to further its reach to batch and SQL users. In doing that, integration with Hive metadata and data is necessary. 
-
-The goal here is neither to replace nor to copy Hive. Rather, we leverage Hive as much as we can. Flink is an alternative batch engine to Hive's batch engine. With Flink and Flink SQL, both Hive and Flink users can enjoy Hive’s rich SQL functionality and ecosystem as well as Flink's outstanding batch processing performance.
-
-
-Supported Hive version
-----------------------
-
-The target versions are Hive `2.3.4` and `1.2.1`
-
-
-Hive Metastore Integration
---------------------------
-
-There are two aspects of Hive metadata integration:
-
-1. Make Hive’s meta-objects such as tables and views available to Flink and Flink is also able to create such meta-objects for and in Hive. This is achieved through `HiveCatalog`.
-
-2. Persist Flink’s meta-objects (tables, views, and UDFs) using Hive Metastore as an persistent storage. This is achieved through `HiveCatalog`, which is under active development.
-
-For how to use and configure `HiveCatalog` in Flink, see [Catalogs]({{ site.baseurl }}/dev/table/catalog.html)
-
-Hive Data Integration
----------------------
-
-Please refer to [Connecting to other systems]({{ site.baseurl }}/dev/batch/connectors.html) for how to query Hive data using Flink's Hive data connector. 
-
-
-Examples
---------
-
-For more detailed examples using Table API or SQL, please refer to [Hive Integration Example] ({{ site.baseurl }}/dev/table/hive_integration_example.html)
-
-
-Trouble Shoot
--------------
-
-Limitations & Future
---------------------
-
-Currently Flink's Hive data connector does not support writing into partitions. The feature of writing into partitions is under active development now.
diff --git a/docs/dev/table/hive_integration_example.md b/docs/dev/table/hive_integration_example.md
deleted file mode 100644
index 4bd900a..0000000
--- a/docs/dev/table/hive_integration_example.md
+++ /dev/null
@@ -1,277 +0,0 @@
----
-title: "Hive Compatibility Example"
-is_beta: true
-nav-parent_id: examples
-nav-pos: 21
----
-<!--
-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.
--->
-
-
-The following examples showcase how to use Flink SQL Client to query Hive meta-data and tables.
-
-## Environment
-
-Assume all physical machines are accessible in the working environment, and the following components have been successfully setup:
-
-- Hadoop Cluster (HDFS + YARN)
-- Hive 2.3.4
-- Flink cluster
-
-## Setup Flink
-
-### Setup Flink Cluster in yarn-session mode
-
-Start a yarn session
-
-{% highlight bash %}
-$ ./bin/yarn-session.sh -n 4 -qu root.default -s 4 -tm 2048 -nm test_session_001
-{% endhighlight %}
-
-### Setup Flink Cluster in local mode
-
-Simply run
-
-{% highlight bash %}
-$ ./bin/start-cluster.sh
-{% endhighlight %}
-
-### Setup Flink SQL Cli
-
-Let's set the SQL Cli yaml config file now. For more detailed instructions on Flink SQL Client and catalogs, see [Flink SQL CLI]({{ site.baseurl }}/dev/table/sqlClient.html) and [Catalogs]({{ site.baseurl }}/dev/table/catalog.html).
-
-{% highlight yaml %}
-execution:
-    # Use batch mode
-    type: batch
-    # Use table result mode
-    result-mode: table
-    time-characteristic: event-time
-    periodic-watermarks-interval: 200
-    parallelism: 1
-    max-parallelism: 12
-    min-idle-state-retention: 0
-    max-idle-state-retention: 0
-    current-catalog: myhive
-    current-database: default
-
-deployment:
-  response-timeout: 5000
-
-catalogs:
-   - name: myhive
-   type: hive
-   property-version: 1
-   hive-conf-dir: /opt/hive-conf
-   hive-version: 2.3.4
-   
-{% endhighlight %}
-
-
-Note that, if users are using Flink yarn-session mode, you'll get the sessionId as `\${appId}`. Set it in `yid: ${appId}` of `deployment` section in the `conf/sql-client-defaults.yaml` file
-
-If users are using Flink local mode, no other config is required.
-
-Make sure all the required jars are in the `/lib` dir, including jars of `flink-connector-hive`, `flink-hadoop-compatibility`, `flink-shaded-hadoop-2-uber`, and Hive's jars. See [Catalogs]({{ site.baseurl }}/dev/table/catalog.html) for more details.
-
-Get Flink SQL CLI running by execute command
-
-{% highlight bash %}
-$ ./bin/sql-client.sh embedded
-{% endhighlight %}
-
-## Flink SQL Client Examples
-
-### Example 1
-
-#### Prepare Hive
-
-Assuming that Hive has been successfully set up and running, let's prepare some data in Hive.
-
-First, we locate the current database in Hive, which is `default` in this case, and make sure no table exists in the database at this time.
-
-{% highlight bash %}
-hive> show databases;
-OK
-default
-Time taken: 0.841 seconds, Fetched: 1 row(s)
-
-hive> show tables;
-OK
-Time taken: 0.087 seconds
-{% endhighlight %}
-
-Second, we create a simple table with two columns - name as string and value as double - in a textfile format and each row is delimited by ','.
-
-{% highlight bash %}
-hive> CREATE TABLE mytable(name string, value double) row format delimited fields terminated by ',' stored as textfile;
-OK
-Time taken: 0.127 seconds
-{% endhighlight %}
-
-This way, we created a table named `mytable` under the `default` database in Hive.
-
-Then let's load the data into table `mytable`. Here's some data we prepared to load into the table, and assume the file path is '/tmp/data.txt'. 
-
-{% highlight txt %}
-Tom,4.72
-John,8.00
-Tom,24.2
-Bob,3.14
-Bob,4.72
-Tom,34.9
-Mary,4.79
-Tiff,2.72
-Bill,4.33
-Mary,77.7
-{% endhighlight %}
-
-Load and check data by running: 
-
-{% highlight bash %}
-hive> load data local inpath '/tmp/data.txt' into table mytable;
-Loading data to table default.mytable
-OK
-Time taken: 0.324 seconds
-
-hive> select * from mytable;
-OK
-Tom	4.72
-John	8.0
-Tom	24.2
-Bob	3.14
-Bob	4.72
-Tom	34.9
-Mary	4.79
-Tiff	2.72
-Bill	4.33
-Mary	77.7
-Time taken: 0.097 seconds, Fetched: 10 row(s)
-{% endhighlight %}
-
-#### Access Hive metadata and data in Flink SQL CLI
-
-In Flink SQL CLI, we can start query Hive metadata.
-
-{% highlight bash %}
-
-Flink SQL> show catalogs;
-myhive
-default_catalog
-
-# ------ Set the current catalog to be 'myhive' catalog if you haven't set it in the yaml file ------
-
-Flink SQL> use catalog myhive;
-
-# ------ See all registered database in catalog 'mytable' ------
-
-Flink SQL> show databases;
-default
-
-# ------ See the previously registered table 'mytable' ------
-
-Flink SQL> show tables;
-mytable
-
-# ------ The table schema that Flink sees is the same that we created in Hive, two columns - name as string and value as double ------ 
-Flink SQL> describe mytable;
-root
- |-- name: name
- |-- type: STRING
- |-- name: value
- |-- type: DOUBLE
-
-
-Flink SQL> select * from mytable;
-
-      name      value
-__________ __________
-
-      Tom        4.72
-      John	     8.0
-      Tom	     24.2
-      Bob	     3.14
-      Bob	     4.72
-      Tom	     34.9
-      Mary	     4.79
-      Tiff	     2.72
-      Bill	     4.33
-      Mary	     77.7
-
-Flink SQL> select * from mytable;
-
-{% endhighlight %}
-
-
-### Example 2
-
-Following all the common steps above, we have prepared another two tables in Hive, order_details and products, which can be described in Hive SQL CLI:
-
-{% highlight bash %}
-Hive> describe order_details;
-OK
-orderid               bigint
-productid             bigint
-unitprice             double
-quantity              int
-discount              double
-
-Hive> describe products;
-OK
-productid             bigint
-productname           string
-supplierid            bigint
-categoryid            bigint
-quantityperunit       string
-unitprice             double
-unitsinstock          bigint
-unitsonorder          bigint
-reorderlevel          int
-discontinued          int
-);
-{% endhighlight %}
-
-We can run a few SQL query to get Hive data
-
-{% highlight sql %}
-Flink SQL> select * from products;
-
-Flink SQL> select count(*) from order_details;
-
-Flink SQL> select
-   t.productid,
-   t.productname,
-   sum(t.price) as sale
-from
-  (select
-      A.productid,
-      A.productname as productname,
-        B.unitprice * discount as price
-     from
-      products as A, order_details as B
-     where A.productid = B.productid) as t
-  group by t.productid, t.productname;
-
-{% endhighlight %}
-
-
-Flink Table API Example
------------------------
-
-To be added.