Skip to content

Commit

Permalink
[FLINK-17599][docs] Add documents for USE statement
Browse files Browse the repository at this point in the history
  • Loading branch information
godfreyhe authored and twalthr committed Jun 23, 2020
1 parent d84becd commit e2e6ee9
Show file tree
Hide file tree
Showing 6 changed files with 403 additions and 34 deletions.
1 change: 1 addition & 0 deletions docs/dev/table/sql/index.md
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ This page lists all the supported statements supported in Flink SQL for now:
- [SQL HINTS](hints.html)
- [DESCRIBE](describe.html)
- [EXPLAIN](explain.html)
- [USE](use.html)

## Data Types

Expand Down
1 change: 1 addition & 0 deletions docs/dev/table/sql/index.zh.md
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ under the License.
- [SQL HINTS](hints.html)
- [DESCRIBE](describe.html)
- [EXPLAIN](explain.html)
- [USE](use.html)

## 数据类型

Expand Down
18 changes: 1 addition & 17 deletions docs/dev/table/sql/queries.md
Original file line number Diff line number Diff line change
Expand Up @@ -382,7 +382,7 @@ String literals must be enclosed in single quotes (e.g., `SELECT 'Hello World'`)

## Operations

### Show and Use
### Show

<div markdown="1">
<table class="table table-bordered">
Expand Down Expand Up @@ -414,22 +414,6 @@ SHOW TABLES;
<p>Show all views in the current database in the current catalog</p>
{% highlight sql %}
SHOW VIEWS;
{% endhighlight %}
</td>
</tr>
<tr>
<td>
<strong>Use</strong><br>
<span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
</td>
<td>
<p>Set current catalog for the session </p>
{% highlight sql %}
USE CATALOG mycatalog;
{% endhighlight %}
<p>Set current database of the current catalog for the session</p>
{% highlight sql %}
USE mydatabase;
{% endhighlight %}
</td>
</tr>
Expand Down
18 changes: 1 addition & 17 deletions docs/dev/table/sql/queries.zh.md
Original file line number Diff line number Diff line change
Expand Up @@ -380,7 +380,7 @@ Flink SQL 对于标识符(表、属性、函数名)有类似于 Java 的词

## 操作符

### Show 与 Use
### Show

<div markdown="1">
<table class="table table-bordered">
Expand Down Expand Up @@ -412,22 +412,6 @@ SHOW TABLES;
<p>显示当前数据库、Catalog中的所有视图</p>
{% highlight sql %}
SHOW VIEWS;
{% endhighlight %}
</td>
</tr>
<tr>
<td>
<strong>Use</strong><br>
<span class="label label-primary">批处理</span> <span class="label label-primary">流处理</span>
</td>
<td>
<p>为本次会话设置 catalog </p>
{% highlight sql %}
USE CATALOG mycatalog;
{% endhighlight %}
<p>为会话设置一个属于当前 catalog 的数据库</p>
{% highlight sql %}
USE mydatabase;
{% endhighlight %}
</td>
</tr>
Expand Down
200 changes: 200 additions & 0 deletions docs/dev/table/sql/use.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,200 @@
---
title: "USE Statements"
nav-parent_id: sql
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.
-->

* This will be replaced by the TOC
{:toc}

USE statements are used to set the current database or catalog.


## Run a USE statement

USE statements can be executed with the `executeSql()` method of the `TableEnvironment`, or executed in [SQL CLI]({{ site.baseurl }}/dev/table/sqlClient.html). The `executeSql()` method returns 'OK' for a successful USE operation, otherwise will throw an exception.

The following examples show how to run a USE statement in `TableEnvironment` and in SQL CLI.

<div class="codetabs" markdown="1">
<div data-lang="java" markdown="1">
{% highlight java %}
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);

