Skip to content
Permalink
Browse files
[INLONG-4157][Sort] Sort lightwieght support load data to Iceberg (#4210
)
  • Loading branch information
thexiay committed May 17, 2022
1 parent 236e5d9 commit f52fd5f583d28d775ce502b7658538a0e205804a
Showing 6 changed files with 317 additions and 47 deletions.
@@ -0,0 +1,97 @@
/*
* 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.inlong.sort.protocol.node.load;

import com.google.common.base.Preconditions;
import lombok.Data;
import lombok.EqualsAndHashCode;
import lombok.NoArgsConstructor;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;
import org.apache.inlong.sort.protocol.FieldInfo;
import org.apache.inlong.sort.protocol.enums.FilterStrategy;
import org.apache.inlong.sort.protocol.node.LoadNode;
import org.apache.inlong.sort.protocol.transformation.FieldRelationShip;
import org.apache.inlong.sort.protocol.transformation.FilterFunction;

import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.Serializable;
import java.util.List;
import java.util.Map;

@JsonTypeName("icebergLoad")
@Data
@NoArgsConstructor
@EqualsAndHashCode(callSuper = true)
public class IcebergLoadNode extends LoadNode implements Serializable {

private static final long serialVersionUID = -1L;

@JsonProperty("tableName")
@Nonnull
private String tableName;

@JsonProperty("tableName")
@Nonnull
private String dbName;

@JsonCreator
public IcebergLoadNode(@JsonProperty("id") String id,
@JsonProperty("name") String name,
@JsonProperty("fields") List<FieldInfo> fields,
@JsonProperty("fieldRelationShips") List<FieldRelationShip> fieldRelationShips,
@JsonProperty("filters") List<FilterFunction> filters,
@JsonProperty("filterStrategy") FilterStrategy filterStrategy,
@Nullable @JsonProperty("sinkParallelism") Integer sinkParallelism,
@JsonProperty("properties") Map<String, String> properties,
@Nonnull @JsonProperty("dbName") String dbName,
@Nonnull @JsonProperty("tableName") String tableName) {
super(id, name, fields, fieldRelationShips, filters, filterStrategy, sinkParallelism, properties);
this.tableName = Preconditions.checkNotNull(tableName, "table name is null");
this.dbName = Preconditions.checkNotNull(dbName, "db name is null");
}

@Override
public Map<String, String> tableOptions() {
Map<String, String> options = super.tableOptions();
options.put("connector", "iceberg");
options.put("catalog-database", dbName);
options.put("catalog-table", tableName);
options.put("default-database", dbName);
return options;
}

@Override
public String genTableName() {
return tableName;
}

@Override
public String getPrimaryKey() {
return super.getPrimaryKey();
}

@Override
public List<FieldInfo> getPartitionFields() {
return super.getPartitionFields();
}

}
@@ -84,19 +84,24 @@
<artifactId>hadoop-common</artifactId>
<version>${hadoop.version}</version>
</dependency>
<dependency>
<groupId>org.apache.orc</groupId>
<artifactId>orc-core</artifactId>
<classifier>nohive</classifier>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-jdbc</artifactId>
<exclusions>
<exclusion>
<groupId>org.apache.orc</groupId>
<artifactId>orc-core</artifactId>
</exclusion>
<exclusion>
<artifactId>parquet-hadoop-bundle</artifactId>
<groupId>org.apache.parquet</groupId>
</exclusion>
</exclusions>
</dependency>
<!--for iceberg-->
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-minicluster</artifactId>
<scope>test</scope>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-flink-1.13</artifactId>
</dependency>
<!--for doris-->
<dependency>
@@ -113,6 +118,27 @@
<groupId>org.apache.pulsar</groupId>
<artifactId>pulsar-client-all</artifactId>
</dependency>

<!--format dependency-->
<dependency>
<groupId>org.apache.orc</groupId>
<artifactId>orc-core</artifactId>
<classifier>nohive</classifier>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-hadoop</artifactId>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</dependency>
<!--test-->
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-minicluster</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_${flink.scala.binary.version}</artifactId>
@@ -79,20 +79,6 @@
<artifactId>flink-streaming-java_${flink.scala.binary.version}</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-flink</artifactId>
<exclusions>
<exclusion>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-hive-metastore</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>org.apache.hadoop</groupId>
@@ -0,0 +1,159 @@
/*
* 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.inlong.sort.singletenant.flink.parser;

import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.test.util.AbstractTestBase;
import org.apache.inlong.sort.formats.common.FloatFormatInfo;
import org.apache.inlong.sort.formats.common.IntFormatInfo;
import org.apache.inlong.sort.formats.common.LongFormatInfo;
import org.apache.inlong.sort.formats.common.StringFormatInfo;
import org.apache.inlong.sort.formats.common.TimestampFormatInfo;
import org.apache.inlong.sort.protocol.FieldInfo;
import org.apache.inlong.sort.protocol.GroupInfo;
import org.apache.inlong.sort.protocol.StreamInfo;
import org.apache.inlong.sort.protocol.node.Node;
import org.apache.inlong.sort.protocol.node.extract.MySqlExtractNode;
import org.apache.inlong.sort.protocol.node.load.IcebergLoadNode;
import org.apache.inlong.sort.protocol.transformation.FieldRelationShip;
import org.apache.inlong.sort.protocol.transformation.relation.NodeRelationShip;
import org.apache.inlong.sort.singletenant.flink.parser.impl.FlinkSqlParser;
import org.apache.inlong.sort.singletenant.flink.parser.result.FlinkSqlParseResult;
import org.junit.Assert;
import org.junit.Test;

import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;

/**
* Test for Iceberg SQL parser.
*/
public class IcebergNodeSqlParserTest extends AbstractTestBase {

private MySqlExtractNode buildMySQLExtractNode(String id) {
List<FieldInfo> fields = Arrays.asList(new FieldInfo("id", new LongFormatInfo()),
new FieldInfo("name", new StringFormatInfo()),
new FieldInfo("age", new IntFormatInfo()),
new FieldInfo("salary", new FloatFormatInfo()),
new FieldInfo("ts", new TimestampFormatInfo()),
new FieldInfo("event_type", new StringFormatInfo()));
// if you hope hive load mode of append, please add this config
Map<String, String> map = new HashMap<>();
map.put("append-mode", "true");
return new MySqlExtractNode(id, "mysql_input", fields,
null, map, "id",
Collections.singletonList("work1"), "localhost", "root", "123456",
"inlong", null, null,
null, null);
}

private IcebergLoadNode buildIcebergLoadNodeWithHadoopCatalog() {
List<FieldInfo> fields = Arrays.asList(new FieldInfo("id", new LongFormatInfo()),
new FieldInfo("name", new StringFormatInfo()),
new FieldInfo("salary", new StringFormatInfo()),
new FieldInfo("ts", new TimestampFormatInfo()));
List<FieldRelationShip> relations = Arrays
.asList(new FieldRelationShip(new FieldInfo("id", new LongFormatInfo()),
new FieldInfo("id", new LongFormatInfo())),
new FieldRelationShip(new FieldInfo("name", new StringFormatInfo()),
new FieldInfo("name", new StringFormatInfo())),
new FieldRelationShip(new FieldInfo("age", new IntFormatInfo()),
new FieldInfo("age", new IntFormatInfo())),
new FieldRelationShip(new FieldInfo("ts", new TimestampFormatInfo()),
new FieldInfo("ts", new TimestampFormatInfo()))
);

Map<String, String> props = new HashMap<>();
props.put("catalog-type", "hadoop");
props.put("catalog-name", "hadoop_prod");
props.put("warehouse", "hdfs://localhost:9000/iceberg/warehouse");
IcebergLoadNode node = new IcebergLoadNode("iceberg", "iceberg_output", fields, relations,
null, null, null, props, "inlong", "inlong_iceberg");
return node;
}

private IcebergLoadNode buildIcebergLoadNodeWithHiveCatalog() {
List<FieldInfo> fields = Arrays.asList(new FieldInfo("id", new LongFormatInfo()),
new FieldInfo("name", new StringFormatInfo()),
new FieldInfo("age", new IntFormatInfo()),
new FieldInfo("ts", new TimestampFormatInfo()));
List<FieldRelationShip> relations = Arrays
.asList(new FieldRelationShip(new FieldInfo("id", new LongFormatInfo()),
new FieldInfo("id", new LongFormatInfo())),
new FieldRelationShip(new FieldInfo("name", new StringFormatInfo()),
new FieldInfo("name", new StringFormatInfo())),
new FieldRelationShip(new FieldInfo("age", new IntFormatInfo()),
new FieldInfo("age", new IntFormatInfo())),
new FieldRelationShip(new FieldInfo("ts", new TimestampFormatInfo()),
new FieldInfo("ts", new TimestampFormatInfo()))
);

// set HIVE_CONF_DIR,or set uri and warehouse
Map<String, String> props = new HashMap<>();
props.put("catalog-type", "hive");
props.put("catalog-name", "hive_prod");
props.put("catalog-database", "default");
props.put("uri", "thrift://localhost:9083");
props.put("warehouse", "/hive/warehouse");
IcebergLoadNode node = new IcebergLoadNode("iceberg", "iceberg_output", fields, relations,
null, null, null, props, "inlong", "inlong_iceberg");
return node;
}

/**
* build node relation
*
* @param inputs extract node
* @param outputs load node
* @return node relation
*/
private NodeRelationShip buildNodeRelation(List<Node> inputs, List<Node> outputs) {
List<String> inputIds = inputs.stream().map(Node::getId).collect(Collectors.toList());
List<String> outputIds = outputs.stream().map(Node::getId).collect(Collectors.toList());
return new NodeRelationShip(inputIds, outputIds);
}

@Test
public void testIceberg() throws Exception {
EnvironmentSettings settings = EnvironmentSettings
.newInstance()
.useBlinkPlanner()
.inStreamingMode()
.build();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);
env.enableCheckpointing(10000);
StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env, settings);
Node inputNode = buildMySQLExtractNode("1");
Node outputNode = buildIcebergLoadNodeWithHiveCatalog();
StreamInfo streamInfo = new StreamInfo("1L", Arrays.asList(inputNode, outputNode),
Arrays.asList(buildNodeRelation(Collections.singletonList(inputNode),
Collections.singletonList(outputNode))));
GroupInfo groupInfo = new GroupInfo("group_id", Collections.singletonList(streamInfo));
FlinkSqlParser parser = FlinkSqlParser.getInstance(tableEnv, groupInfo);
FlinkSqlParseResult result = parser.parse();
Assert.assertTrue(!result.getLoadSqls().isEmpty() && !result.getCreateTableSqls().isEmpty());
}
}

0 comments on commit f52fd5f

Please sign in to comment.