Skip to content
Projects
Groups
Snippets
Help
Loading...
Help
Contribute to GitLab
Sign in / Register
Toggle navigation
D
dlink
Project
Project
Details
Activity
Cycle Analytics
Repository
Repository
Files
Commits
Branches
Tags
Contributors
Graph
Compare
Charts
Issues
0
Issues
0
List
Board
Labels
Milestones
Merge Requests
0
Merge Requests
0
CI / CD
CI / CD
Pipelines
Jobs
Schedules
Charts
Wiki
Wiki
Snippets
Snippets
Members
Members
Collapse sidebar
Close sidebar
Activity
Graph
Charts
Create a new issue
Jobs
Commits
Issue Boards
Open sidebar
zhaowei
dlink
Commits
5126bead
Unverified
Commit
5126bead
authored
Jul 10, 2022
by
Arnu-
Committed by
GitHub
Jul 10, 2022
Browse files
Options
Browse Files
Download
Email Patches
Plain Diff
实现了使用mysql来持久化flink table元数据的catalog。 (#703)
parent
5be884c8
Changes
13
Expand all
Hide whitespace changes
Inline
Side-by-side
Showing
13 changed files
with
1724 additions
and
0 deletions
+1724
-0
README.md
dlink-catalog/README.md
+43
-0
pom.xml
...alog/dlink-catalog-mysql/dlink-catalog-mysql-1.14/pom.xml
+36
-0
DlinkMysqlCatalog.java
.../main/java/com/dlink/flink/catalog/DlinkMysqlCatalog.java
+1140
-0
DlinkMysqlCatalogFactory.java
...dlink/flink/catalog/factory/DlinkMysqlCatalogFactory.java
+69
-0
DlinkMysqlCatalogFactoryOptions.java
...link/catalog/factory/DlinkMysqlCatalogFactoryOptions.java
+92
-0
org.apache.flink.table.factories.Factory
...ETA-INF/services/org.apache.flink.table.factories.Factory
+16
-0
dlink-mysql-catalog.properties
...ql-1.14/src/main/resources/dlink-mysql-catalog.properties
+6
-0
dlinkmysqlcatalog.sql
...talog-mysql-1.14/src/main/resources/dlinkmysqlcatalog.sql
+67
-0
DlinkMysqlCatalogTest.java
...t/java/com/dlink/flink/catalog/DlinkMysqlCatalogTest.java
+139
-0
DlinkMysqlCatalogFactoryTest.java
...k/flink/catalog/factory/DlinkMysqlCatalogFactoryTest.java
+74
-0
pom.xml
dlink-catalog/dlink-catalog-mysql/pom.xml
+23
-0
pom.xml
dlink-catalog/pom.xml
+18
-0
pom.xml
pom.xml
+1
-0
No files found.
dlink-catalog/README.md
0 → 100644
View file @
5126bead
# 自定义实现的catalog
## 目标
用于保存 flink sql创建的元数据
-
实现了 database table view 的 create ,alter, drop,list,get 操作。
-
partition相关操作: 根据文档,只有再文件系统的connector才需要这个功能。
>例如 在 hdfs 上,其作用是在文件系统上根据分区字段创建目录,列出已有的分区目录,修改分区目录等。根据评估,如果是应对流数据,暂时不需要实现该功能。
如果是针对hdfs,则使用hive catalog来作为 catalog 更好
如果是其他fs类的支持,需要适配各种环境,不利于做到通用性。
故这个部分未进行实现。
-
对各类 Statistics 也未进行实现,因为流数据无法获取相关的条数,数据大小等。
## 用法
1.
将打好包的jar放到flink lib目录
2.
引入mysql driver,需要支持 com.mysql.cj.jdbc.Driver 这个暂时是写死的。
3.
使用mycatalog.sql 初始化meta库所需的表。
4.
在flink conf目录下创建配置文件 mysql-catalog.properties
5.
根据你的环境启动sql client 或者在代码中使用 flink sql来创建catalog,注册catalog。
> flink sql:
```
roomsql
create catalog mycatalog
with(
'type'='dlink_mysql_catalog'
);
use catalog mycatalog;
```
> java
```
java
DlinkMysqlCatalog
catalog
=
new
DlinkMysqlCatalog
(
"myCatalog"
);
tableEnv
.
registerCatalog
(
DlinkMysqlCatalogFactoryOptions
.
IDENTIFIER
,
catalog
);
tableEnv
.
useCatalog
(
DlinkMysqlCatalogFactoryOptions
.
IDENTIFIER
);
```
## 注意
> 在dlink mysql catalog的实现中,对默认数据库进行了限制,不允许用户自定义默认数据库名称。
todo: 在yaml文件中定义 catalog 的方法。
dlink-catalog/dlink-catalog-mysql/dlink-catalog-mysql-1.14/pom.xml
0 → 100644
View file @
5126bead
<?xml version="1.0" encoding="UTF-8"?>
<project
xmlns=
"http://maven.apache.org/POM/4.0.0"
xmlns:xsi=
"http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation=
"http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
>
<parent>
<artifactId>
dlink-catalog-mysql
</artifactId>
<groupId>
com.dlink
</groupId>
<version>
0.6.6-SNAPSHOT
</version>
</parent>
<modelVersion>
4.0.0
</modelVersion>
<artifactId>
dlink-catalog-mysql-1.14
</artifactId>
<properties>
<maven.compiler.source>
8
</maven.compiler.source>
<maven.compiler.target>
8
</maven.compiler.target>
</properties>
<dependencies>
<dependency>
<groupId>
com.dlink
</groupId>
<artifactId>
dlink-common
</artifactId>
</dependency>
<dependency>
<groupId>
com.dlink
</groupId>
<artifactId>
dlink-flink-1.14
</artifactId>
<!-- <scope>provided</scope>-->
</dependency>
<dependency>
<groupId>
junit
</groupId>
<artifactId>
junit
</artifactId>
<version>
4.13.2
</version>
<scope>
test
</scope>
</dependency>
</dependencies>
</project>
\ No newline at end of file
dlink-catalog/dlink-catalog-mysql/dlink-catalog-mysql-1.14/src/main/java/com/dlink/flink/catalog/DlinkMysqlCatalog.java
0 → 100644
View file @
5126bead
This diff is collapsed.
Click to expand it.
dlink-catalog/dlink-catalog-mysql/dlink-catalog-mysql-1.14/src/main/java/com/dlink/flink/catalog/factory/DlinkMysqlCatalogFactory.java
0 → 100644
View file @
5126bead
/*
* 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
com
.
dlink
.
flink
.
catalog
.
factory
;
import
com.dlink.flink.catalog.DlinkMysqlCatalog
;
import
org.apache.flink.configuration.ConfigOption
;
import
org.apache.flink.table.catalog.Catalog
;
import
org.apache.flink.table.factories.CatalogFactory
;
import
org.apache.flink.table.factories.FactoryUtil
;
import
java.util.HashSet
;
import
java.util.Set
;
import
static
com
.
dlink
.
flink
.
catalog
.
factory
.
DlinkMysqlCatalogFactoryOptions
.*;
import
static
org
.
apache
.
flink
.
table
.
factories
.
FactoryUtil
.
PROPERTY_VERSION
;
/** Factory for {@link DlinkMysqlCatalog}. */
public
class
DlinkMysqlCatalogFactory
implements
CatalogFactory
{
@Override
public
String
factoryIdentifier
()
{
return
DlinkMysqlCatalogFactoryOptions
.
IDENTIFIER
;
}
@Override
public
Set
<
ConfigOption
<?>>
requiredOptions
()
{
final
Set
<
ConfigOption
<?>>
options
=
new
HashSet
<>();
return
options
;
}
@Override
public
Set
<
ConfigOption
<?>>
optionalOptions
()
{
final
Set
<
ConfigOption
<?>>
options
=
new
HashSet
<>();
options
.
add
(
USERNAME
);
options
.
add
(
PASSWORD
);
options
.
add
(
URL
);
options
.
add
(
PROPERTY_VERSION
);
return
options
;
}
@Override
public
Catalog
createCatalog
(
Context
context
)
{
final
FactoryUtil
.
CatalogFactoryHelper
helper
=
FactoryUtil
.
createCatalogFactoryHelper
(
this
,
context
);
helper
.
validate
();
return
new
DlinkMysqlCatalog
(
context
.
getName
(),
helper
.
getOptions
().
get
(
URL
),
helper
.
getOptions
().
get
(
USERNAME
),
helper
.
getOptions
().
get
(
PASSWORD
));
}
}
dlink-catalog/dlink-catalog-mysql/dlink-catalog-mysql-1.14/src/main/java/com/dlink/flink/catalog/factory/DlinkMysqlCatalogFactoryOptions.java
0 → 100644
View file @
5126bead
/*
* 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
com
.
dlink
.
flink
.
catalog
.
factory
;
import
com.dlink.flink.catalog.DlinkMysqlCatalog
;
import
org.apache.flink.annotation.Internal
;
import
org.apache.flink.configuration.ConfigConstants
;
import
org.apache.flink.configuration.ConfigOption
;
import
org.apache.flink.configuration.ConfigOptions
;
import
org.apache.flink.table.catalog.CommonCatalogOptions
;
import
org.apache.flink.table.catalog.exceptions.CatalogException
;
import
java.io.File
;
import
java.io.FileInputStream
;
import
java.io.InputStream
;
import
java.util.Properties
;
/**
* {@link ConfigOption}s for {@link DlinkMysqlCatalog}.
*/
@Internal
public
class
DlinkMysqlCatalogFactoryOptions
{
public
static
final
String
IDENTIFIER
=
"dlink_mysql_catalog"
;
public
static
final
ConfigOption
<
String
>
USERNAME
;
// =
// ConfigOptions.key("mysql-catalog-username").stringType().noDefaultValue();
public
static
final
ConfigOption
<
String
>
PASSWORD
;
// =
// ConfigOptions.key("mysql-catalog-password").stringType().noDefaultValue();
public
static
final
ConfigOption
<
String
>
URL
;
// =
// ConfigOptions.key("mysql-catalog-url").stringType().noDefaultValue();
public
static
final
String
prefix
=
"dlink-mysql-catalog"
;
static
{
try
{
String
configPath
=
System
.
getenv
(
ConfigConstants
.
ENV_FLINK_CONF_DIR
);
if
(!
configPath
.
endsWith
(
"/"
))
{
configPath
=
configPath
+
"/"
;
}
configPath
=
configPath
+
addPrefix
(
".properties"
);
File
propFile
=
new
File
(
configPath
);
if
(!
propFile
.
exists
())
{
throw
new
CatalogException
(
"配置文件不存在!"
);
}
InputStream
is
=
new
FileInputStream
(
propFile
);
Properties
props
=
new
Properties
();
props
.
load
(
is
);
String
username
=
props
.
getProperty
(
addPrefix
(
"-username"
));
USERNAME
=
ConfigOptions
.
key
(
addPrefix
(
"-username"
))
.
stringType
()
.
defaultValue
(
username
);
String
password
=
props
.
getProperty
(
addPrefix
(
"-password"
));
PASSWORD
=
ConfigOptions
.
key
(
addPrefix
(
"-password"
))
.
stringType
()
.
defaultValue
(
password
);
String
url
=
props
.
getProperty
(
addPrefix
(
"-url"
));
URL
=
ConfigOptions
.
key
(
addPrefix
(
"-url"
))
.
stringType
()
.
defaultValue
(
url
);
}
catch
(
Exception
e
)
{
throw
new
CatalogException
(
"获取配置信息失败!"
,
e
);
}
}
private
static
String
addPrefix
(
String
key
)
{
return
prefix
+
key
;
}
private
DlinkMysqlCatalogFactoryOptions
()
{
}
}
dlink-catalog/dlink-catalog-mysql/dlink-catalog-mysql-1.14/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
0 → 100644
View file @
5126bead
# 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.
com.dlink.flink.catalog.factory.DlinkMysqlCatalogFactory
dlink-catalog/dlink-catalog-mysql/dlink-catalog-mysql-1.14/src/main/resources/dlink-mysql-catalog.properties
0 → 100644
View file @
5126bead
# catalog数据库用户名
dlink-mysql-catalog-username
=
root
# catalog 数据库密码
dlink-mysql-catalog-password
=
123456
# catalog 数据库url
dlink-mysql-catalog-url
=
jdbc:mysql://localhost:3306/flink_metastore?useUnicode=true&characterEncoding=utf8&serverTimezone=UTC
\ No newline at end of file
dlink-catalog/dlink-catalog-mysql/dlink-catalog-mysql-1.14/src/main/resources/dlinkmysqlcatalog.sql
0 → 100644
View file @
5126bead
drop
table
if
exists
`metadata_database`
create
table
if
not
exists
`metadata_database`
(
`id`
int
(
11
)
not
null
AUTO_INCREMENT
COMMENT
'主键'
,
`database_name`
varchar
(
255
)
NOT
NULL
COMMENT
'名称'
,
`description`
varchar
(
255
)
null
comment
'描述'
,
`update_time`
datetime
DEFAULT
NULL
COMMENT
'更新时间'
,
`create_time`
datetime
NULL
DEFAULT
CURRENT_TIMESTAMP
COMMENT
'创建时间'
,
PRIMARY
KEY
(
`id`
)
)
ENGINE
=
InnoDB
DEFAULT
CHARSET
=
utf8mb4
COMMENT
=
'元数据对象信息'
drop
table
if
exists
`metadata_table`
create
table
if
not
exists
`metadata_table`
(
`id`
int
(
11
)
not
null
AUTO_INCREMENT
COMMENT
'主键'
,
`table_name`
varchar
(
255
)
NOT
NULL
COMMENT
'名称'
,
`table_type`
varchar
(
255
)
NOT
null
comment
'对象类型,分为:database 和 table view'
,
`database_id`
int
(
11
)
not
null
COMMENT
'数据库主键'
,
`description`
varchar
(
255
)
null
comment
'描述'
,
`update_time`
datetime
DEFAULT
NULL
COMMENT
'更新时间'
,
`create_time`
datetime
NULL
DEFAULT
CURRENT_TIMESTAMP
COMMENT
'创建时间'
,
PRIMARY
KEY
(
`id`
)
)
ENGINE
=
InnoDB
DEFAULT
CHARSET
=
utf8mb4
COMMENT
=
'元数据对象信息'
drop
table
if
exists
`metadata_database_property`
create
table
if
not
exists
`metadata_database_property`
(
`key`
varchar
(
255
)
NOT
NULL
COMMENT
'属性key'
,
`value`
varchar
(
255
)
NULL
COMMENT
'属性value'
,
`database_id`
int
(
11
)
not
null
COMMENT
'数据库主键'
,
`update_time`
datetime
DEFAULT
NULL
COMMENT
'更新时间'
,
`create_time`
datetime
NOT
NULL
DEFAULT
CURRENT_TIMESTAMP
COMMENT
'创建时间'
,
PRIMARY
KEY
(
`key`
,
`database_id`
)
)
ENGINE
=
InnoDB
DEFAULT
CHARSET
=
utf8mb4
COMMENT
=
'元数据属性信息'
drop
table
if
exists
`metadata_table_property`
create
table
if
not
exists
`metadata_table_property`
(
`key`
varchar
(
255
)
NOT
NULL
COMMENT
'属性key'
,
`value`
varchar
(
255
)
NULL
COMMENT
'属性value'
,
`table_id`
int
(
11
)
not
null
COMMENT
'数据表名称'
,
`update_time`
datetime
DEFAULT
NULL
COMMENT
'更新时间'
,
`create_time`
datetime
NOT
NULL
DEFAULT
CURRENT_TIMESTAMP
COMMENT
'创建时间'
,
PRIMARY
KEY
(
`key`
,
`table_id`
)
)
ENGINE
=
InnoDB
DEFAULT
CHARSET
=
utf8mb4
COMMENT
=
'元数据属性信息'
drop
table
if
exists
metadata_column
create
table
if
not
exists
`metadata_column`
(
`column_name`
varchar
(
255
)
NOT
NULL
COMMENT
'列名'
,
`column_type`
varchar
(
255
)
NOT
NULL
COMMENT
'列类型, 有Physical Metadata Computed WATERMARK '
,
`data_type`
varchar
(
255
)
NOT
NULL
COMMENT
'数据类型'
,
`expr`
varchar
(
255
)
NULL
COMMENT
'表达式'
,
`description`
varchar
(
255
)
NOT
NULL
COMMENT
'字段描述'
,
`table_id`
int
(
11
)
not
null
COMMENT
'数据表名称'
,
`primary`
bit
null
comment
'主键'
,
`update_time`
datetime
DEFAULT
NULL
COMMENT
'更新时间'
,
`create_time`
datetime
NOT
NULL
DEFAULT
CURRENT_TIMESTAMP
COMMENT
'创建时间'
,
PRIMARY
KEY
(
`table_id`
,
`column_name`
)
)
ENGINE
=
InnoDB
DEFAULT
CHARSET
=
utf8mb4
COMMENT
=
'数据列信息'
drop
table
if
exists
`metadata_function`
create
table
if
not
exists
`metadata_function`
(
`id`
int
(
11
)
not
null
AUTO_INCREMENT
COMMENT
'主键'
,
`function_name`
varchar
(
255
)
NOT
NULL
COMMENT
'名称'
,
`class_name`
varchar
(
255
)
NOT
null
comment
'类名'
,
`database_id`
int
(
11
)
not
null
COMMENT
'数据库主键'
,
`function_language`
varchar
(
255
)
null
comment
'UDF语言'
,
`update_time`
datetime
DEFAULT
NULL
COMMENT
'更新时间'
,
`create_time`
datetime
NULL
DEFAULT
CURRENT_TIMESTAMP
COMMENT
'创建时间'
,
PRIMARY
KEY
(
`id`
)
)
ENGINE
=
InnoDB
DEFAULT
CHARSET
=
utf8mb4
COMMENT
=
'UDF信息'
\ No newline at end of file
dlink-catalog/dlink-catalog-mysql/dlink-catalog-mysql-1.14/src/test/java/com/dlink/flink/catalog/DlinkMysqlCatalogTest.java
0 → 100644
View file @
5126bead
/*
# __
# / | ____ ___ _
# / / | / __// // / /
# /_/`_|/_/ / /_//___/
create @ 2022/7/9
*/
package
com
.
dlink
.
flink
.
catalog
;
import
org.apache.flink.table.api.EnvironmentSettings
;
import
org.apache.flink.table.api.TableEnvironment
;
import
org.junit.Before
;
import
org.junit.Test
;
import
static
org
.
apache
.
flink
.
table
.
api
.
config
.
ExecutionConfigOptions
.
TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM
;
public
class
DlinkMysqlCatalogTest
{
protected
static
String
url
;
protected
static
DlinkMysqlCatalog
catalog
;
protected
static
final
String
TEST_CATALOG_NAME
=
"mysql-catalog"
;
protected
static
final
String
TEST_USERNAME
=
"flink_metastore"
;
protected
static
final
String
TEST_PWD
=
"flink_metastore"
;
private
TableEnvironment
tableEnv
;
@Before
public
void
setup
(){
url
=
"jdbc:mysql://localhost:3306/flink_metastore?useUnicode=true&characterEncoding=utf8&serverTimezone=UTC"
;
catalog
=
new
DlinkMysqlCatalog
(
TEST_CATALOG_NAME
,
url
,
TEST_USERNAME
,
TEST_PWD
);
this
.
tableEnv
=
TableEnvironment
.
create
(
EnvironmentSettings
.
inStreamingMode
());
tableEnv
.
getConfig
()
.
getConfiguration
()
.
setInteger
(
TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM
.
key
(),
1
);
}
@Test
public
void
testSqlCatalog
(){
String
createSql
=
"create catalog myCatalog \n"
+
" with('type'='dlink_mysql_catalog',\n"
+
" 'mysql-catalog-username'='flink_metastore',\n"
+
" 'mysql-catalog-password'='flink_metastore',\n"
+
" 'mysql-catalog-url'='jdbc:mysql://localhost:3306/"
+
"flink_metastore?useUnicode=true&characterEncoding=utf8&serverTimezone=UTC')"
;
tableEnv
.
executeSql
(
createSql
);
tableEnv
.
executeSql
(
"use catalog myCatalog"
);
}
@Test
public
void
test
()
{
// 这个 test 依赖个人环境,直接保留会导致打包不通过。但是展示了用法。
/*//1\. 获取上下文环境 table的环境
// use mysql-catalog
tableEnv.registerCatalog(DlinkMysqlCatalogFactoryOptions.IDENTIFIER, catalog);
tableEnv.useCatalog(DlinkMysqlCatalogFactoryOptions.IDENTIFIER);
//2\. 读取score.csv
String csvFile = "D:\\code\\test\\mycatalog\\src\\test\\resources\\score.csv";
String createTable = "CREATE TABLE IF NOT EXISTS player_data\n" +
"( season varchar,\n" +
" player varchar,\n" +
" play_num varchar,\n" +
" first_court int,\n" +
" `time` double,\n" +
" assists double,\n" +
" steals double,\n" +
" blocks double,\n" +
" scores double\n" +
") WITH ( \n" +
" 'connector' = 'filesystem',\n" +
" 'path' = '" + csvFile + " ',\n" +
" 'format' = 'csv'\n" +
")";
tableEnv.executeSql(createTable);
String createView= "CREATE VIEW IF NOT EXISTS test_view " +
" (player, play_num" +
" ,sumaabb)" +
" COMMENT 'test view' " +
" AS SELECT player, play_num, assists + steals as sumaabb FROM player_data";
tableEnv.executeSql(createView);
String createSinkTable = "CREATE TABLE IF NOT EXISTS mysql_player_from_view\n" +
"( " +
" player varchar,\n" +
" play_num varchar,\n" +
" sumaabb double\n" +
") WITH ( \n" +
" 'connector' = 'jdbc',\n" +
" 'url' = 'jdbc:mysql://localhost:3306/a01_rep_db',\n" +
" 'table-name' = 'mysql_player_from_view',\n" +
" 'username' = 'root',\n" +
" 'password' = '123456'\n" +
")";
tableEnv.executeSql(createSinkTable);
tableEnv.executeSql("Insert into mysql_player_from_view\n" +
"SELECT \n" +
"player ,\n" +
" play_num ,\n" +
" sumaabb \n" +
"FROM test_view");
List<Row> results =
CollectionUtil.iteratorToList(
tableEnv.sqlQuery("select * from mysql_player_from_view")
.execute()
.collect());
List<Row> tresults =
CollectionUtil.iteratorToList(
tableEnv.sqlQuery("select * from test_view")
.execute()
.collect());
List<Row> presults =
CollectionUtil.iteratorToList(
tableEnv.sqlQuery("select * from player_data")
.execute()
.collect());
*/
}
}
dlink-catalog/dlink-catalog-mysql/dlink-catalog-mysql-1.14/src/test/java/com/dlink/flink/catalog/com/dlink/flink/catalog/factory/DlinkMysqlCatalogFactoryTest.java
0 → 100644
View file @
5126bead
/*
# __
# / | ____ ___ _
# / / | / __// // / /
# /_/`_|/_/ / /_//___/
create @ 2022/7/9
*/
package
com
.
dlink
.
flink
.
catalog
.
com
.
dlink
.
flink
.
catalog
.
factory
;
import
com.dlink.flink.catalog.DlinkMysqlCatalog
;
import
com.dlink.flink.catalog.factory.DlinkMysqlCatalogFactoryOptions
;
import
org.apache.flink.table.catalog.Catalog
;
import
org.apache.flink.table.catalog.CommonCatalogOptions
;
import
org.apache.flink.table.factories.FactoryUtil
;
import
org.junit.BeforeClass
;
import
org.junit.Test
;
import
java.sql.SQLException
;
import
java.util.HashMap
;
import
java.util.Map
;
import
static
org
.
junit
.
Assert
.
assertEquals
;
import
static
org
.
junit
.
Assert
.
assertTrue
;
public
class
DlinkMysqlCatalogFactoryTest
{
protected
static
String
url
;
protected
static
DlinkMysqlCatalog
catalog
;
protected
static
final
String
TEST_CATALOG_NAME
=
"mysql-catalog"
;
protected
static
final
String
TEST_USERNAME
=
"flink_metastore"
;
protected
static
final
String
TEST_PWD
=
"flink_metastore"
;
@BeforeClass
public
static
void
setup
()
throws
SQLException
{
url
=
"jdbc:mysql://localhost:3306/flink_metastore?useUnicode=true&characterEncoding=utf8&serverTimezone=UTC"
;
catalog
=
new
DlinkMysqlCatalog
(
TEST_CATALOG_NAME
,
TEST_USERNAME
,
TEST_PWD
,
url
);
}
@Test
public
void
test
()
{
final
Map
<
String
,
String
>
options
=
new
HashMap
<>();
options
.
put
(
CommonCatalogOptions
.
CATALOG_TYPE
.
key
(),
DlinkMysqlCatalogFactoryOptions
.
IDENTIFIER
);
options
.
put
(
DlinkMysqlCatalogFactoryOptions
.
USERNAME
.
key
(),
TEST_USERNAME
);
options
.
put
(
DlinkMysqlCatalogFactoryOptions
.
PASSWORD
.
key
(),
TEST_PWD
);
options
.
put
(
DlinkMysqlCatalogFactoryOptions
.
URL
.
key
(),
url
);
final
Catalog
actualCatalog
=
FactoryUtil
.
createCatalog
(
TEST_CATALOG_NAME
,
options
,
null
,
Thread
.
currentThread
().
getContextClassLoader
());
checkEquals
(
catalog
,
(
DlinkMysqlCatalog
)
actualCatalog
);
assertTrue
(
actualCatalog
instanceof
DlinkMysqlCatalog
);
}
private
static
void
checkEquals
(
DlinkMysqlCatalog
c1
,
DlinkMysqlCatalog
c2
)
{
assertEquals
(
c1
.
getName
(),
c2
.
getName
());
assertEquals
(
c1
.
getDefaultDatabase
(),
c2
.
getDefaultDatabase
());
assertEquals
(
c1
.
getUser
(),
c2
.
getUser
());
assertEquals
(
c1
.
getPwd
(),
c2
.
getPwd
());
assertEquals
(
c1
.
getUrl
(),
c2
.
getUrl
());
}
}
dlink-catalog/dlink-catalog-mysql/pom.xml
0 → 100644
View file @
5126bead
<?xml version="1.0" encoding="UTF-8"?>
<project
xmlns=
"http://maven.apache.org/POM/4.0.0"
xmlns:xsi=
"http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation=
"http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
>
<parent>
<artifactId>
dlink-catalog
</artifactId>
<groupId>
com.dlink
</groupId>
<version>
0.6.6-SNAPSHOT
</version>
</parent>
<modelVersion>
4.0.0
</modelVersion>
<artifactId>
dlink-catalog-mysql
</artifactId>
<packaging>
pom
</packaging>
<modules>
<module>
dlink-catalog-mysql-1.14
</module>
</modules>
<properties>
<maven.compiler.source>
8
</maven.compiler.source>
<maven.compiler.target>
8
</maven.compiler.target>
</properties>
</project>
\ No newline at end of file
dlink-catalog/pom.xml
0 → 100644
View file @
5126bead
<?xml version="1.0" encoding="UTF-8"?>
<project
xmlns=
"http://maven.apache.org/POM/4.0.0"
xmlns:xsi=
"http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation=
"http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
>
<parent>
<artifactId>
dlink
</artifactId>
<groupId>
com.dlink
</groupId>
<version>
0.6.6-SNAPSHOT
</version>
</parent>
<modelVersion>
4.0.0
</modelVersion>
<packaging>
pom
</packaging>
<artifactId>
dlink-catalog
</artifactId>
<modules>
<module>
dlink-catalog-mysql
</module>
</modules>
</project>
\ No newline at end of file
pom.xml
View file @
5126bead
...
...
@@ -26,6 +26,7 @@
<module>
dlink-web
</module>
<module>
dlink-admin
</module>
<module>
dlink-assembly
</module>
<module>
dlink-catalog
</module>
</modules>
<properties>
...
...
Write
Preview
Markdown
is supported
0%
Try again
or
attach a new file
Attach a file
Cancel
You are about to add
0
people
to the discussion. Proceed with caution.
Finish editing this message first!
Cancel
Please
register
or
sign in
to comment