// create a catalog
tEnv.executeSql("CREATE CATALOG cat1 WITH (...)");
tEnv.executeSql("SHOW CATALOGS").print();
// +-----------------+
// | catalog name |
// +-----------------+
// | default_catalog |
// | cat1 |
// +-----------------+

// change default catalog
tEnv.executeSql("USE CATALOG cat1");

tEnv.executeSql("SHOW DATABASES").print();
// databases are empty
// +---------------+
// | database name |
// +---------------+
// +---------------+

// create a database
tEnv.executeSql("CREATE DATABASE db1 WITH (...)");
tEnv.executeSql("SHOW DATABASES").print();
// +---------------+
// | database name |
// +---------------+
// | db1 |
// +---------------+

// change default database
tEnv.executeSql("USE db1");

{% endhighlight %}
</div>

<div data-lang="scala" markdown="1">
{% highlight scala %}
val env = StreamExecutionEnvironment.getExecutionEnvironment()
val tEnv = StreamTableEnvironment.create(env)

// create a catalog
tEnv.executeSql("CREATE CATALOG cat1 WITH (...)")
tEnv.executeSql("SHOW CATALOGS").print()
// +-----------------+
// | catalog name |
// +-----------------+
// | default_catalog |
// | cat1 |
// +-----------------+

// change default catalog
tEnv.executeSql("USE CATALOG cat1")

tEnv.executeSql("SHOW DATABASES").print()
// databases are empty
// +---------------+
// | database name |
// +---------------+
// +---------------+

// create a database
tEnv.executeSql("CREATE DATABASE db1 WITH (...)")
tEnv.executeSql("SHOW DATABASES").print()
// +---------------+
// | database name |
// +---------------+
// | db1 |
// +---------------+

// change default database
tEnv.executeSql("USE db1")

{% endhighlight %}
</div>

<div data-lang="python" markdown="1">
{% highlight python %}
settings = EnvironmentSettings.new_instance()...
table_env = StreamTableEnvironment.create(env, settings)

# create a catalog
table_env.execute_sql("CREATE CATALOG cat1 WITH (...)")
table_env.execute_sql("SHOW CATALOGS").print()
# +-----------------+
# | catalog name |
# +-----------------+
# | default_catalog |
# | cat1 |
# +-----------------+

# change default catalog
table_env.execute_sql("USE CATALOG cat1")

table_env.execute_sql("SHOW DATABASES").print()
# databases are empty
# +---------------+
# | database name |
# +---------------+
# +---------------+

# create a database
table_env.execute_sql("CREATE DATABASE db1 WITH (...)")
table_env.execute_sql("SHOW DATABASES").print()
# +---------------+
# | database name |
# +---------------+
# | db1 |
# +---------------+

# change default database
table_env.execute_sql("USE db1")

{% endhighlight %}
</div>

<div data-lang="SQL CLI" markdown="1">
{% highlight sql %}
Flink SQL> CREATE CATALOG cat1 WITH (...);
[INFO] Catalog has been created.

Flink SQL> SHOW CATALOGS;
default_catalog
cat1

Flink SQL> USE CATALOG cat1;

Flink SQL> SHOW DATABASES;

Flink SQL> CREATE DATABASE db1 WITH (...);
[INFO] Database has been created.

Flink SQL> SHOW DATABASES;
db1

Flink SQL> USE db1;

{% endhighlight %}
</div>
</div>

{% top %}

## USE CATLOAG

{% highlight sql %}
USE CATALOG catalog_name
{% endhighlight %}

Set the current catalog. All subsequent commands that do not explicitly specify a catalog will use this one. If the provided catalog does not exist, an exception is thrown. The default current catalog is `default_catalog`.


## USE

{% highlight sql %}
USE [catalog_name.]database_name
{% endhighlight %}

Set the current database. All subsequent commands that do not explicitly specify a database will use this one. If the provided database does not exist, an exception is thrown. The default current database is `default_database`.
Loading

0 comments on commit e2e6ee9

Please sign in to comment.