├── .gitignore ├── README.md ├── pom.xml └── src ├── main └── java │ └── data │ └── bytedance │ └── net │ ├── ck │ └── hive │ │ ├── ClickHouseHelper.java │ │ ├── ClickHouseInputFormat.java │ │ ├── ClickHouseOutputFormat.java │ │ ├── ClickHouseRecordWriter.java │ │ ├── ClickHouseSerDe.java │ │ ├── ClickHouseStorageHandler.java │ │ ├── ClickHouseWritable.java │ │ └── Constants.java │ └── utils │ └── Tuple.java └── test └── data └── bytedance └── net └── ck └── hive ├── ClickHouseHelperTest.java ├── ClickHouseRecordWriterTest.java ├── ClickHouseSerDeTest.java └── TestHelper.java /.gitignore: -------------------------------------------------------------------------------- 1 | .idea/ 2 | *.iml 3 | target 4 | .settings/ 5 | .project/ 6 | .classpath/ 7 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # ClickHouse Hadoop 2 | 3 | Integrate ClickHouse natively with Hive, currently only writing is supported. Connecting Hadoop's massive data storage and deep processing power with the high performance of ClickHouse. 4 | 5 | ## Build the Project 6 | 7 | ```bash 8 | mvn package -Phadoop26 -DskipTests 9 | ``` 10 | 11 | ## Run the test cases 12 | 13 | It is required that a clickhouse-server is running in the localhost to correctly run the test cases. 14 | 15 | 16 | ## Usage 17 | 18 | 19 | ### Create ClickHouse table 20 | 21 | ```sql 22 | CREATE TABLE hive_test 23 | ( 24 | c1 String, 25 | c2 Float64, 26 | c3 String 27 | ) 28 | ENGINE = MergeTree() 29 | PARTITION BY c3 30 | ORDER BY c1 31 | ``` 32 | 33 | ### Create Hive External Table 34 | 35 | Before starting the hive cli, set the environment variable `HIVE_AUX_JARS_PATH` 36 | 37 | ```bash 38 | export HIVE_AUX_JARS_PATH=/target/clickhouse-hadoop-.jar 39 | ``` 40 | 41 | Then start the `hive-cli` and create Hive external table: 42 | 43 | ```sql 44 | CREATE EXTERNAL TABLE default.ck_test( 45 | c1 string, 46 | c2 double, 47 | c3 string 48 | ) 49 | STORED BY 'data.bytedance.net.ck.hive.ClickHouseStorageHandler' 50 | TBLPROPERTIES('clickhouse.conn.urls'='jdbc:clickhouse://:,jdbc:clickhouse://:', 51 | 'clickhouse.table.name'='hive_test'); 52 | ``` 53 | 54 | ### Data Ingestion 55 | 56 | In `hive-cli` 57 | 58 | ```sql 59 | INSERT INTO default.ck_test 60 | select c1, c2, c3 FROM default.source_table where part='part_val' 61 | ``` 62 | 63 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | data.bytedance.net 8 | clickhouse-hadoop 9 | 1.0-SNAPSHOT 10 | 11 | 12 | 13 | org.apache.hive 14 | hive-exec 15 | 1.2.2 16 | compile 17 | 18 | 19 | org.apache.hive 20 | hive-metastore 21 | 1.2.2 22 | compile 23 | 24 | 25 | org.apache.hive 26 | hive-common 27 | 1.2.2 28 | compile 29 | 30 | 31 | org.slf4j 32 | slf4j-api 33 | 1.7.25 34 | 35 | 36 | com.github.housepower 37 | clickhouse-native-jdbc 38 | 1.6-stable 39 | 40 | 41 | org.slf4j 42 | slf4j-log4j12 43 | 1.7.25 44 | compile 45 | 46 | 47 | 48 | org.junit.jupiter 49 | junit-jupiter-api 50 | 5.3.1 51 | test 52 | 53 | 54 | 55 | 56 | hadoop26 57 | 58 | true 59 | 60 | 61 | 62 | org.apache.hadoop 63 | hadoop-common 64 | 2.6.5 65 | provided 66 | 67 | 68 | org.apache.hadoop 69 | hadoop-mapreduce-client-core 70 | 2.6.5 71 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | 79 | 80 | org.apache.maven.plugins 81 | maven-compiler-plugin 82 | 3.8.0 83 | 84 | 1.8 85 | 1.8 86 | true 87 | true 88 | true 89 | 90 | 91 | 92 | 93 | org.apache.maven.plugins 94 | maven-shade-plugin 95 | 3.2.0 96 | 97 | 98 | 99 | *:* 100 | 101 | META-INF/*.SF 102 | META-INF/*.DSA 103 | META-INF/*.RSA 104 | 105 | 106 | 107 | 108 | 109 | 110 | package 111 | 112 | shade 113 | 114 | 115 | 116 | 117 | 118 | 119 | 120 | 121 | -------------------------------------------------------------------------------- /src/main/java/data/bytedance/net/ck/hive/ClickHouseHelper.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.ck.hive; 2 | 3 | import data.bytedance.net.utils.Tuple; 4 | import org.slf4j.Logger; 5 | import org.slf4j.LoggerFactory; 6 | 7 | import java.sql.*; 8 | import java.util.ArrayList; 9 | import java.util.HashMap; 10 | import java.util.List; 11 | import java.util.Random; 12 | 13 | public class ClickHouseHelper { 14 | private static final Logger logger = LoggerFactory.getLogger(ClickHouseHelper.class); 15 | private static HashMap, ClickHouseHelper> ckHelperCache = new HashMap<>(); 16 | private final String[] connStrings; 17 | private final String tableName; 18 | private List columnNames = new ArrayList<>(); 19 | private List columnTypes = new ArrayList<>(); 20 | private HashMap nameTypeMap = new HashMap<>(); 21 | private Random rnd = new Random(); 22 | 23 | static { 24 | try { 25 | Class.forName("com.github.housepower.jdbc.ClickHouseDriver"); 26 | } catch (ClassNotFoundException e) { 27 | logger.error("Can't find suitable driver", e); 28 | } 29 | } 30 | 31 | public static ClickHouseHelper getClickHouseHelper(String connStrings, String tableName) throws SQLException { 32 | Tuple k = new Tuple<>(connStrings, tableName); 33 | 34 | if (ckHelperCache.containsKey(k)) { 35 | return ckHelperCache.get(k); 36 | } else { 37 | String[] connStrArr = connStrings.split(","); 38 | ClickHouseHelper helper = new ClickHouseHelper(connStrArr, tableName); 39 | ckHelperCache.put(k, helper); 40 | return helper; 41 | } 42 | } 43 | 44 | 45 | private ClickHouseHelper(String[] connStrings, String tableName) throws SQLException { 46 | this.connStrings = connStrings; 47 | this.tableName = tableName; 48 | initColumnNamesAndTypesFromSystemQuery(); 49 | } 50 | 51 | public Connection getClickHouseConnection() throws SQLException { 52 | Connection connection = DriverManager.getConnection(pickConnStr()); 53 | return connection; 54 | } 55 | 56 | 57 | public void initColumnNamesAndTypesFromSystemQuery() throws SQLException { 58 | Connection conn = getClickHouseConnection(); 59 | try { 60 | Statement stmt = conn.createStatement(); 61 | String query = "SELECT name, type from system.columns where table = '" + getTableName() + "';"; 62 | logger.info("Initializing columns and types with " + query); 63 | ResultSet rs = stmt.executeQuery(query); 64 | while (rs.next()) { 65 | this.columnNames.add(rs.getString(1)); 66 | this.columnTypes.add(rs.getString(2)); 67 | nameTypeMap.put(rs.getString(1), rs.getString(2)); 68 | } 69 | } finally { 70 | conn.close(); 71 | } 72 | } 73 | 74 | public HashMap getNameTypeMap() { 75 | return nameTypeMap; 76 | } 77 | 78 | public String pickConnStr() { 79 | return connStrings[rnd.nextInt(connStrings.length)]; 80 | } 81 | 82 | public String getTableName() { 83 | return tableName; 84 | } 85 | 86 | 87 | public List getColumnNames() { 88 | return columnNames; 89 | } 90 | 91 | public List getColumnTypes() { 92 | return columnTypes; 93 | } 94 | 95 | public static String ClickHouseToHiveType(String hiveTypeStr) { 96 | switch (hiveTypeStr) { 97 | case "Int8": 98 | case "UInt8": 99 | return "tinyint"; 100 | case "Int16": 101 | case "UInt16": 102 | return "smallint"; 103 | case "Int32": 104 | case "UInt32": 105 | return "int"; 106 | case "Int64": 107 | case "UInt64": 108 | return "bigint"; 109 | case "Float64": 110 | return "double"; 111 | case "Float32": 112 | return "float"; 113 | case "String": 114 | return "string"; 115 | case "DateTime": 116 | return "timestamp"; 117 | case "Date": 118 | return "date"; 119 | default: 120 | return null; 121 | } 122 | } 123 | } 124 | -------------------------------------------------------------------------------- /src/main/java/data/bytedance/net/ck/hive/ClickHouseInputFormat.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.ck.hive; 2 | 3 | import org.apache.hadoop.mapred.InputFormat; 4 | import org.apache.hadoop.mapred.InputSplit; 5 | import org.apache.hadoop.mapred.JobConf; 6 | import org.apache.hadoop.mapred.RecordReader; 7 | import org.apache.hadoop.mapred.Reporter; 8 | 9 | import java.io.IOException; 10 | 11 | /** 12 | * A dummpy implementation of input format, read is not supported 13 | */ 14 | public class ClickHouseInputFormat implements InputFormat { 15 | @Override 16 | public InputSplit[] getSplits(JobConf jobConf, int i) throws IOException { 17 | return new InputSplit[0]; 18 | } 19 | 20 | @Override 21 | public RecordReader getRecordReader(InputSplit inputSplit, JobConf jobConf, Reporter reporter) throws IOException { 22 | return null; 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/main/java/data/bytedance/net/ck/hive/ClickHouseOutputFormat.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.ck.hive; 2 | 3 | 4 | import org.apache.hadoop.fs.FileSystem; 5 | import org.apache.hadoop.fs.Path; 6 | import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter; 7 | import org.apache.hadoop.hive.ql.io.HiveOutputFormat; 8 | import org.apache.hadoop.io.Writable; 9 | import org.apache.hadoop.mapred.JobConf; 10 | import org.apache.hadoop.util.Progressable; 11 | import org.slf4j.Logger; 12 | import org.slf4j.LoggerFactory; 13 | 14 | import java.io.IOException; 15 | import java.sql.SQLException; 16 | import java.util.List; 17 | import java.util.Properties; 18 | import java.util.Map.Entry; 19 | import java.util.Set; 20 | 21 | public class ClickHouseOutputFormat implements HiveOutputFormat { 22 | private static Logger logger = LoggerFactory.getLogger(ClickHouseOutputFormat.class); 23 | 24 | @Override 25 | public RecordWriter getHiveRecordWriter( 26 | JobConf jobConf, 27 | Path path, 28 | Class aClass, 29 | boolean b, 30 | Properties tblProps, 31 | Progressable progressable 32 | ) throws IOException { 33 | logger.info("Table Properties"); 34 | Set> entries = tblProps.entrySet(); 35 | for (Entry entry: entries) { 36 | logger.info(entry.getKey() + " : " + entry.getValue()); 37 | } 38 | 39 | String ckConnectionStrings = tblProps.getProperty(Constants.CK_CONN_STRS); 40 | String tblName = tblProps.getProperty(Constants.CK_TBL_NAME); 41 | // Table name and connection string are required 42 | if (ckConnectionStrings == null || ckConnectionStrings == "") { 43 | throw new IOException(Constants.CK_CONN_STRS + " must be set in TBLPROPERTIES"); 44 | } 45 | 46 | if (tblName == null || tblName == "") { 47 | throw new IOException(Constants.CK_TBL_NAME + " must be set in TBLPROPERTIES"); 48 | } 49 | 50 | String batchSizeStr = tblProps.getProperty(Constants.CK_BATCH_SIZE); 51 | int batchSize = 0; 52 | try { 53 | if (batchSizeStr == null || batchSizeStr == "") { 54 | batchSize = Constants.DEFAULT_BATCH_SIZE; 55 | } else { 56 | batchSize = Integer.parseInt(batchSizeStr); 57 | } 58 | } catch (NumberFormatException e) { 59 | logger.info(String.format("Parsing %s failed, use default", batchSizeStr), e); 60 | batchSize = Constants.DEFAULT_BATCH_SIZE; 61 | } 62 | 63 | ClickHouseHelper ckHelper; 64 | try { 65 | ckHelper = ClickHouseHelper.getClickHouseHelper(ckConnectionStrings, tblName); 66 | } catch (SQLException e) { 67 | logger.error("Can't create ckHelper ", e); 68 | throw new IOException(e); 69 | } 70 | List columnNames = ckHelper.getColumnNames(); 71 | List columnTypes = ckHelper.getColumnTypes(); 72 | return new ClickHouseRecordWriter(ckHelper, batchSize, tblName, columnNames, columnTypes); 73 | } 74 | 75 | @Override 76 | public org.apache.hadoop.mapred.RecordWriter getRecordWriter( 77 | FileSystem fileSystem, JobConf jobConf, String s, Progressable progressable 78 | ) throws IOException { 79 | throw new UnsupportedOperationException(); 80 | } 81 | 82 | @Override 83 | public void checkOutputSpecs(FileSystem fileSystem, JobConf jobConf) throws IOException { 84 | // hive should not invoke this method 85 | } 86 | } 87 | -------------------------------------------------------------------------------- /src/main/java/data/bytedance/net/ck/hive/ClickHouseRecordWriter.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.ck.hive; 2 | 3 | import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter; 4 | import org.apache.hadoop.io.Writable; 5 | import org.joda.time.DateTime; 6 | import org.slf4j.Logger; 7 | import org.slf4j.LoggerFactory; 8 | import parquet.Preconditions; 9 | 10 | import java.io.IOException; 11 | import java.sql.*; 12 | import java.util.ArrayList; 13 | import java.util.Arrays; 14 | import java.util.List; 15 | import java.util.Map; 16 | 17 | public class ClickHouseRecordWriter implements RecordWriter { 18 | private static Logger logger = LoggerFactory.getLogger(ClickHouseRecordWriter.class); 19 | private final int batchSize; 20 | // the column names of the remote clickhouse table 21 | private final List clickhouseColNames; 22 | // the column types of the remote clickhouse table 23 | private final List clickhouseColTypes; 24 | private final String insertQuery; 25 | private final ClickHouseHelper clickHouseHelper; 26 | 27 | private ArrayList> data = new ArrayList<>(); 28 | 29 | public ClickHouseRecordWriter(ClickHouseHelper helper, int batchSize, String tableName, List columnNames, 30 | List columnTypes) { 31 | this.batchSize = batchSize; 32 | this.clickhouseColNames = columnNames; 33 | this.clickhouseColTypes = columnTypes; 34 | this.insertQuery = constructInsertQuery(tableName, columnNames); 35 | this.clickHouseHelper = helper; 36 | 37 | } 38 | 39 | public static String constructInsertQuery(String tableName, List columnNames) { 40 | StringBuilder sql = new StringBuilder(); 41 | sql.append("INSERT INTO ").append(tableName).append(" "); 42 | 43 | String fields = String.join(",", columnNames); 44 | String[] valueSlice = new String[columnNames.size()]; 45 | Arrays.fill(valueSlice, "?"); 46 | String values = String.join(",", valueSlice); 47 | sql.append("(").append(fields).append(") VALUES (").append(values).append(")"); 48 | return sql.toString(); 49 | } 50 | 51 | private static void addValuesToBatch(Map value, PreparedStatement stmt, List columnNames, 52 | List columnTypes) throws SQLException { 53 | Preconditions.checkArgument(columnNames.size() == columnTypes.size(), 54 | "Column types and column names must be matched"); 55 | for (int i = 0; i < columnNames.size(); i++) { 56 | String columnName = columnNames.get(i); 57 | String columnType = columnTypes.get(i); 58 | Object obj = value.containsKey(columnName) ? value.get(columnName) : null; 59 | if (obj != null) { 60 | switch (columnType) { 61 | case "Int8": 62 | case "UInt8": 63 | stmt.setByte(i + 1, (byte) obj); 64 | break; 65 | case "Int16": 66 | case "UInt16": 67 | stmt.setShort(i + 1, (short) obj); 68 | break; 69 | case "Int32": 70 | case "UInt32": 71 | stmt.setInt(i + 1, (int) obj); 72 | break; 73 | case "Int64": 74 | case "UInt64": 75 | stmt.setLong(i + 1, (long) obj); 76 | break; 77 | case "Float32": 78 | stmt.setFloat(i + 1, (float) obj); 79 | break; 80 | case "Float64": 81 | stmt.setDouble(i + 1, (double) obj); 82 | break; 83 | case "String": 84 | stmt.setString(i + 1, (String) obj); 85 | break; 86 | case "DateTime": 87 | stmt.setTimestamp(i + 1, (Timestamp) obj); 88 | break; 89 | case "Date": 90 | stmt.setDate(i + 1, (Date) obj); 91 | break; 92 | default: 93 | throw new SQLException(String.format("Un-supported type %s", columnType)); 94 | } 95 | } else { 96 | switch (columnType) { 97 | case "Int8": 98 | case "UInt8": 99 | stmt.setByte(i + 1, (byte) 0); 100 | break; 101 | case "Int16": 102 | case "UInt16": 103 | stmt.setShort(i + 1, (byte) 0); 104 | break; 105 | case "Int32": 106 | case "UInt32": 107 | stmt.setInt(i + 1, 0); 108 | break; 109 | case "Int64": 110 | case "UInt64": 111 | stmt.setLong(i + 1, 0L); 112 | break; 113 | case "Float32": 114 | stmt.setFloat(i + 1, 0.0f); 115 | break; 116 | case "Float64": 117 | stmt.setDouble(i + 1, 0.0); 118 | break; 119 | case "String": 120 | stmt.setString(i + 1, ""); 121 | break; 122 | case "DateTime": 123 | stmt.setTimestamp(i + 1, new Timestamp(DateTime.now().getMillis())); 124 | break; 125 | case "Date": 126 | stmt.setDate(i + 1, new Date(DateTime.now().getMillis())); 127 | break; 128 | default: 129 | throw new SQLException(String.format("Un-supported type %s", columnType)); 130 | } 131 | } 132 | 133 | } 134 | stmt.addBatch(); 135 | } 136 | 137 | // reall does the flush 138 | private void doFlush() throws IOException { 139 | if (data.isEmpty()) { 140 | return; 141 | } 142 | Connection connection = null; 143 | PreparedStatement statement = null; 144 | try { 145 | connection = clickHouseHelper.getClickHouseConnection(); 146 | statement = connection.prepareStatement(this.insertQuery); 147 | 148 | for (Map value : data) { 149 | addValuesToBatch(value, statement, clickhouseColNames, clickhouseColTypes); 150 | } 151 | statement.executeBatch(); 152 | 153 | logger.info("Flushed " + data.size() + " rows of data"); 154 | } catch (SQLException e) { 155 | throw new IOException(e); 156 | } finally { 157 | try { 158 | if (statement != null) { 159 | statement.close(); 160 | } 161 | if (connection != null) { 162 | connection.close(); 163 | } 164 | } catch (SQLException e) { 165 | logger.error("Error closing resource", e); 166 | } 167 | } 168 | } 169 | 170 | public void flush(int retry) throws IOException { 171 | try { 172 | while (retry-- > 0) { 173 | try { 174 | doFlush(); 175 | break; 176 | } catch (Exception e) { 177 | logger.error("Error flushing, retrying", e); 178 | } 179 | } 180 | } finally { 181 | data.clear(); 182 | } 183 | } 184 | 185 | // Write the data, the writable comes from the ClickHouseSerDe 186 | @Override 187 | public void write(Writable w) throws IOException { 188 | ClickHouseWritable ckWritable = (ClickHouseWritable) w; 189 | data.add(ckWritable.getValue()); 190 | if (data.size() >= batchSize) { 191 | flush(3); 192 | } 193 | } 194 | 195 | @Override 196 | public void close(boolean abort) throws IOException { 197 | logger.info("Closing Writer, flush remaining: " + data.size()); 198 | flush(3); 199 | } 200 | } 201 | -------------------------------------------------------------------------------- /src/main/java/data/bytedance/net/ck/hive/ClickHouseSerDe.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.ck.hive; 2 | 3 | import com.google.common.collect.Lists; 4 | import org.apache.hadoop.conf.Configuration; 5 | import org.apache.hadoop.hive.serde.serdeConstants; 6 | import org.apache.hadoop.hive.serde2.AbstractSerDe; 7 | import org.apache.hadoop.hive.serde2.SerDeException; 8 | import org.apache.hadoop.hive.serde2.SerDeStats; 9 | import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; 10 | import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; 11 | import org.apache.hadoop.hive.serde2.objectinspector.StructField; 12 | import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; 13 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; 14 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector; 15 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector; 16 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector; 17 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector; 18 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector; 19 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector; 20 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector; 21 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector; 22 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector; 23 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector; 24 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector; 25 | import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; 26 | import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; 27 | import org.apache.hadoop.io.Writable; 28 | import org.slf4j.Logger; 29 | import org.slf4j.LoggerFactory; 30 | 31 | import java.util.*; 32 | 33 | public class ClickHouseSerDe extends AbstractSerDe { 34 | private static final Logger logger = LoggerFactory.getLogger(ClickHouseSerDe.class); 35 | private ObjectInspector inspector; 36 | 37 | // The column and type mapping 38 | private String[] columns; 39 | private PrimitiveTypeInfo[] types; 40 | 41 | /** 42 | * Set up the tlbProps of the destination - columns - column.types 43 | * 44 | * @param configuration 45 | * @param tblProps the properties defined in the TBLPROPERTIES clause of 46 | * the CREATE TABLE statement 47 | * @throws SerDeException 48 | */ 49 | @Override 50 | public void initialize(Configuration configuration, Properties tblProps) throws SerDeException { 51 | if (logger.isDebugEnabled()) { 52 | logger.debug("tblProps" + tblProps); 53 | } 54 | // a list of connection strings separated by comma, for load balancing 55 | String ckConnectionStrings = tblProps.getProperty(Constants.CK_CONN_STRS); 56 | String tblName = tblProps.getProperty(Constants.CK_TBL_NAME); 57 | 58 | // Table name and connection string are required 59 | if (ckConnectionStrings == null || ckConnectionStrings == "") { 60 | throw new SerDeException(Constants.CK_CONN_STRS + " must be set in TBLPROPERTIES"); 61 | } 62 | 63 | if (tblName == null || tblName == "") { 64 | throw new SerDeException(Constants.CK_TBL_NAME + " must be set in TBLPROPERTIES"); 65 | } 66 | 67 | String columnNameProperty = tblProps.getProperty(serdeConstants.LIST_COLUMNS); 68 | String columnTypeProperty = tblProps.getProperty(serdeConstants.LIST_COLUMN_TYPES); 69 | logger.info("Column Names: " + columnNameProperty); 70 | logger.info("Column Types: " + columnTypeProperty); 71 | 72 | // if columns and column types are not explicitly defined, we need to find them 73 | // out from clickhouse schema 74 | List columnNames = Arrays.asList(columnNameProperty.split(",")); 75 | List columnTypeNames = Arrays.asList(columnTypeProperty.split(":")); 76 | List columnTypes = Lists.transform(columnTypeNames, 77 | (String s) -> TypeInfoFactory.getPrimitiveTypeInfo(s)); 78 | List inspectors = new ArrayList<>(); 79 | 80 | columns = columnNames.toArray(new String[columnNames.size()]); 81 | types = columnTypes.toArray(new PrimitiveTypeInfo[columnTypes.size()]); 82 | 83 | inspectors.addAll(Lists.transform(columnTypes, 84 | (PrimitiveTypeInfo type) -> PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(type))); 85 | inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); 86 | } 87 | 88 | @Override 89 | public Class getSerializedClass() { 90 | return ClickHouseWritable.class; 91 | } 92 | 93 | /** 94 | * This method takes an object representing a row of data from Hive, and uses 95 | * the ObjectInspector to get the data for each data and serialize it into 96 | * Writable -- a serializable object 97 | * 98 | * @param o 99 | * @param objectInspector 100 | * @return 101 | * @throws SerDeException 102 | */ 103 | @Override 104 | public Writable serialize(Object o, ObjectInspector objectInspector) throws SerDeException { 105 | if (objectInspector.getCategory() != ObjectInspector.Category.STRUCT) { 106 | throw new SerDeException(getClass().toString() + " can only serialize struct types, but we got: " 107 | + objectInspector.getTypeName()); 108 | } 109 | 110 | StructObjectInspector soi = (StructObjectInspector) objectInspector; 111 | List fields = soi.getAllStructFieldRefs(); 112 | List values = soi.getStructFieldsDataAsList(o); 113 | 114 | final Map value = new HashMap<>(); 115 | for (int i = 0; i < columns.length; i++) { 116 | if (values.get(i) == null) { 117 | // null, we just add it 118 | value.put(columns[i], null); 119 | continue; 120 | } 121 | final Object res; 122 | switch (types[i].getPrimitiveCategory()) { 123 | case TIMESTAMP: 124 | res = ((TimestampObjectInspector) fields.get(i).getFieldObjectInspector()) 125 | .getPrimitiveJavaObject(values.get(i)); 126 | break; 127 | case BYTE: 128 | res = ((ByteObjectInspector) fields.get(i).getFieldObjectInspector()).get(values.get(i)); 129 | break; 130 | case SHORT: 131 | res = ((ShortObjectInspector) fields.get(i).getFieldObjectInspector()).get(values.get(i)); 132 | break; 133 | case INT: 134 | res = ((IntObjectInspector) fields.get(i).getFieldObjectInspector()).get(values.get(i)); 135 | break; 136 | case LONG: 137 | res = ((LongObjectInspector) fields.get(i).getFieldObjectInspector()).get(values.get(i)); 138 | break; 139 | case FLOAT: 140 | res = ((FloatObjectInspector) fields.get(i).getFieldObjectInspector()).get(values.get(i)); 141 | break; 142 | case DOUBLE: 143 | res = ((DoubleObjectInspector) fields.get(i).getFieldObjectInspector()).get(values.get(i)); 144 | break; 145 | case CHAR: 146 | res = ((HiveCharObjectInspector) fields.get(i).getFieldObjectInspector()) 147 | .getPrimitiveJavaObject(values.get(i)).getValue(); 148 | break; 149 | case VARCHAR: 150 | res = ((HiveVarcharObjectInspector) fields.get(i).getFieldObjectInspector()) 151 | .getPrimitiveJavaObject(values.get(i)).getValue(); 152 | break; 153 | case STRING: 154 | res = ((StringObjectInspector) fields.get(i).getFieldObjectInspector()) 155 | .getPrimitiveJavaObject(values.get(i)); 156 | break; 157 | case DATE: 158 | res = ((DateObjectInspector) fields.get(i).getFieldObjectInspector()) 159 | .getPrimitiveJavaObject(values.get(i)); 160 | break; 161 | default: 162 | throw new SerDeException("Unsupported type: " + types[i].getPrimitiveCategory()); 163 | } 164 | value.put(columns[i], res); 165 | } 166 | return new ClickHouseWritable(value); 167 | } 168 | 169 | /** 170 | * This method does the work of deserializing a record into Java objects that 171 | * hive can work with via the ObjectInspector interface We don't support 172 | * querying at this moment, thus this method will throw an exception 173 | * 174 | * @param writable 175 | * @return 176 | * @throws SerDeException 177 | */ 178 | @Override 179 | public Object deserialize(Writable writable) throws SerDeException { 180 | throw new UnsupportedOperationException("Reads are not allowed"); 181 | } 182 | 183 | /** 184 | * Get the object inspector that can be used to navigate through the internal 185 | * structure of the Object returned from deserialize(...). Not supported for 186 | * this case 187 | * 188 | * @return 189 | * @throws SerDeException 190 | */ 191 | @Override 192 | public ObjectInspector getObjectInspector() throws SerDeException { 193 | return inspector; 194 | } 195 | 196 | @Override 197 | public SerDeStats getSerDeStats() { 198 | return null; 199 | } 200 | } 201 | -------------------------------------------------------------------------------- /src/main/java/data/bytedance/net/ck/hive/ClickHouseStorageHandler.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.ck.hive; 2 | 3 | import org.apache.hadoop.conf.Configuration; 4 | import org.apache.hadoop.hive.metastore.HiveMetaHook; 5 | import org.apache.hadoop.hive.metastore.api.MetaException; 6 | import org.apache.hadoop.hive.metastore.api.Table; 7 | import org.apache.hadoop.hive.ql.metadata.HiveException; 8 | import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; 9 | import org.apache.hadoop.hive.ql.plan.TableDesc; 10 | import org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider; 11 | import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider; 12 | import org.apache.hadoop.hive.serde2.SerDe; 13 | 14 | import java.util.Map; 15 | 16 | public class ClickHouseStorageHandler implements HiveStorageHandler { 17 | private Configuration conf; 18 | @Override 19 | public Class getInputFormatClass() { 20 | return ClickHouseInputFormat.class; 21 | } 22 | 23 | @Override 24 | public Class getOutputFormatClass() { 25 | return ClickHouseOutputFormat.class; 26 | } 27 | 28 | @Override 29 | public Class getSerDeClass() { 30 | return ClickHouseSerDe.class; 31 | } 32 | 33 | @Override 34 | public HiveMetaHook getMetaHook() { 35 | return new CkHook(); 36 | } 37 | 38 | @Override 39 | public HiveAuthorizationProvider getAuthorizationProvider() throws HiveException { 40 | return new DefaultHiveAuthorizationProvider(); 41 | } 42 | 43 | @Override 44 | public void configureInputJobProperties(TableDesc tableDesc, Map jobProperties) { 45 | // configureTableJobProperties(tableDesc, jobProperties); 46 | } 47 | 48 | @Override 49 | public void configureOutputJobProperties(TableDesc tableDesc, Map jobProperties) { 50 | // configureTableJobProperties(tableDesc, jobProperties); 51 | } 52 | 53 | @Override 54 | public void configureTableJobProperties(TableDesc tableDesc, Map jopProperties) { 55 | /* 56 | Properties tableProps = tableDesc.getProperties(); 57 | for (String key: tableProps.stringPropertyNames()) { 58 | if (conf == null || conf.get(key) == null) { 59 | jopProperties.put(key, tableProps.getProperty(key)); 60 | } 61 | } 62 | */ 63 | } 64 | 65 | @Override 66 | public void configureJobConf(TableDesc tableDesc, org.apache.hadoop.mapred.JobConf jobConf) { 67 | } 68 | 69 | @Override 70 | public void setConf(Configuration configuration) { 71 | conf = configuration; 72 | } 73 | 74 | @Override 75 | public Configuration getConf() { 76 | return conf; 77 | } 78 | 79 | /** 80 | * Dummy implementation, do nothing 81 | */ 82 | private static class CkHook implements HiveMetaHook { 83 | @Override 84 | public void preCreateTable(Table table) throws MetaException { 85 | 86 | } 87 | 88 | @Override 89 | public void rollbackCreateTable(Table table) throws MetaException { 90 | 91 | } 92 | 93 | @Override 94 | public void commitCreateTable(Table table) throws MetaException { 95 | 96 | } 97 | 98 | @Override 99 | public void preDropTable(Table table) throws MetaException { 100 | 101 | } 102 | 103 | @Override 104 | public void rollbackDropTable(Table table) throws MetaException { 105 | 106 | } 107 | 108 | @Override 109 | public void commitDropTable(Table table, boolean b) throws MetaException { 110 | 111 | } 112 | } 113 | 114 | 115 | } 116 | 117 | -------------------------------------------------------------------------------- /src/main/java/data/bytedance/net/ck/hive/ClickHouseWritable.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.ck.hive; 2 | 3 | import org.apache.hadoop.io.Writable; 4 | 5 | import java.io.DataInput; 6 | import java.io.DataOutput; 7 | import java.io.IOException; 8 | import java.util.Map; 9 | 10 | /** 11 | * Represents the serialized data getting from Hive. 12 | */ 13 | public class ClickHouseWritable implements Writable { 14 | Map value; 15 | 16 | public ClickHouseWritable(Map value) { 17 | this.value = value; 18 | } 19 | 20 | public Map getValue() { 21 | return value; 22 | } 23 | 24 | /** 25 | * Serialize the fields of this object to out 26 | * 27 | * @param out 28 | * @throws IOException 29 | */ 30 | @Override 31 | public void write(DataOutput out) throws IOException { 32 | throw new UnsupportedOperationException(); 33 | } 34 | 35 | /** 36 | * Deserialize the fields of this object from in 37 | * 38 | * @param in 39 | * @throws IOException 40 | */ 41 | @Override 42 | public void readFields(DataInput in) throws IOException { 43 | throw new UnsupportedOperationException(); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/main/java/data/bytedance/net/ck/hive/Constants.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.ck.hive; 2 | 3 | public class Constants { 4 | // config key for the column names 5 | public static final String LIST_COLUMNS = "columns"; 6 | // the type of each column 7 | public static final String CK_CONN_STRS = "clickhouse.conn.urls"; 8 | public static final String CK_TBL_NAME = "clickhouse.table.name"; 9 | public static final String CK_BATCH_SIZE = "clickhouse.insert.batch.size"; 10 | public static final int DEFAULT_BATCH_SIZE = 500; 11 | } 12 | -------------------------------------------------------------------------------- /src/main/java/data/bytedance/net/utils/Tuple.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.utils; 2 | 3 | public class Tuple { 4 | public final X x; 5 | public final Y y; 6 | 7 | public Tuple(X x, Y y) { 8 | this.x = x; 9 | this.y = y; 10 | } 11 | 12 | @Override 13 | public String toString() { 14 | return "(" + x + "," + y + ")"; 15 | } 16 | 17 | @Override 18 | public boolean equals(Object other) { 19 | if (other == this) { 20 | return true; 21 | } 22 | 23 | if (!(other instanceof Tuple)) { 24 | return false; 25 | } 26 | 27 | Tuple other_ = (Tuple) other; 28 | 29 | // this may cause NPE if nulls are valid values for x or y. The logic may be improved to handle nulls properly, if needed. 30 | return other_.x.equals(this.x) && other_.y.equals(this.y); 31 | } 32 | 33 | @Override 34 | public int hashCode() { 35 | final int prime = 31; 36 | int result = 1; 37 | result = prime * result + ((x == null) ? 0 : x.hashCode()); 38 | result = prime * result + ((y == null) ? 0 : y.hashCode()); 39 | return result; 40 | } 41 | } 42 | 43 | -------------------------------------------------------------------------------- /src/test/data/bytedance/net/ck/hive/ClickHouseHelperTest.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.ck.hive; 2 | 3 | import junit.framework.Assert; 4 | import org.junit.jupiter.api.Test; 5 | 6 | import java.sql.Connection; 7 | import java.sql.DriverManager; 8 | import java.sql.SQLException; 9 | import java.sql.Statement; 10 | import java.util.List; 11 | 12 | public class ClickHouseHelperTest { 13 | @Test 14 | void testInitColumnsFromCK() throws SQLException, ClassNotFoundException { 15 | Class.forName("com.github.housepower.jdbc.ClickHouseDriver"); 16 | Connection conn = DriverManager.getConnection(TestHelper.ckConnStr); 17 | try { 18 | Statement stmt = conn.createStatement(); 19 | stmt.executeQuery("drop table if exists test_ck_helper;"); 20 | stmt.executeQuery("create table test_ck_helper(day default toDate( toDateTime(timestamp) ), timestamp UInt32, name String, impressions UInt32) Engine=MergeTree(day, (timestamp, name), 8192)"); 21 | ClickHouseHelper helper = ClickHouseHelper.getClickHouseHelper(TestHelper.ckConnStr, "test_ck_helper"); 22 | List columnNames = helper.getColumnNames(); 23 | List columnTypes = helper.getColumnTypes(); 24 | Assert.assertEquals(4, columnNames.size()); 25 | Assert.assertEquals(4, columnTypes.size()); 26 | Assert.assertEquals("day", columnNames.get(0)); 27 | Assert.assertEquals("Date", columnTypes.get(0)); 28 | Assert.assertEquals("timestamp", columnNames.get(1)); 29 | Assert.assertEquals("UInt32", columnTypes.get(1)); 30 | Assert.assertEquals("name", columnNames.get(2)); 31 | Assert.assertEquals("String", columnTypes.get(2)); 32 | Assert.assertEquals("impressions", columnNames.get(3)); 33 | Assert.assertEquals("UInt32", columnTypes.get(3)); 34 | stmt.executeQuery("drop table if exists test_ck_helper;"); 35 | 36 | } finally { 37 | conn.close(); 38 | } 39 | 40 | } 41 | } -------------------------------------------------------------------------------- /src/test/data/bytedance/net/ck/hive/ClickHouseRecordWriterTest.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.ck.hive; 2 | 3 | import junit.framework.Assert; 4 | import org.apache.hadoop.conf.Configuration; 5 | import org.apache.hadoop.hive.common.type.HiveChar; 6 | import org.apache.hadoop.hive.common.type.HiveVarchar; 7 | import org.apache.hadoop.hive.serde2.SerDeException; 8 | import org.apache.hadoop.hive.serde2.SerDeUtils; 9 | import org.apache.hadoop.hive.serde2.io.ByteWritable; 10 | import org.apache.hadoop.hive.serde2.io.DoubleWritable; 11 | import org.apache.hadoop.hive.serde2.io.HiveCharWritable; 12 | import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable; 13 | import org.apache.hadoop.hive.serde2.io.ShortWritable; 14 | import org.apache.hadoop.hive.serde2.io.TimestampWritable; 15 | import org.apache.hadoop.io.FloatWritable; 16 | import org.apache.hadoop.io.IntWritable; 17 | import org.apache.hadoop.io.LongWritable; 18 | import org.apache.hadoop.io.Text; 19 | import org.junit.jupiter.api.AfterEach; 20 | import org.junit.jupiter.api.BeforeEach; 21 | import org.junit.jupiter.api.Test; 22 | 23 | import java.io.IOException; 24 | import java.sql.Connection; 25 | import java.sql.DriverManager; 26 | import java.sql.ResultSet; 27 | import java.sql.SQLException; 28 | import java.sql.Statement; 29 | import java.sql.Timestamp; 30 | import java.util.ArrayList; 31 | import java.util.List; 32 | import java.util.Properties; 33 | 34 | public class ClickHouseRecordWriterTest { 35 | 36 | private static final String TABLE_NAME = "testWriter"; 37 | private static final String CLICKHOUSE_CREATE_TABLE_STMT = "CREATE TABLE IF NOT EXISTS testWriter\n" + 38 | "(\n" + 39 | " time Datetime,\n" + 40 | " c0 String,\n" + 41 | " c1 String,\n" + 42 | " c2 String,\n" + 43 | " c3 Float64,\n" + 44 | " c4 Float32,\n" + 45 | " c5 Int64,\n" + 46 | " c6 Int32,\n" + 47 | " c7 Int16,\n" + 48 | " c8 Int8\n" + 49 | ")\n" + 50 | "ENGINE = MergeTree\n" + 51 | "PARTITION BY time\n" + 52 | "ORDER BY c0\n" + 53 | "SETTINGS index_granularity = 8192"; 54 | 55 | private static final String COLUMN_HIVE_TYPES = "timestamp,string,char(6),varchar(8),double,float,bigint,int,smallint,tinyint"; 56 | private static ClickHouseHelper helper; 57 | 58 | @BeforeEach 59 | public void beforeTest() throws SQLException, ClassNotFoundException { 60 | Class.forName("com.github.housepower.jdbc.ClickHouseDriver"); 61 | Connection c = DriverManager.getConnection(TestHelper.ckConnStr); 62 | try { 63 | Statement stmt = c.createStatement(); 64 | stmt.executeQuery("DROP TABLE IF EXISTS " + TABLE_NAME); 65 | stmt.executeQuery(CLICKHOUSE_CREATE_TABLE_STMT); 66 | } finally { 67 | c.close(); 68 | } 69 | helper = ClickHouseHelper.getClickHouseHelper(TestHelper.ckConnStr, TABLE_NAME); 70 | } 71 | 72 | @AfterEach 73 | public void afterTest() throws SQLException { 74 | Connection c = helper.getClickHouseConnection(); 75 | try { 76 | Statement stmt = c.createStatement(); 77 | stmt.executeQuery("DROP TABLE IF EXISTS " + TABLE_NAME); 78 | } finally { 79 | c.close(); 80 | } 81 | 82 | } 83 | 84 | /** 85 | * Test that the insert query can be correctly constructed from column names and table name 86 | */ 87 | @Test 88 | public void testConstructInsertQuery() { 89 | List columns = new ArrayList(); 90 | columns.add("c1"); 91 | columns.add("c2"); 92 | columns.add("c3"); 93 | String query = ClickHouseRecordWriter.constructInsertQuery("test", columns); 94 | String expected = "INSERT INTO test (c1,c2,c3) VALUES (?,?,?)"; 95 | Assert.assertEquals(expected, query); 96 | } 97 | 98 | /** 99 | * Test that the writer can correctly write data to ClickHouse 100 | * 101 | * @throws SQLException 102 | * @throws ClassNotFoundException 103 | * @throws SerDeException 104 | * @throws IOException 105 | */ 106 | @Test 107 | public void testWriteSingleRow() throws SQLException, ClassNotFoundException, SerDeException, IOException { 108 | List columnNames = helper.getColumnNames(); 109 | ClickHouseRecordWriter recordWriter = new ClickHouseRecordWriter( 110 | helper, 5, TABLE_NAME, columnNames, helper.getColumnTypes()); 111 | 112 | ClickHouseSerDe serDe = new ClickHouseSerDe(); 113 | Configuration conf = new Configuration(); 114 | Properties tblProps = TestHelper.createPropertiesSource(TABLE_NAME, String.join(",", columnNames)); 115 | SerDeUtils.initializeSerDe(serDe, conf, tblProps, null); 116 | 117 | Object[] row_object1 = new Object[]{ 118 | new TimestampWritable(new Timestamp(1377907200000L)), 119 | new Text("dim1_val"), 120 | new HiveCharWritable(new HiveChar("dim2_v", 6)), 121 | new HiveVarcharWritable(new HiveVarchar("dim3_val", 8)), 122 | new DoubleWritable(10669.3D), 123 | new FloatWritable(10669.45F), 124 | new LongWritable(1113939), 125 | new IntWritable(1112123), 126 | new ShortWritable((short) 12), 127 | new ByteWritable((byte) 0), 128 | }; 129 | ClickHouseWritable writable = TestHelper.serializeObject(serDe, row_object1, COLUMN_HIVE_TYPES); 130 | recordWriter.write(writable); 131 | // force flush 132 | recordWriter.close(true); 133 | Connection conn = helper.getClickHouseConnection(); 134 | Statement stmt = conn.createStatement(); 135 | try { 136 | ResultSet rs = stmt.executeQuery("SELECT " + String.join(",", columnNames) + " FROM " + TABLE_NAME); 137 | int count = 0; 138 | while (rs.next()) { 139 | ++count; 140 | if (count > 1) { 141 | Assert.assertTrue("There can only be one row", count <= 1); 142 | } 143 | Assert.assertEquals(new Timestamp(1377907200000L), rs.getTimestamp(1)); 144 | Assert.assertEquals("dim1_val", rs.getString(2)); 145 | Assert.assertEquals("dim2_v", rs.getString(3)); 146 | Assert.assertEquals("dim3_val", rs.getString(4)); 147 | Assert.assertEquals(10669.3D, rs.getDouble(5)); 148 | Assert.assertEquals(10669.45F, rs.getFloat(6)); 149 | Assert.assertEquals(1113939, rs.getLong(7)); 150 | Assert.assertEquals(1112123, rs.getInt(8)); 151 | Assert.assertEquals((short) 12, rs.getShort(9)); 152 | Assert.assertEquals((byte) 0, rs.getByte(10)); 153 | } 154 | } finally { 155 | conn.close(); 156 | stmt.close(); 157 | } 158 | } 159 | 160 | /** 161 | * Make sure that the RecordWriter will correctly flush the in memory rows as long as the number reaches the 162 | * batch size 163 | * 164 | * @throws SerDeException 165 | * @throws IOException 166 | * @throws SQLException 167 | */ 168 | @Test 169 | public void testWriteAutoBatch() throws SerDeException, IOException, SQLException { 170 | int batchSize = 5; 171 | List columnNames = helper.getColumnNames(); 172 | ClickHouseRecordWriter recordWriter = new ClickHouseRecordWriter( 173 | helper, batchSize, TABLE_NAME, columnNames, helper.getColumnTypes()); 174 | 175 | ClickHouseSerDe serDe = new ClickHouseSerDe(); 176 | Configuration conf = new Configuration(); 177 | Properties tblProps = TestHelper.createPropertiesSource(TABLE_NAME, String.join(",", columnNames)); 178 | SerDeUtils.initializeSerDe(serDe, conf, tblProps, null); 179 | 180 | Object[] row_object1 = new Object[]{ 181 | new TimestampWritable(new Timestamp(1377907200000L)), 182 | new Text("dim1_val"), 183 | new HiveCharWritable(new HiveChar("dim2_v", 6)), 184 | new HiveVarcharWritable(new HiveVarchar("dim3_val", 8)), 185 | new DoubleWritable(10669.3D), 186 | new FloatWritable(10669.45F), 187 | new LongWritable(1113939), 188 | new IntWritable(1112123), 189 | new ShortWritable((short) 12), 190 | new ByteWritable((byte) 0), 191 | }; 192 | ClickHouseWritable writable = TestHelper.serializeObject(serDe, row_object1, COLUMN_HIVE_TYPES); 193 | for (int i = 0; i < batchSize; ++i) { 194 | recordWriter.write(writable); 195 | } 196 | Connection conn = helper.getClickHouseConnection(); 197 | Statement stmt = conn.createStatement(); 198 | ResultSet rs = stmt.executeQuery("SELECT COUNT(*) FROM " + TABLE_NAME); 199 | rs.next(); 200 | Assert.assertEquals(5, rs.getInt(1)); 201 | 202 | } 203 | } -------------------------------------------------------------------------------- /src/test/data/bytedance/net/ck/hive/ClickHouseSerDeTest.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.ck.hive; 2 | 3 | import junit.framework.Assert; 4 | import org.apache.hadoop.conf.Configuration; 5 | import org.apache.hadoop.hive.common.type.HiveChar; 6 | import org.apache.hadoop.hive.common.type.HiveVarchar; 7 | import org.apache.hadoop.hive.serde2.SerDeException; 8 | import org.apache.hadoop.hive.serde2.SerDeUtils; 9 | import org.apache.hadoop.hive.serde2.io.HiveCharWritable; 10 | import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable; 11 | import org.apache.hadoop.hive.serde2.io.TimestampWritable; 12 | import org.apache.hadoop.io.*; 13 | import org.junit.jupiter.api.AfterAll; 14 | import org.junit.jupiter.api.BeforeAll; 15 | import org.junit.jupiter.api.Test; 16 | 17 | import java.sql.Connection; 18 | import java.sql.SQLException; 19 | import java.sql.Statement; 20 | import java.sql.Timestamp; 21 | import java.util.Properties; 22 | 23 | public class ClickHouseSerDeTest { 24 | 25 | private static final String COLUMN_NAMES = "time,c0,c1,c2,c3,c4,c5,c6,c7,c8"; 26 | private static final Object[] ROW_OBJECT = new Object[]{ 27 | new TimestampWritable(new Timestamp(1377907200000L)), 28 | new Text("dim1_val"), 29 | new HiveCharWritable(new HiveChar("dim2_v", 6)), 30 | new HiveVarcharWritable(new HiveVarchar("dim3_val", 8)), 31 | new DoubleWritable(10669.3D), 32 | new FloatWritable(10669.45F), 33 | new LongWritable(1113939), 34 | new IntWritable(1112123), 35 | new ShortWritable((short) 12), 36 | new ByteWritable((byte) 0), 37 | }; 38 | private static final String TABLE_NAME = "testSerDe"; 39 | private static final String CLICKHOUSE_CREATE_TABLE_STMT = "CREATE TABLE IF NOT EXISTS testSerDe\n" + 40 | "(\n" + 41 | " time Datetime,\n" + 42 | " c0 String,\n" + 43 | " c1 String,\n" + 44 | " c2 String,\n" + 45 | " c3 Float64,\n" + 46 | " c4 Float32,\n" + 47 | " c5 Int64,\n" + 48 | " c6 Int32,\n" + 49 | " c7 Int16,\n" + 50 | " c8 Int8\n" + 51 | ")\n" + 52 | "ENGINE = MergeTree\n" + 53 | "PARTITION BY time\n" + 54 | "ORDER BY c0\n" + 55 | "SETTINGS index_granularity = 8192"; 56 | 57 | private static final String COLUMN_HIVE_TYPES = "timestamp,string,char(6),varchar(8),double,float,bigint,int,smallint,tinyint"; 58 | 59 | @BeforeAll 60 | public static void beforeTest() throws SQLException { 61 | ClickHouseHelper helper = ClickHouseHelper.getClickHouseHelper(TestHelper.ckConnStr, TABLE_NAME); 62 | Connection c = helper.getClickHouseConnection(); 63 | try { 64 | Statement stmt = c.createStatement(); 65 | stmt.executeQuery("DROP TABLE IF EXISTS " + TABLE_NAME); 66 | stmt.executeQuery(CLICKHOUSE_CREATE_TABLE_STMT); 67 | } finally { 68 | c.close(); 69 | } 70 | } 71 | 72 | 73 | @AfterAll 74 | public static void afterTest() throws SQLException { 75 | ClickHouseHelper helper = ClickHouseHelper.getClickHouseHelper(TestHelper.ckConnStr, TABLE_NAME); 76 | Connection c = helper.getClickHouseConnection(); 77 | try { 78 | Statement stmt = c.createStatement(); 79 | stmt.executeQuery("DROP TABLE IF EXISTS" + TABLE_NAME); 80 | } finally { 81 | c.close(); 82 | } 83 | 84 | } 85 | 86 | /** 87 | * Test that the data from Hive can be correctly serialized 88 | * @throws SerDeException 89 | */ 90 | @Test 91 | public void testClickHouseObjectSerializer() throws SerDeException { 92 | ClickHouseSerDe serDe = new ClickHouseSerDe(); 93 | Configuration conf = new Configuration(); 94 | Properties tblProps; 95 | tblProps = TestHelper.createPropertiesSource(TABLE_NAME, COLUMN_NAMES); 96 | SerDeUtils.initializeSerDe(serDe, conf, tblProps, null); 97 | ClickHouseWritable writable = TestHelper.serializeObject(serDe, ROW_OBJECT, COLUMN_HIVE_TYPES); 98 | Assert.assertEquals(10, writable.getValue().size()); 99 | } 100 | 101 | 102 | } -------------------------------------------------------------------------------- /src/test/data/bytedance/net/ck/hive/TestHelper.java: -------------------------------------------------------------------------------- 1 | package data.bytedance.net.ck.hive; 2 | 3 | import com.google.common.base.Function; 4 | import com.google.common.collect.Lists; 5 | import org.apache.hadoop.hive.serde2.SerDeException; 6 | import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; 7 | import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; 8 | import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; 9 | import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; 10 | import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; 11 | import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; 12 | 13 | import java.util.ArrayList; 14 | import java.util.List; 15 | import java.util.Properties; 16 | 17 | public class TestHelper { 18 | public static String ckConnStr = "jdbc:clickhouse://localhost:9000"; 19 | 20 | public static ClickHouseWritable serializeObject(ClickHouseSerDe serDe, 21 | Object[] rowObject, String types) throws SerDeException { 22 | List columnNames = serDe.getColumnNames(); 23 | List colTypes = TypeInfoUtils.getTypeInfosFromTypeString(types); 24 | List inspectors = new ArrayList<>(); 25 | inspectors.addAll(Lists.transform(colTypes, 26 | (Function) type -> PrimitiveObjectInspectorFactory 27 | .getPrimitiveWritableObjectInspector(TypeInfoFactory.getPrimitiveTypeInfo(type.getTypeName())))); 28 | ObjectInspector inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); 29 | 30 | ClickHouseWritable writable = (ClickHouseWritable) serDe.serialize(rowObject, inspector); 31 | return writable; 32 | } 33 | 34 | public static Properties createPropertiesSource(String tableName, String columnNames) { 35 | Properties tbl = new Properties(); 36 | tbl.setProperty(Constants.LIST_COLUMNS, columnNames); 37 | tbl.setProperty(Constants.CK_CONN_STRS, TestHelper.ckConnStr); 38 | tbl.setProperty(Constants.CK_TBL_NAME, tableName); 39 | return tbl; 40 | } 41 | } 42 | --------------------------------------------------------------------------------