From a1b83cdc6c2ce0328267c0f42d2ab213a9cd1add Mon Sep 17 00:00:00 2001 From: "awol2005ex@163.com" Date: Mon, 11 Nov 2024 11:24:59 +0800 Subject: [PATCH 01/40] set hive jdbc connection loginTimeout ,fixed the hive connection timeout error --- .../java/com/wgzhao/addax/rdbms/util/DBUtil.java | 13 +++++++------ plugin/reader/hivereader/pom.xml | 4 ++-- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DBUtil.java b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DBUtil.java index 6e9717396..c245a4ff8 100644 --- a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DBUtil.java +++ b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DBUtil.java @@ -34,12 +34,7 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.sql.Connection; -import java.sql.ResultSet; -import java.sql.ResultSetMetaData; -import java.sql.SQLException; -import java.sql.SQLFeatureNotSupportedException; -import java.sql.Statement; +import java.sql.*; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -58,6 +53,7 @@ public final class DBUtil { private static final Logger LOG = LoggerFactory.getLogger(DBUtil.class); private static final int DEFAULT_SOCKET_TIMEOUT_SEC = 20_000; + private static final int DEFAULT_LOGIN_TIMEOUT_SEC = 1000; private static final ThreadLocal rsExecutors = ThreadLocal.withInitial(() -> Executors.newFixedThreadPool(1, new ThreadFactoryBuilder() .setNameFormat("rsExecutors-%d") @@ -245,6 +241,11 @@ private static synchronized Connection connect(DataBaseType dataBaseType, String bds.setDriverClassName("org.apache.hive.jdbc.HiveDriver"); } else { + if("org.apache.hive.jdbc.HiveDriver".equals(bds.getDriverClassName())){ + DriverManager.setLoginTimeout(DEFAULT_LOGIN_TIMEOUT_SEC); + } + + LOG.debug("Connecting to database with driver {}", dataBaseType.getDriverClassName()); bds.setDriverClassName(dataBaseType.getDriverClassName()); } diff --git a/plugin/reader/hivereader/pom.xml b/plugin/reader/hivereader/pom.xml index 76b8e06b4..fb409852f 100644 --- a/plugin/reader/hivereader/pom.xml +++ b/plugin/reader/hivereader/pom.xml @@ -12,8 +12,8 @@ jar - 2.7.5 - 2.1.1 + 3.2.4 + 3.1.3 4.2.0 From 69f806e3627480a99716df3ac4a139918ba03294 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Mon, 25 Nov 2024 10:54:07 +0800 Subject: [PATCH 02/40] rdbmsreader jdbc_url is string not list --- .../com/wgzhao/addax/plugin/reader/rdbmsreader/RdbmsReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/reader/rdbmsreader/src/main/java/com/wgzhao/addax/plugin/reader/rdbmsreader/RdbmsReader.java b/plugin/reader/rdbmsreader/src/main/java/com/wgzhao/addax/plugin/reader/rdbmsreader/RdbmsReader.java index a0e065aa3..3e3d30fd7 100644 --- a/plugin/reader/rdbmsreader/src/main/java/com/wgzhao/addax/plugin/reader/rdbmsreader/RdbmsReader.java +++ b/plugin/reader/rdbmsreader/src/main/java/com/wgzhao/addax/plugin/reader/rdbmsreader/RdbmsReader.java @@ -68,7 +68,7 @@ public void init() String jdbcDriver = this.originalConfig.getString(JDBC_DRIVER, null); if (jdbcDriver == null || StringUtils.isBlank(jdbcDriver)) { // guess jdbc driver name from jdbc url if not set - final String jdbcType = connection.getList(Key.JDBC_URL).get(0).toString().split(":")[1]; + final String jdbcType = connection.getString(Key.JDBC_URL).split(":")[1]; Arrays.stream(DataBaseType.values()).filter( dataBaseType -> dataBaseType.getTypeName().equals(jdbcType)).findFirst().ifPresent(dataBaseType -> DATABASE_TYPE.setDriverClassName(dataBaseType.getDriverClassName())); From 16dcda50c9478dd955a39fe23d9c31169a4d44c3 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Mon, 25 Nov 2024 14:35:27 +0800 Subject: [PATCH 03/40] S3writer: add pathStyleAccessEnabled config for fixed error 400 when upload to minio(set pathStyleAccessEnabled true when upload to minio) --- .../wgzhao/addax/plugin/writer/s3writer/S3Key.java | 2 ++ .../wgzhao/addax/plugin/writer/s3writer/S3Util.java | 11 ++++++++--- pom.xml | 2 +- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java index 25c66e012..110944c95 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java @@ -20,4 +20,6 @@ public class S3Key extends Key public static final String MAX_FILE_SIZE = "maxFileSize"; public static final String DEFAULT_SUFFIX = "defaultSuffix"; + + public static final String PATH_STYLE_ACCESS_ENABLED = "pathStyleAccessEnabled"; } diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java index 6e7f5db88..72dfbe256 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java @@ -18,18 +18,23 @@ public static S3Client initS3Client(Configuration conf) { Region region = Region.of(regionStr); String accessId = conf.getString(S3Key.ACCESS_ID); String accessKey = conf.getString(S3Key.ACCESS_KEY); + String pathStyleAccessEnabled =conf.getString(S3Key.PATH_STYLE_ACCESS_ENABLED,""); - return initS3Client(conf.getString(S3Key.ENDPOINT), region, accessId, accessKey); + return initS3Client(conf.getString(S3Key.ENDPOINT), region, accessId, accessKey ,pathStyleAccessEnabled); } - public static S3Client initS3Client(String endpoint, Region region, String accessId, String accessKey) { + public static S3Client initS3Client(String endpoint, Region region, String accessId, String accessKey ,String pathStyleAccessEnabled) { if (null == region) { region = Region.of("ap-northeast-1"); } try { AwsBasicCredentials awsCreds = AwsBasicCredentials.create(accessId, accessKey); - return S3Client.builder() + return S3Client.builder().serviceConfiguration(e -> { + if("true".equals(pathStyleAccessEnabled)) { + e.pathStyleAccessEnabled(true); + } + }) .credentialsProvider(StaticCredentialsProvider.create(awsCreds)) .region(region) .endpointOverride(URI.create(endpoint)) diff --git a/pom.xml b/pom.xml index 8f02122af..98d64beaf 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ 0.25 - 2.20.5 + 2.29.20 1.11.3 1.9.4 1.10 From 796703ce1eecb1a83b4c43888cb0d49a0fbfb510 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Mon, 25 Nov 2024 17:03:42 +0800 Subject: [PATCH 04/40] [bugfix][plugin][s3writer]S3writer:don't submit upload when no record to write --- .../plugin/writer/s3writer/S3Writer.java | 32 +++++++++++-------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java index c707228c6..6de34697d 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java @@ -307,22 +307,26 @@ public void startWrite(RecordReceiver lineReceiver) completedParts.add(completedPart); outputStream.reset(); } - // Finally, call completeMultipartUpload operation to tell S3 to merge all uploaded - // parts and finish the multipart operation. - CompletedMultipartUpload completedMultipartUpload = CompletedMultipartUpload.builder() - .parts(completedParts) - .build(); + if(!completedParts.isEmpty()) { + // Finally, call completeMultipartUpload operation to tell S3 to merge all uploaded + // parts and finish the multipart operation. + CompletedMultipartUpload completedMultipartUpload = CompletedMultipartUpload.builder() + .parts(completedParts) + .build(); - CompleteMultipartUploadRequest completeMultipartUploadRequest = - CompleteMultipartUploadRequest.builder() - .bucket(bucket) - .key(object) - .uploadId(uploadId) - .multipartUpload(completedMultipartUpload) - .build(); + CompleteMultipartUploadRequest completeMultipartUploadRequest = + CompleteMultipartUploadRequest.builder() + .bucket(bucket) + .key(object) + .uploadId(uploadId) + .multipartUpload(completedMultipartUpload) + .build(); - s3Client.completeMultipartUpload(completeMultipartUploadRequest); - LOG.info("end do write"); + s3Client.completeMultipartUpload(completeMultipartUploadRequest); + LOG.info("end do write"); + } else { + LOG.info("no content do write"); + } } private String record2String(Record record) From 663d9aeb95b9eac022de5fd665fe261d5348709a Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Tue, 26 Nov 2024 14:27:16 +0800 Subject: [PATCH 05/40] S3 textwriter --- .../addax/plugin/writer/s3writer/S3Key.java | 2 + .../addax/plugin/writer/s3writer/S3Util.java | 5 +- .../plugin/writer/s3writer/S3Writer.java | 120 ++------- .../s3writer/formatwriter/IFormatWriter.java | 10 + .../s3writer/formatwriter/TextWriter.java | 229 ++++++++++++++++++ 5 files changed, 261 insertions(+), 105 deletions(-) create mode 100644 plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java create mode 100644 plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/TextWriter.java diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java index 110944c95..e3f9ed0ef 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java @@ -22,4 +22,6 @@ public class S3Key extends Key public static final String DEFAULT_SUFFIX = "defaultSuffix"; public static final String PATH_STYLE_ACCESS_ENABLED = "pathStyleAccessEnabled"; + + public static final String FILE_FORMAT = "fileFormat"; } diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java index 03aeb8785..5a8194c8e 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java @@ -19,12 +19,13 @@ public static S3Client initS3Client(Configuration conf) { String accessId = conf.getString(S3Key.ACCESS_ID); String accessKey = conf.getString(S3Key.ACCESS_KEY); String pathStyleAccessEnabled =conf.getString(S3Key.PATH_STYLE_ACCESS_ENABLED,""); + String fileFormat = conf.getString(S3Key.FILE_FORMAT,"text"); - return initS3Client(conf.getString(S3Key.ENDPOINT), region, accessId, accessKey ,pathStyleAccessEnabled); + return initS3Client(conf.getString(S3Key.ENDPOINT), region, accessId, accessKey ,pathStyleAccessEnabled,fileFormat); } - public static S3Client initS3Client(String endpoint, Region region, String accessId, String accessKey ,String pathStyleAccessEnabled) { + public static S3Client initS3Client(String endpoint, Region region, String accessId, String accessKey ,String pathStyleAccessEnabled,String fileFormat) { if (null == region) { region = Region.of("ap-northeast-1"); } diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java index 6de34697d..682bab083 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java @@ -7,6 +7,7 @@ import com.wgzhao.addax.common.plugin.RecordReceiver; import com.wgzhao.addax.common.spi.Writer; import com.wgzhao.addax.common.util.Configuration; +import com.wgzhao.addax.plugin.writer.s3writer.formatwriter.TextWriter; import com.wgzhao.addax.storage.writer.StorageWriterUtil; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -221,6 +222,7 @@ public static class Task private String dateFormat; private List header; private int maxFileSize;// MB + private String fileFormat; @Override public void init() @@ -238,118 +240,30 @@ public void init() // unit MB int DEFAULT_MAX_FILE_SIZE = 10 * 10000; this.maxFileSize = writerSliceConfig.getInt(S3Key.MAX_FILE_SIZE, DEFAULT_MAX_FILE_SIZE); + + this.fileFormat = writerSliceConfig.getString(S3Key.FILE_FORMAT, "text"); } @Override public void startWrite(RecordReceiver lineReceiver) { - // 设置每块字符串长度 - final int partSize = 1024 * 1024 * 10; - long numberCalc = (this.maxFileSize * 1024 * 1024L) / partSize; - final long maxPartNumber = numberCalc >= 1 ? numberCalc : 1; - //warn: may be StringBuffer->StringBuilder - Record record; - - LOG.info("Begin do write, each object's max file size is {}MB...", maxPartNumber * 10); - // First create a multipart upload and get the upload id - CreateMultipartUploadRequest createMultipartUploadRequest = CreateMultipartUploadRequest.builder() - .bucket(bucket) - .key(object) - .build(); - CreateMultipartUploadResponse response = s3Client.createMultipartUpload(createMultipartUploadRequest); - String uploadId = response.uploadId(); - int currPart = 1; - List completedParts = new ArrayList<>(); - - UploadPartRequest uploadPartRequest; - uploadPartRequest = UploadPartRequest.builder() - .bucket(bucket) - .key(object) - .uploadId(uploadId) - .partNumber(currPart).build(); - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - Charset charset = Charset.forName(encoding); - boolean needInit = true; - while ((record = lineReceiver.getFromReader()) != null) { - try { - if (needInit && !header.isEmpty()) { - // write header - outputStream.write(String.join(String.valueOf(fieldDelimiter), header).getBytes(charset)); - outputStream.write("\n".getBytes(charset)); - needInit = false; - } - outputStream.write(record2String(record).getBytes(charset)); - outputStream.write("\n".getBytes(charset)); - - if (outputStream.size() > partSize) { - String etag = s3Client.uploadPart(uploadPartRequest, RequestBody.fromBytes(outputStream.toByteArray())).eTag(); - CompletedPart completedPart = CompletedPart.builder().partNumber(currPart).eTag(etag).build(); - completedParts.add(completedPart); - currPart += 1; - outputStream.reset(); - - uploadPartRequest = UploadPartRequest.builder() - .bucket(bucket) - .key(object) - .uploadId(uploadId) - .partNumber(currPart).build(); - needInit = true; - } - } - catch (IOException e) { - throw AddaxException.asAddaxException(IO_ERROR, e.getMessage()); - } + if("text".equals(this.fileFormat)){ + TextWriter textWriter = new TextWriter() + .setBucket(this.bucket) + .setDateFormat(this.dateFormat) + .setEncoding(this.encoding) + .setHeader(this.header) + .setNullFormat(this.nullFormat) + .setObject(this.object) + .setS3Client(this.s3Client) + .setFieldDelimiter(this.fieldDelimiter) + .setMaxFileSize(this.maxFileSize ); + textWriter.write(lineReceiver, this.getPluginJobConf(), this.getTaskPluginCollector()); } - // remain bytes - if (outputStream.size() > 0) { - String etag = s3Client.uploadPart(uploadPartRequest, RequestBody.fromBytes(outputStream.toByteArray())).eTag(); - CompletedPart completedPart = CompletedPart.builder().partNumber(currPart).eTag(etag).build(); - completedParts.add(completedPart); - outputStream.reset(); - } - if(!completedParts.isEmpty()) { - // Finally, call completeMultipartUpload operation to tell S3 to merge all uploaded - // parts and finish the multipart operation. - CompletedMultipartUpload completedMultipartUpload = CompletedMultipartUpload.builder() - .parts(completedParts) - .build(); - - CompleteMultipartUploadRequest completeMultipartUploadRequest = - CompleteMultipartUploadRequest.builder() - .bucket(bucket) - .key(object) - .uploadId(uploadId) - .multipartUpload(completedMultipartUpload) - .build(); - s3Client.completeMultipartUpload(completeMultipartUploadRequest); - LOG.info("end do write"); - } else { - LOG.info("no content do write"); - } } - private String record2String(Record record) - { - StringJoiner sj = new StringJoiner(this.fieldDelimiter + ""); - int columnNum = record.getColumnNumber(); - for (int i = 0; i < columnNum; i++) { - Column column = record.getColumn(i); - if (column == null || column.asString() == null) { - sj.add(this.nullFormat); - } - assert column != null; - Column.Type type = column.getType(); - if (type == Column.Type.DATE) { - SimpleDateFormat sdf = new SimpleDateFormat(this.dateFormat); - sj.add(sdf.format(column.asDate())); - } - else { - sj.add(column.asString()); - } - } - return sj.toString(); - } + @Override public void destroy() diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java new file mode 100644 index 000000000..1a51c9437 --- /dev/null +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java @@ -0,0 +1,10 @@ +package com.wgzhao.addax.plugin.writer.s3writer.formatwriter; + +import com.wgzhao.addax.common.plugin.RecordReceiver; +import com.wgzhao.addax.common.plugin.TaskPluginCollector; +import com.wgzhao.addax.common.util.Configuration; + +public interface IFormatWriter { + void write(RecordReceiver lineReceiver, Configuration config, + TaskPluginCollector taskPluginCollector); +} diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/TextWriter.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/TextWriter.java new file mode 100644 index 000000000..be189e95c --- /dev/null +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/TextWriter.java @@ -0,0 +1,229 @@ +package com.wgzhao.addax.plugin.writer.s3writer.formatwriter; + +import com.wgzhao.addax.common.element.Column; +import com.wgzhao.addax.common.element.Record; +import com.wgzhao.addax.common.exception.AddaxException; +import com.wgzhao.addax.common.plugin.RecordReceiver; +import com.wgzhao.addax.common.plugin.TaskPluginCollector; +import com.wgzhao.addax.common.util.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.*; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.charset.Charset; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.List; +import java.util.StringJoiner; + +import static com.wgzhao.addax.common.spi.ErrorCode.IO_ERROR; + +public class TextWriter implements IFormatWriter { + private static final Logger LOG = LoggerFactory.getLogger(TextWriter.class); + private char fieldDelimiter; + private String nullFormat; + private String dateFormat; + private String encoding; + private String bucket; + private String object; + private int maxFileSize; + private List header; + private S3Client s3Client; + + public TextWriter() { + } + + public char getFieldDelimiter() { + return fieldDelimiter; + } + + public TextWriter setFieldDelimiter(char fieldDelimiter) { + this.fieldDelimiter = fieldDelimiter; + return this; + } + + public String getNullFormat() { + return nullFormat; + } + + public TextWriter setNullFormat(String nullFormat) { + this.nullFormat = nullFormat; + return this; + } + + public String getDateFormat() { + return dateFormat; + } + + public TextWriter setDateFormat(String dateFormat) { + this.dateFormat = dateFormat; + return this; + } + + public String getEncoding() { + return encoding; + } + + public TextWriter setEncoding(String encoding) { + this.encoding = encoding; + return this; + } + + public String getBucket() { + return bucket; + } + + public TextWriter setBucket(String bucket) { + this.bucket = bucket; + return this; + } + + public String getObject() { + return object; + } + + public TextWriter setObject(String object) { + this.object = object; + return this; + } + + public int getMaxFileSize() { + return maxFileSize; + } + + public TextWriter setMaxFileSize(int maxFileSize) { + this.maxFileSize = maxFileSize; + return this; + } + + public List getHeader() { + return header; + } + + public TextWriter setHeader(List header) { + this.header = header; + return this; + } + + public S3Client getS3Client() { + return s3Client; + } + + public TextWriter setS3Client(S3Client s3Client) { + this.s3Client = s3Client; + return this; + } + + @Override + public void write(RecordReceiver lineReceiver, Configuration config, TaskPluginCollector taskPluginCollector) { + + // 设置每块字符串长度 + final int partSize = 1024 * 1024 * 10; + long numberCalc = (this.maxFileSize * 1024 * 1024L) / partSize; + final long maxPartNumber = numberCalc >= 1 ? numberCalc : 1; + //warn: may be StringBuffer->StringBuilder + Record record; + + LOG.info("Begin do write, each object's max file size is {}MB...", maxPartNumber * 10); + // First create a multipart upload and get the upload id + CreateMultipartUploadRequest createMultipartUploadRequest = CreateMultipartUploadRequest.builder() + .bucket(bucket) + .key(object) + .build(); + CreateMultipartUploadResponse response = s3Client.createMultipartUpload(createMultipartUploadRequest); + String uploadId = response.uploadId(); + int currPart = 1; + List completedParts = new ArrayList<>(); + + UploadPartRequest uploadPartRequest; + uploadPartRequest = UploadPartRequest.builder() + .bucket(bucket) + .key(object) + .uploadId(uploadId) + .partNumber(currPart).build(); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + Charset charset = Charset.forName(encoding); + boolean needInit = true; + while ((record = lineReceiver.getFromReader()) != null) { + try { + if (needInit && !header.isEmpty()) { + // write header + outputStream.write(String.join(String.valueOf(fieldDelimiter), header).getBytes(charset)); + outputStream.write("\n".getBytes(charset)); + needInit = false; + } + outputStream.write(record2String(record).getBytes(charset)); + outputStream.write("\n".getBytes(charset)); + + if (outputStream.size() > partSize) { + String etag = s3Client.uploadPart(uploadPartRequest, RequestBody.fromBytes(outputStream.toByteArray())).eTag(); + CompletedPart completedPart = CompletedPart.builder().partNumber(currPart).eTag(etag).build(); + completedParts.add(completedPart); + currPart += 1; + outputStream.reset(); + + uploadPartRequest = UploadPartRequest.builder() + .bucket(bucket) + .key(object) + .uploadId(uploadId) + .partNumber(currPart).build(); + needInit = true; + } + } catch (IOException e) { + throw AddaxException.asAddaxException(IO_ERROR, e.getMessage()); + } + } + // remain bytes + if (outputStream.size() > 0) { + String etag = s3Client.uploadPart(uploadPartRequest, RequestBody.fromBytes(outputStream.toByteArray())).eTag(); + CompletedPart completedPart = CompletedPart.builder().partNumber(currPart).eTag(etag).build(); + completedParts.add(completedPart); + outputStream.reset(); + } + if (!completedParts.isEmpty()) { + // Finally, call completeMultipartUpload operation to tell S3 to merge all uploaded + // parts and finish the multipart operation. + CompletedMultipartUpload completedMultipartUpload = CompletedMultipartUpload.builder() + .parts(completedParts) + .build(); + + CompleteMultipartUploadRequest completeMultipartUploadRequest = + CompleteMultipartUploadRequest.builder() + .bucket(bucket) + .key(object) + .uploadId(uploadId) + .multipartUpload(completedMultipartUpload) + .build(); + + s3Client.completeMultipartUpload(completeMultipartUploadRequest); + LOG.info("end do write"); + } else { + LOG.info("no content do write"); + } + } + + + private String record2String(Record record) { + StringJoiner sj = new StringJoiner(this.fieldDelimiter + ""); + int columnNum = record.getColumnNumber(); + for (int i = 0; i < columnNum; i++) { + Column column = record.getColumn(i); + if (column == null || column.asString() == null) { + sj.add(this.nullFormat); + } + assert column != null; + Column.Type type = column.getType(); + if (type == Column.Type.DATE) { + SimpleDateFormat sdf = new SimpleDateFormat(this.dateFormat); + sj.add(sdf.format(column.asDate())); + } else { + sj.add(column.asString()); + } + } + return sj.toString(); + } +} From a12a4f11c7e51e0ae66cde839bb03aaf2261f923 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Tue, 26 Nov 2024 17:39:12 +0800 Subject: [PATCH 06/40] S3 orcwriter --- plugin/writer/s3writer/pom.xml | 295 +++++++++++++++++ .../addax/plugin/writer/s3writer/S3Key.java | 7 +- .../addax/plugin/writer/s3writer/S3Util.java | 6 +- .../plugin/writer/s3writer/S3Writer.java | 111 +++---- .../s3writer/formatwriter/IFormatWriter.java | 1 + .../s3writer/formatwriter/OrcWriter.java | 308 ++++++++++++++++++ .../formatwriter/SupportHiveDataType.java | 40 +++ .../s3writer/formatwriter/TextWriter.java | 5 + 8 files changed, 700 insertions(+), 73 deletions(-) create mode 100644 plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/OrcWriter.java create mode 100644 plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/SupportHiveDataType.java diff --git a/plugin/writer/s3writer/pom.xml b/plugin/writer/s3writer/pom.xml index e598cb8d2..5590bef90 100644 --- a/plugin/writer/s3writer/pom.xml +++ b/plugin/writer/s3writer/pom.xml @@ -57,6 +57,301 @@ + + com.google.protobuf + protobuf-java + ${protobuf.version} + + + + org.apache.avro + avro + ${avro.version} + + + com.fasterxml.jackson.core + jackson-databind + + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + + org.apache.hadoop + hadoop-aws + ${hadoop.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + + + com.fasterxml.jackson.core + jackson-databind + + + commons-codec + commons-codec + + + io.netty + netty + + + org.eclipse.jetty + jetty-util + + + + + + org.apache.hive + hive-common + ${hive.version} + + + slf4j-api + org.slf4j + + + com.google.guava + guava + + + com.fasterxml.jackson.core + jackson-databind + + + commons-net + commons-net + + + log4j + log4j + + + org.apache.ant + ant + + + org.apache.hadoop + hadoop-yarn-server-common + + + org.apache.hadoop + hadoop-yarn-server-resourcemanager + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-io + + + org.eclipse.jetty + jetty-http + + + org.eclipse.jetty + jetty-client + + + org.apache.thrift + libthrift + + + org.apache.zookeeper + zookeeper + + + io.netty + netty + + + + + + org.apache.hive + hive-serde + 3.1.3 + compile + + + + org.apache.hive + hive-storage-api + 2.8.1 + + + + org.apache.orc + orc-core + ${orc.version} + + + + org.apache.parquet + parquet-avro + ${parquet.version} + + + commons-codec + commons-codec + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + + + + org.apache.parquet + parquet-hadoop-bundle + ${parquet.version} + + + + org.xerial.snappy + snappy-java + ${snappy.version} + + + + com.fasterxml.woodstox + woodstox-core + ${woodstox.version} + + diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java index e3f9ed0ef..5fbca34df 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java @@ -23,5 +23,10 @@ public class S3Key extends Key public static final String PATH_STYLE_ACCESS_ENABLED = "pathStyleAccessEnabled"; - public static final String FILE_FORMAT = "fileFormat"; + public static final String FILE_TYPE = "fileType"; + + + public static final String SSLENABLED = "sslEnabled"; + + } diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java index 5a8194c8e..a7e4a97fe 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java @@ -19,13 +19,13 @@ public static S3Client initS3Client(Configuration conf) { String accessId = conf.getString(S3Key.ACCESS_ID); String accessKey = conf.getString(S3Key.ACCESS_KEY); String pathStyleAccessEnabled =conf.getString(S3Key.PATH_STYLE_ACCESS_ENABLED,""); - String fileFormat = conf.getString(S3Key.FILE_FORMAT,"text"); - return initS3Client(conf.getString(S3Key.ENDPOINT), region, accessId, accessKey ,pathStyleAccessEnabled,fileFormat); + + return initS3Client(conf.getString(S3Key.ENDPOINT), region, accessId, accessKey ,pathStyleAccessEnabled); } - public static S3Client initS3Client(String endpoint, Region region, String accessId, String accessKey ,String pathStyleAccessEnabled,String fileFormat) { + public static S3Client initS3Client(String endpoint, Region region, String accessId, String accessKey ,String pathStyleAccessEnabled) { if (null == region) { region = Region.of("ap-northeast-1"); } diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java index 682bab083..c67f76758 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java @@ -1,78 +1,47 @@ package com.wgzhao.addax.plugin.writer.s3writer; import com.wgzhao.addax.common.base.Constant; -import com.wgzhao.addax.common.element.Column; -import com.wgzhao.addax.common.element.Record; import com.wgzhao.addax.common.exception.AddaxException; import com.wgzhao.addax.common.plugin.RecordReceiver; import com.wgzhao.addax.common.spi.Writer; import com.wgzhao.addax.common.util.Configuration; +import com.wgzhao.addax.plugin.writer.s3writer.formatwriter.OrcWriter; import com.wgzhao.addax.plugin.writer.s3writer.formatwriter.TextWriter; import com.wgzhao.addax.storage.writer.StorageWriterUtil; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; -import software.amazon.awssdk.services.s3.model.CompletedMultipartUpload; -import software.amazon.awssdk.services.s3.model.CompletedPart; -import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest; -import software.amazon.awssdk.services.s3.model.CreateMultipartUploadResponse; -import software.amazon.awssdk.services.s3.model.Delete; -import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; -import software.amazon.awssdk.services.s3.model.ObjectIdentifier; -import software.amazon.awssdk.services.s3.model.S3Exception; -import software.amazon.awssdk.services.s3.model.S3Object; -import software.amazon.awssdk.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.services.s3.model.*; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.nio.charset.Charset; -import java.text.SimpleDateFormat; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.StringJoiner; -import java.util.UUID; +import java.util.*; -import static com.wgzhao.addax.common.spi.ErrorCode.ILLEGAL_VALUE; -import static com.wgzhao.addax.common.spi.ErrorCode.IO_ERROR; -import static com.wgzhao.addax.common.spi.ErrorCode.REQUIRED_VALUE; -import static com.wgzhao.addax.common.spi.ErrorCode.RUNTIME_ERROR; +import static com.wgzhao.addax.common.spi.ErrorCode.*; public class S3Writer - extends Writer -{ + extends Writer { public static class Job - extends Writer.Job - { + extends Writer.Job { private static final Logger LOG = LoggerFactory.getLogger(Job.class); private Configuration writerSliceConfig = null; private S3Client s3Client = null; @Override - public void init() - { + public void init() { this.writerSliceConfig = this.getPluginJobConf(); this.validateParameter(); this.s3Client = S3Util.initS3Client(this.writerSliceConfig); } @Override - public void destroy() - { + public void destroy() { if (this.s3Client != null) { this.s3Client.close(); } } - private void validateParameter() - { + private void validateParameter() { this.writerSliceConfig.getNecessaryValue(S3Key.REGION, REQUIRED_VALUE); this.writerSliceConfig.getNecessaryValue(S3Key.ACCESS_ID, REQUIRED_VALUE); this.writerSliceConfig.getNecessaryValue(S3Key.ACCESS_KEY, REQUIRED_VALUE); @@ -83,8 +52,7 @@ private void validateParameter() } @Override - public void prepare() - { + public void prepare() { LOG.info("begin do prepare..."); String bucket = this.writerSliceConfig.getString(S3Key.BUCKET); String object = this.writerSliceConfig.getString(S3Key.OBJECT); @@ -94,8 +62,7 @@ public void prepare() if ("truncate".equals(writeMode)) { LOG.info("It will cleanup all objects which starts with [{}] in [{}]", object, bucket); deleteBucketObjects(bucket, object); - } - else if ("nonConflict".equals(writeMode)) { + } else if ("nonConflict".equals(writeMode)) { LOG.info("Begin to check for existing objects that starts with [{}] in bucket [{}]", object, bucket); List objs = listObjects(bucket, object); if (!objs.isEmpty()) { @@ -106,8 +73,7 @@ else if ("nonConflict".equals(writeMode)) { } @Override - public List split(int mandatoryNumber) - { + public List split(int mandatoryNumber) { LOG.info("begin do split..."); List writerSplitConfigs = new ArrayList<>(); String object = this.writerSliceConfig.getString(S3Key.OBJECT); @@ -148,12 +114,11 @@ public List split(int mandatoryNumber) /** * find all objects which starts with objectName and return * - * @param bucket the S3 bucket name + * @param bucket the S3 bucket name * @param objectName the object prefix will be found * @return {@link List} */ - private List listObjects(String bucket, String objectName) - { + private List listObjects(String bucket, String objectName) { String suffix = null; if (objectName.contains(".")) { suffix = "." + objectName.split("\\.", -1)[1]; @@ -172,8 +137,7 @@ private List listObjects(String bucket, String objectName) for (S3Object obj : objects) { if (suffix == null) { result.add(obj); - } - else if (obj.key().endsWith(suffix)) { + } else if (obj.key().endsWith(suffix)) { result.add(obj); } } @@ -183,11 +147,10 @@ else if (obj.key().endsWith(suffix)) { /** * delete all objects which starts with objectName in bucket * - * @param bucket the S3 bucket name + * @param bucket the S3 bucket name * @param objectName the object prefix will be deleted */ - private void deleteBucketObjects(String bucket, String objectName) - { + private void deleteBucketObjects(String bucket, String objectName) { List objects = listObjects(bucket, objectName); ArrayList toDelete = new ArrayList<>(); if (!objects.isEmpty()) { @@ -200,8 +163,7 @@ private void deleteBucketObjects(String bucket, String objectName) .delete(Delete.builder().objects(toDelete).build()) .build(); s3Client.deleteObjects(dor); - } - catch (S3Exception e) { + } catch (S3Exception e) { throw AddaxException.asAddaxException(RUNTIME_ERROR, e.getMessage()); } } @@ -209,8 +171,7 @@ private void deleteBucketObjects(String bucket, String objectName) } public static class Task - extends Writer.Task - { + extends Writer.Task { private static final Logger LOG = LoggerFactory.getLogger(Task.class); private S3Client s3Client; @@ -222,11 +183,11 @@ public static class Task private String dateFormat; private List header; private int maxFileSize;// MB - private String fileFormat; + private String fileType; + private String sslEnabled; @Override - public void init() - { + public void init() { Configuration writerSliceConfig = this.getPluginJobConf(); this.s3Client = S3Util.initS3Client(writerSliceConfig); this.bucket = writerSliceConfig.getString(S3Key.BUCKET); @@ -241,13 +202,13 @@ public void init() int DEFAULT_MAX_FILE_SIZE = 10 * 10000; this.maxFileSize = writerSliceConfig.getInt(S3Key.MAX_FILE_SIZE, DEFAULT_MAX_FILE_SIZE); - this.fileFormat = writerSliceConfig.getString(S3Key.FILE_FORMAT, "text"); + this.fileType = writerSliceConfig.getString(S3Key.FILE_TYPE, "text"); + this.sslEnabled = writerSliceConfig.getString(S3Key.SSLENABLED, "true"); } @Override - public void startWrite(RecordReceiver lineReceiver) - { - if("text".equals(this.fileFormat)){ + public void startWrite(RecordReceiver lineReceiver) { + if ("text".equals(this.fileType)) { TextWriter textWriter = new TextWriter() .setBucket(this.bucket) .setDateFormat(this.dateFormat) @@ -257,17 +218,29 @@ public void startWrite(RecordReceiver lineReceiver) .setObject(this.object) .setS3Client(this.s3Client) .setFieldDelimiter(this.fieldDelimiter) - .setMaxFileSize(this.maxFileSize ); + .setMaxFileSize(this.maxFileSize); textWriter.write(lineReceiver, this.getPluginJobConf(), this.getTaskPluginCollector()); } + if ("orc".equals(this.fileType)) { + OrcWriter orcWriter = new OrcWriter() + .setBucket(this.bucket) + .setDateFormat(this.dateFormat) + .setEncoding(this.encoding) + .setHeader(this.header) + .setNullFormat(this.nullFormat) + .setObject(this.object) + .setS3Client(this.s3Client) + .setFieldDelimiter(this.fieldDelimiter) + .setSslEnabled(this.sslEnabled); + orcWriter.init(this.getPluginJobConf()); + orcWriter.write(lineReceiver, this.getPluginJobConf(), this.getTaskPluginCollector()); + } } - @Override - public void destroy() - { + public void destroy() { if (this.s3Client != null) { this.s3Client.close(); } diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java index 1a51c9437..df2e9a806 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java @@ -5,6 +5,7 @@ import com.wgzhao.addax.common.util.Configuration; public interface IFormatWriter { + void init(Configuration config); void write(RecordReceiver lineReceiver, Configuration config, TaskPluginCollector taskPluginCollector); } diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/OrcWriter.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/OrcWriter.java new file mode 100644 index 000000000..093e58ae8 --- /dev/null +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/OrcWriter.java @@ -0,0 +1,308 @@ +package com.wgzhao.addax.plugin.writer.s3writer.formatwriter; + +import com.wgzhao.addax.common.base.Constant; +import com.wgzhao.addax.common.base.Key; +import com.wgzhao.addax.common.element.Column; +import com.wgzhao.addax.common.element.DateColumn; +import com.wgzhao.addax.common.element.Record; +import com.wgzhao.addax.common.exception.AddaxException; +import com.wgzhao.addax.common.plugin.RecordReceiver; +import com.wgzhao.addax.common.plugin.TaskPluginCollector; +import com.wgzhao.addax.common.util.Configuration; +import com.wgzhao.addax.plugin.writer.s3writer.S3Key; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.exec.vector.*; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.orc.CompressionKind; +import org.apache.orc.OrcFile; +import org.apache.orc.TypeDescription; +import org.apache.orc.Writer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.Delete; +import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.text.SimpleDateFormat; +import java.time.LocalDate; +import java.util.Base64; +import java.util.List; +import java.util.StringJoiner; + +import static com.wgzhao.addax.common.spi.ErrorCode.*; + +public class OrcWriter + implements IFormatWriter +{ + private final Logger logger = LoggerFactory.getLogger(OrcWriter.class.getName()); + private char fieldDelimiter; + private String nullFormat; + private String dateFormat; + private String encoding; + private String bucket; + private String object; + private List header; + private String sslEnabled; + private S3Client s3Client; + + private String fileName; + + private org.apache.hadoop.conf.Configuration hadoopConf = null; + + public char getFieldDelimiter() { + return fieldDelimiter; + } + + public OrcWriter setFieldDelimiter(char fieldDelimiter) { + this.fieldDelimiter = fieldDelimiter; + return this; + } + + public String getNullFormat() { + return nullFormat; + } + + public OrcWriter setNullFormat(String nullFormat) { + this.nullFormat = nullFormat; + return this; + } + + + + public String getSslEnabled() { + return sslEnabled; + } + + public OrcWriter setSslEnabled(String sslEnabled) { + this.sslEnabled = sslEnabled; + return this; + } + + public String getDateFormat() { + return dateFormat; + } + + public OrcWriter setDateFormat(String dateFormat) { + this.dateFormat = dateFormat; + return this; + } + + public String getEncoding() { + return encoding; + } + + public OrcWriter setEncoding(String encoding) { + this.encoding = encoding; + return this; + } + + public String getBucket() { + return bucket; + } + + public OrcWriter setBucket(String bucket) { + this.bucket = bucket; + return this; + } + + public String getObject() { + return object; + } + + public OrcWriter setObject(String object) { + this.object = object; + return this; + } + + + public List getHeader() { + return header; + } + + public OrcWriter setHeader(List header) { + this.header = header; + return this; + } + public S3Client getS3Client() { + return s3Client; + } + + public OrcWriter setS3Client(S3Client s3Client) { + this.s3Client = s3Client; + return this; + } + + + public OrcWriter() + { + } + + /** + * write an orc record + * + * @param batch {@link VectorizedRowBatch} + * @param row row number + * @param record {@link Record} + * @param columns table columns, {@link List} + * @param taskPluginCollector {@link TaskPluginCollector} + */ + private void setRow(VectorizedRowBatch batch, int row, Record record, List columns, + TaskPluginCollector taskPluginCollector) + { + for (int i = 0; i < columns.size(); i++) { + Configuration eachColumnConf = columns.get(i); + String type = eachColumnConf.getString(Key.TYPE).trim().toUpperCase(); + SupportHiveDataType columnType; + ColumnVector col = batch.cols[i]; + if (type.startsWith("DECIMAL")) { + columnType = SupportHiveDataType.DECIMAL; + } + else { + columnType = SupportHiveDataType.valueOf(type); + } + if (record.getColumn(i) == null || record.getColumn(i).getRawData() == null) { + col.isNull[row] = true; + col.noNulls = false; + continue; + } + + try { + switch (columnType) { + case TINYINT: + case SMALLINT: + case INT: + case BIGINT: + case BOOLEAN: + ((LongColumnVector) col).vector[row] = record.getColumn(i).asLong(); + break; + case DATE: + ((LongColumnVector) col).vector[row] = LocalDate.parse(record.getColumn(i).asString()).toEpochDay(); + break; + case FLOAT: + case DOUBLE: + ((DoubleColumnVector) col).vector[row] = record.getColumn(i).asDouble(); + break; + case DECIMAL: + HiveDecimalWritable hdw = new HiveDecimalWritable(); + hdw.set(HiveDecimal.create(record.getColumn(i).asBigDecimal()) + .setScale(eachColumnConf.getInt(Key.SCALE), HiveDecimal.ROUND_HALF_UP)); + ((DecimalColumnVector) col).set(row, hdw); + break; + case TIMESTAMP: + ((TimestampColumnVector) col).set(row, record.getColumn(i).asTimestamp()); + break; + case STRING: + case VARCHAR: + case CHAR: + byte[] buffer; + Column column = record.getColumn(i); + Column.Type colType = column.getType(); + if (colType == Column.Type.BYTES) { + //convert bytes to base64 string + buffer = Base64.getEncoder().encode((byte[]) column.getRawData()); + } + else if (colType == Column.Type.DATE) { + if (((DateColumn) column).getSubType() == DateColumn.DateType.TIME) { + buffer = column.asString().getBytes(StandardCharsets.UTF_8); + } + else { + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd"); + buffer = sdf.format(record.getColumn(i).asDate()).getBytes(StandardCharsets.UTF_8); + } + } + else { + buffer = record.getColumn(i).getRawData().toString().getBytes(StandardCharsets.UTF_8); + } + ((BytesColumnVector) col).setRef(row, buffer, 0, buffer.length); + break; + case BINARY: + byte[] content = (byte[]) record.getColumn(i).getRawData(); + ((BytesColumnVector) col).setRef(row, content, 0, content.length); + break; + default: + throw AddaxException + .asAddaxException( + NOT_SUPPORT_TYPE, + String.format("The columns configuration is incorrect. the field type is unsupported yet. Field name: [%s], Field type name:[%s].", + eachColumnConf.getString(Key.NAME), + eachColumnConf.getString(Key.TYPE))); + } + } + catch (Exception e) { + taskPluginCollector.collectDirtyRecord(record, e.getMessage()); + throw AddaxException.asAddaxException(RUNTIME_ERROR, + String.format("Failed to set ORC row, source field type: %s, destination field original type: %s, " + + "destination field hive type: %s, field name: %s, source field value: %s, root cause:%n%s", + record.getColumn(i).getType(), columnType, eachColumnConf.getString(Key.TYPE), + eachColumnConf.getString(Key.NAME), + record.getColumn(i).getRawData(), e)); + } + } + } + + @Override + public void init(Configuration config) { + + this.fileName ="s3a://"+this.bucket+"/"+this.object; + + hadoopConf = new org.apache.hadoop.conf.Configuration(); + hadoopConf.set("fs.s3a.access.key",config.getString(S3Key.ACCESS_ID)); + hadoopConf.set("fs.s3a.secret.key",config.getString(S3Key.ACCESS_KEY)); + hadoopConf.set("fs.s3a.endpoint",config.getString(S3Key.ENDPOINT)); + hadoopConf.set("fs.s3a.ssl.enabled",config.getString(S3Key.SSLENABLED,"true")); + hadoopConf.set("fs.s3a.path.style.access",config.getString(S3Key.PATH_STYLE_ACCESS_ENABLED,"false")); + + } + + @Override + public void write(RecordReceiver lineReceiver, Configuration config, + TaskPluginCollector taskPluginCollector) + { + List columns = config.getListConfiguration(Key.COLUMN); + String compress = config.getString(Key.COMPRESS, "NONE").toUpperCase(); + StringJoiner joiner = new StringJoiner(","); + for (Configuration column : columns) { + if ("decimal".equals(column.getString(Key.TYPE))) { + joiner.add(String.format("%s:%s(%s,%s)", column.getString(Key.NAME), "decimal", + column.getInt(Key.PRECISION, Constant.DEFAULT_DECIMAL_MAX_PRECISION), + column.getInt(Key.SCALE, Constant.DEFAULT_DECIMAL_MAX_SCALE))); + } + else { + joiner.add(String.format("%s:%s", column.getString(Key.NAME), column.getString(Key.TYPE))); + } + } + TypeDescription schema = TypeDescription.fromString("struct<" + joiner + ">"); + try (Writer writer = OrcFile.createWriter(new Path(fileName), + OrcFile.writerOptions(hadoopConf) + .setSchema(schema) + .compress(CompressionKind.valueOf(compress)))) { + Record record; + VectorizedRowBatch batch = schema.createRowBatch(1024); + while ((record = lineReceiver.getFromReader()) != null) { + int row = batch.size++; + setRow(batch, row, record, columns, taskPluginCollector); + if (batch.size == batch.getMaxSize()) { + writer.addRowBatch(batch); + batch.reset(); + } + } + if (batch.size != 0) { + writer.addRowBatch(batch); + batch.reset(); + } + } + catch (IOException e) { + logger.error("IO exception occurred while writing file [{}}.", fileName); + DeleteObjectsRequest dor = DeleteObjectsRequest.builder() + .bucket(bucket) + .delete(Delete.builder().objects(ObjectIdentifier.builder().key(object).build()).build()) + .build(); + s3Client.deleteObjects(dor); + throw AddaxException.asAddaxException(IO_ERROR, e); + } + } +} diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/SupportHiveDataType.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/SupportHiveDataType.java new file mode 100644 index 000000000..8912cf4db --- /dev/null +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/SupportHiveDataType.java @@ -0,0 +1,40 @@ +/* + * 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.wgzhao.addax.plugin.writer.s3writer.formatwriter; + +public enum SupportHiveDataType +{ + TINYINT, + SMALLINT, + INT, + INTEGER, + BIGINT, + FLOAT, + DOUBLE, + TIMESTAMP, + DATE, + DECIMAL, + STRING, + VARCHAR, + CHAR, + LONG, + BOOLEAN, + BINARY +} diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/TextWriter.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/TextWriter.java index be189e95c..bf4223380 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/TextWriter.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/TextWriter.java @@ -118,6 +118,11 @@ public TextWriter setS3Client(S3Client s3Client) { return this; } + @Override + public void init(Configuration config) { + + } + @Override public void write(RecordReceiver lineReceiver, Configuration config, TaskPluginCollector taskPluginCollector) { From 357ab198c9b5416d43ba9d59b6dc222f5800125d Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Wed, 27 Nov 2024 10:31:51 +0800 Subject: [PATCH 07/40] S3 parquetwriter --- .../addax/plugin/writer/s3writer/S3Key.java | 3 +- .../addax/plugin/writer/s3writer/S3Util.java | 11 +- .../plugin/writer/s3writer/S3Writer.java | 17 +- .../s3writer/formatwriter/IFormatWriter.java | 1 + .../s3writer/formatwriter/OrcWriter.java | 62 ++- .../s3writer/formatwriter/ParquetWriter.java | 388 ++++++++++++++++++ .../formatwriter/SupportHiveDataType.java | 3 +- 7 files changed, 435 insertions(+), 50 deletions(-) create mode 100644 plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/ParquetWriter.java diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java index 5fbca34df..3ae281738 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Key.java @@ -2,8 +2,7 @@ import com.wgzhao.addax.common.base.Key; -public class S3Key extends Key -{ +public class S3Key extends Key { public static final String REGION = "region"; public static final String ENDPOINT = "endpoint"; diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java index a7e4a97fe..507c2c1d3 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Util.java @@ -11,28 +11,27 @@ import static com.wgzhao.addax.common.spi.ErrorCode.ILLEGAL_VALUE; -public class S3Util -{ +public class S3Util { public static S3Client initS3Client(Configuration conf) { String regionStr = conf.getString(S3Key.REGION); Region region = Region.of(regionStr); String accessId = conf.getString(S3Key.ACCESS_ID); String accessKey = conf.getString(S3Key.ACCESS_KEY); - String pathStyleAccessEnabled =conf.getString(S3Key.PATH_STYLE_ACCESS_ENABLED,""); + String pathStyleAccessEnabled = conf.getString(S3Key.PATH_STYLE_ACCESS_ENABLED, ""); - return initS3Client(conf.getString(S3Key.ENDPOINT), region, accessId, accessKey ,pathStyleAccessEnabled); + return initS3Client(conf.getString(S3Key.ENDPOINT), region, accessId, accessKey, pathStyleAccessEnabled); } - public static S3Client initS3Client(String endpoint, Region region, String accessId, String accessKey ,String pathStyleAccessEnabled) { + public static S3Client initS3Client(String endpoint, Region region, String accessId, String accessKey, String pathStyleAccessEnabled) { if (null == region) { region = Region.of("ap-northeast-1"); } try { AwsBasicCredentials awsCreds = AwsBasicCredentials.create(accessId, accessKey); return S3Client.builder().serviceConfiguration(e -> { - if("true".equals(pathStyleAccessEnabled)) { + if ("true".equals(pathStyleAccessEnabled)) { e.pathStyleAccessEnabled(true); } }) diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java index c67f76758..7c77ffda6 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java @@ -6,6 +6,7 @@ import com.wgzhao.addax.common.spi.Writer; import com.wgzhao.addax.common.util.Configuration; import com.wgzhao.addax.plugin.writer.s3writer.formatwriter.OrcWriter; +import com.wgzhao.addax.plugin.writer.s3writer.formatwriter.ParquetWriter; import com.wgzhao.addax.plugin.writer.s3writer.formatwriter.TextWriter; import com.wgzhao.addax.storage.writer.StorageWriterUtil; import org.apache.commons.lang3.StringUtils; @@ -220,8 +221,7 @@ public void startWrite(RecordReceiver lineReceiver) { .setFieldDelimiter(this.fieldDelimiter) .setMaxFileSize(this.maxFileSize); textWriter.write(lineReceiver, this.getPluginJobConf(), this.getTaskPluginCollector()); - } - if ("orc".equals(this.fileType)) { + } else if ("orc".equals(this.fileType)) { OrcWriter orcWriter = new OrcWriter() .setBucket(this.bucket) .setDateFormat(this.dateFormat) @@ -234,6 +234,19 @@ public void startWrite(RecordReceiver lineReceiver) { .setSslEnabled(this.sslEnabled); orcWriter.init(this.getPluginJobConf()); orcWriter.write(lineReceiver, this.getPluginJobConf(), this.getTaskPluginCollector()); + } else if ("parquet".equals(this.fileType)) { + ParquetWriter parquetWriter = new ParquetWriter() + .setBucket(this.bucket) + .setDateFormat(this.dateFormat) + .setEncoding(this.encoding) + .setHeader(this.header) + .setNullFormat(this.nullFormat) + .setObject(this.object) + .setS3Client(this.s3Client) + .setFieldDelimiter(this.fieldDelimiter) + .setSslEnabled(this.sslEnabled); + parquetWriter.init(this.getPluginJobConf()); + parquetWriter.write(lineReceiver, this.getPluginJobConf(), this.getTaskPluginCollector()); } } diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java index df2e9a806..e021ad4b6 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java @@ -6,6 +6,7 @@ public interface IFormatWriter { void init(Configuration config); + void write(RecordReceiver lineReceiver, Configuration config, TaskPluginCollector taskPluginCollector); } diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/OrcWriter.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/OrcWriter.java index 093e58ae8..90a87fcea 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/OrcWriter.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/OrcWriter.java @@ -36,8 +36,7 @@ import static com.wgzhao.addax.common.spi.ErrorCode.*; public class OrcWriter - implements IFormatWriter -{ + implements IFormatWriter { private final Logger logger = LoggerFactory.getLogger(OrcWriter.class.getName()); private char fieldDelimiter; private String nullFormat; @@ -53,6 +52,9 @@ public class OrcWriter private org.apache.hadoop.conf.Configuration hadoopConf = null; + public OrcWriter() { + } + public char getFieldDelimiter() { return fieldDelimiter; } @@ -71,8 +73,6 @@ public OrcWriter setNullFormat(String nullFormat) { return this; } - - public String getSslEnabled() { return sslEnabled; } @@ -118,7 +118,6 @@ public OrcWriter setObject(String object) { return this; } - public List getHeader() { return header; } @@ -127,6 +126,7 @@ public OrcWriter setHeader(List header) { this.header = header; return this; } + public S3Client getS3Client() { return s3Client; } @@ -136,23 +136,17 @@ public OrcWriter setS3Client(S3Client s3Client) { return this; } - - public OrcWriter() - { - } - /** * write an orc record * - * @param batch {@link VectorizedRowBatch} - * @param row row number - * @param record {@link Record} - * @param columns table columns, {@link List} + * @param batch {@link VectorizedRowBatch} + * @param row row number + * @param record {@link Record} + * @param columns table columns, {@link List} * @param taskPluginCollector {@link TaskPluginCollector} */ private void setRow(VectorizedRowBatch batch, int row, Record record, List columns, - TaskPluginCollector taskPluginCollector) - { + TaskPluginCollector taskPluginCollector) { for (int i = 0; i < columns.size(); i++) { Configuration eachColumnConf = columns.get(i); String type = eachColumnConf.getString(Key.TYPE).trim().toUpperCase(); @@ -160,8 +154,7 @@ private void setRow(VectorizedRowBatch batch, int row, Record record, List columns = config.getListConfiguration(Key.COLUMN); String compress = config.getString(Key.COMPRESS, "NONE").toUpperCase(); StringJoiner joiner = new StringJoiner(","); @@ -270,8 +258,7 @@ public void write(RecordReceiver lineReceiver, Configuration config, joiner.add(String.format("%s:%s(%s,%s)", column.getString(Key.NAME), "decimal", column.getInt(Key.PRECISION, Constant.DEFAULT_DECIMAL_MAX_PRECISION), column.getInt(Key.SCALE, Constant.DEFAULT_DECIMAL_MAX_SCALE))); - } - else { + } else { joiner.add(String.format("%s:%s", column.getString(Key.NAME), column.getString(Key.TYPE))); } } @@ -294,8 +281,7 @@ public void write(RecordReceiver lineReceiver, Configuration config, writer.addRowBatch(batch); batch.reset(); } - } - catch (IOException e) { + } catch (IOException e) { logger.error("IO exception occurred while writing file [{}}.", fileName); DeleteObjectsRequest dor = DeleteObjectsRequest.builder() .bucket(bucket) diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/ParquetWriter.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/ParquetWriter.java new file mode 100644 index 000000000..a455be265 --- /dev/null +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/ParquetWriter.java @@ -0,0 +1,388 @@ +/* + * 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.wgzhao.addax.plugin.writer.s3writer.formatwriter; + +import com.wgzhao.addax.common.base.Constant; +import com.wgzhao.addax.common.base.Key; +import com.wgzhao.addax.common.element.Column; +import com.wgzhao.addax.common.element.Record; +import com.wgzhao.addax.common.plugin.RecordReceiver; +import com.wgzhao.addax.common.plugin.TaskPluginCollector; +import com.wgzhao.addax.common.util.Configuration; +import com.wgzhao.addax.plugin.writer.s3writer.S3Key; +import org.apache.avro.Conversions; +import org.apache.avro.generic.GenericData; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.avro.AvroReadSupport; +import org.apache.parquet.avro.AvroWriteSupport; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.example.data.simple.SimpleGroupFactory; +import org.apache.parquet.hadoop.example.ExampleParquetWriter; +import org.apache.parquet.hadoop.example.GroupWriteSupport; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.hadoop.util.HadoopOutputFile; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.S3Client; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.sql.Timestamp; +import java.time.ZoneId; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.parquet.schema.LogicalTypeAnnotation.decimalType; + +public class ParquetWriter + implements IFormatWriter { + private static final Logger logger = LoggerFactory.getLogger(ParquetWriter.class.getName()); + private char fieldDelimiter; + private String nullFormat; + private String dateFormat; + private String encoding; + private String bucket; + private String object; + private List header; + private String sslEnabled; + private S3Client s3Client; + + private String fileName; + + private org.apache.hadoop.conf.Configuration hadoopConf = null; + + public ParquetWriter() { + } + + public char getFieldDelimiter() { + return fieldDelimiter; + } + + public ParquetWriter setFieldDelimiter(char fieldDelimiter) { + this.fieldDelimiter = fieldDelimiter; + return this; + } + + public String getNullFormat() { + return nullFormat; + } + + public ParquetWriter setNullFormat(String nullFormat) { + this.nullFormat = nullFormat; + return this; + } + + public String getSslEnabled() { + return sslEnabled; + } + + public ParquetWriter setSslEnabled(String sslEnabled) { + this.sslEnabled = sslEnabled; + return this; + } + + public String getDateFormat() { + return dateFormat; + } + + public ParquetWriter setDateFormat(String dateFormat) { + this.dateFormat = dateFormat; + return this; + } + + public String getEncoding() { + return encoding; + } + + public ParquetWriter setEncoding(String encoding) { + this.encoding = encoding; + return this; + } + + public String getBucket() { + return bucket; + } + + public ParquetWriter setBucket(String bucket) { + this.bucket = bucket; + return this; + } + + public String getObject() { + return object; + } + + public ParquetWriter setObject(String object) { + this.object = object; + return this; + } + + public List getHeader() { + return header; + } + + public ParquetWriter setHeader(List header) { + this.header = header; + return this; + } + + public S3Client getS3Client() { + return s3Client; + } + + public ParquetWriter setS3Client(S3Client s3Client) { + this.s3Client = s3Client; + return this; + } + + @Override + public void init(Configuration config) { + + this.fileName = "s3a://" + this.bucket + "/" + this.object; + + hadoopConf = new org.apache.hadoop.conf.Configuration(); + hadoopConf.set("fs.s3a.access.key", config.getString(S3Key.ACCESS_ID)); + hadoopConf.set("fs.s3a.secret.key", config.getString(S3Key.ACCESS_KEY)); + hadoopConf.set("fs.s3a.endpoint", config.getString(S3Key.ENDPOINT)); + hadoopConf.set("fs.s3a.ssl.enabled", config.getString(S3Key.SSLENABLED, "true")); + hadoopConf.set("fs.s3a.path.style.access", config.getString(S3Key.PATH_STYLE_ACCESS_ENABLED, "false")); + + } + + /* + * the schema of a parquet file is as follows: + * { + * "type": "record", + * "name": "testFile", + * "doc": "test records", + * "fields": + * [{ + * "name": "id", + * "type": ["null", "int"] + * + * }, + * { + * "name": "empName", + * "type": "string" + * } + * ] + * } + * "null" 表示该字段允许为空 + */ + @Override + public void write(RecordReceiver lineReceiver, Configuration config, TaskPluginCollector taskPluginCollector) { + List columns = config.getListConfiguration(Key.COLUMN); + String compress = config.getString(Key.COMPRESS, "UNCOMPRESSED").toUpperCase().trim(); + if ("NONE".equals(compress)) { + compress = "UNCOMPRESSED"; + } + CompressionCodecName codecName = CompressionCodecName.fromConf(compress); + // construct parquet schema + MessageType s = generateParquetSchema(columns); + Path path = new Path(fileName); + logger.info("Begin to write parquet file [{}]", fileName); + + GenericData decimalSupport = new GenericData(); + decimalSupport.addLogicalTypeConversion(new Conversions.DecimalConversion()); + hadoopConf.setBoolean(AvroReadSupport.READ_INT96_AS_FIXED, true); + hadoopConf.setBoolean(AvroWriteSupport.WRITE_FIXED_AS_INT96, true); + GroupWriteSupport.setSchema(s, hadoopConf); + Map extraMeta = new HashMap<>(); + // hive need timezone info to handle timestamp + extraMeta.put("writer.time.zone", ZoneId.systemDefault().toString()); + try (org.apache.parquet.hadoop.ParquetWriter writer = ExampleParquetWriter.builder(HadoopOutputFile.fromPath(path, hadoopConf)) + .withCompressionCodec(codecName) + .withConf(hadoopConf) + .enableDictionaryEncoding() + .withPageSize(1024) + .withDictionaryPageSize(512) + .withValidation(false) + .withWriterVersion(ParquetProperties.WriterVersion.PARQUET_1_0) + .withExtraMetaData(extraMeta) + .build()) { + SimpleGroupFactory simpleGroupFactory = new SimpleGroupFactory(s); + Group group; + Record record; + while ((record = lineReceiver.getFromReader()) != null) { + group = buildRecord(record, columns, taskPluginCollector, simpleGroupFactory); + writer.write(group); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public Group buildRecord( + Record record, List columns, + TaskPluginCollector taskPluginCollector, SimpleGroupFactory simpleGroupFactory) { + Column column; + Group group = simpleGroupFactory.newGroup(); + for (int i = 0; i < record.getColumnNumber(); i++) { + column = record.getColumn(i); + String colName = columns.get(i).getString(Key.NAME); + String typename = columns.get(i).getString(Key.TYPE).toUpperCase(); + if (null == column || column.getRawData() == null) { + continue; + } + SupportHiveDataType columnType = SupportHiveDataType.valueOf(typename); + switch (columnType) { + case INT: + case INTEGER: + group.append(colName, Integer.parseInt(column.getRawData().toString())); + break; + case BIGINT: + case LONG: + group.append(colName, column.asLong()); + break; + case FLOAT: + group.append(colName, column.asDouble().floatValue()); + break; + case DOUBLE: + group.append(colName, column.asDouble()); + break; + case STRING: + group.append(colName, column.asString()); + break; + case BOOLEAN: + group.append(colName, column.asBoolean()); + break; + case DECIMAL: + int scale = columns.get(i).getInt(Key.SCALE, Constant.DEFAULT_DECIMAL_MAX_SCALE); + group.append(colName, decimalToBinary(column.asString(), scale)); + break; + case TIMESTAMP: + group.append(colName, tsToBinary(column.asTimestamp())); + break; + case DATE: + group.append(colName, (int) Math.round(column.asLong() * 1.0 / 86400000)); + break; + default: + logger.debug("convert type[{}] into string", column.getType()); + group.append(colName, column.asString()); + break; + } + } + return group; + } + + /** + * convert timestamp to parquet INT96 + * + * @param ts the {@link Timestamp} want to convert + * @return {@link Binary} + */ + private Binary tsToBinary(Timestamp ts) { + long millis = ts.getTime(); + int julianDays = (int) (millis / 86400000L) + 2440588; + long nanosOfDay = (millis % 86400000L) * 1000000L; + + // Write INT96 timestamp + byte[] timestampBuffer = new byte[12]; + ByteBuffer buf = ByteBuffer.wrap(timestampBuffer); + buf.order(ByteOrder.LITTLE_ENDIAN).putLong(nanosOfDay).putInt(julianDays).flip(); + + // This is the properly encoded INT96 timestamp + return Binary.fromReusedByteArray(timestampBuffer); + } + + /** + * convert Decimal to {@link Binary} using fix 16 bytes array + * + * @param bigDecimal the decimal value string want to convert + * @return {@link Binary} + */ + private Binary decimalToBinary(String bigDecimal, int scale) { + int realScale = new BigDecimal(bigDecimal).scale(); + RoundingMode mode = scale >= realScale ? RoundingMode.UNNECESSARY : RoundingMode.HALF_UP; + byte[] decimalBytes = new BigDecimal(bigDecimal) + .setScale(scale, mode) + .unscaledValue() + .toByteArray(); + + byte[] myDecimalBuffer = new byte[16]; + if (myDecimalBuffer.length >= decimalBytes.length) { + //Because we set our fixed byte array size as 16 bytes, we need to + //pad-left our original value's bytes with zeros + int myDecimalBufferIndex = myDecimalBuffer.length - 1; + for (int i = decimalBytes.length - 1; i >= 0; i--) { + myDecimalBuffer[myDecimalBufferIndex] = decimalBytes[i]; + myDecimalBufferIndex--; + } + return Binary.fromConstantByteArray(myDecimalBuffer); + } else { + throw new IllegalArgumentException(String.format("Decimal size: %d was greater than the allowed max: %d", + decimalBytes.length, myDecimalBuffer.length)); + } + } + + private MessageType generateParquetSchema(List columns) { + String type; + String fieldName; + Type t; + Types.MessageTypeBuilder builder = Types.buildMessage(); + Type.Repetition repetition = Type.Repetition.OPTIONAL; + for (Configuration column : columns) { + type = column.getString(Key.TYPE).trim().toUpperCase(); + fieldName = column.getString(Key.NAME); + switch (type) { + case "INT": + t = Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition).named(fieldName); + break; + case "BIGINT": + case "LONG": + t = Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition).named(fieldName); + break; + case "DECIMAL": + // use fixed 16 bytes array + int prec = column.getInt(Key.PRECISION, Constant.DEFAULT_DECIMAL_MAX_PRECISION); + int scale = column.getInt(Key.SCALE, Constant.DEFAULT_DECIMAL_MAX_SCALE); + t = Types.primitive(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, repetition) + .length(16) + .as(decimalType(scale, prec)) + .named(fieldName); + break; + case "STRING": + t = Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition).as(LogicalTypeAnnotation.stringType()).named(fieldName); + break; + case "BYTES": + t = Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition).named(fieldName); + break; + case "DATE": + t = Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition).as(LogicalTypeAnnotation.dateType()).named(fieldName); + break; + case "TIMESTAMP": + t = Types.primitive(PrimitiveType.PrimitiveTypeName.INT96, repetition).named(fieldName); + break; + default: + t = Types.primitive(PrimitiveType.PrimitiveTypeName.valueOf(type), Type.Repetition.OPTIONAL).named(fieldName); + break; + } + builder.addField(t); + } + return builder.named("addax"); + } +} diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/SupportHiveDataType.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/SupportHiveDataType.java index 8912cf4db..8baebfae0 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/SupportHiveDataType.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/SupportHiveDataType.java @@ -19,8 +19,7 @@ package com.wgzhao.addax.plugin.writer.s3writer.formatwriter; -public enum SupportHiveDataType -{ +public enum SupportHiveDataType { TINYINT, SMALLINT, INT, From 8667e8f23296d01f84f606c43e7010fb316dcc56 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 28 Nov 2024 11:16:10 +0800 Subject: [PATCH 08/40] =?UTF-8?q?DataBaseType=20should=20not=20be=20call?= =?UTF-8?q?=20set=20method=EF=BC=8Cso=20that=20the=20driver=20is=20error?= =?UTF-8?q?=20when=20rdbmsreader=20to=20rdbmswriter?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/wgzhao/addax/rdbms/reader/CommonRdbmsReader.java | 7 ++++++- .../rdbms/reader/util/OriginalConfPretreatmentUtil.java | 7 ++++++- .../java/com/wgzhao/addax/rdbms/util/DataBaseType.java | 7 ++----- .../rdbms/writer/util/OriginalConfPretreatmentUtil.java | 5 ++++- .../addax/plugin/reader/rdbmsreader/RdbmsReader.java | 8 +++++--- .../addax/plugin/writer/rdbmswriter/RdbmsWriter.java | 7 +++++-- 6 files changed, 28 insertions(+), 13 deletions(-) diff --git a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/CommonRdbmsReader.java b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/CommonRdbmsReader.java index 8dcc20c74..b62e55f7f 100644 --- a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/CommonRdbmsReader.java +++ b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/CommonRdbmsReader.java @@ -272,7 +272,12 @@ protected Column createColumn(ResultSet rs, ResultSetMetaData metaData, int i) return new DateColumn(rs.getDate(i)); case Types.TIMESTAMP: - return new TimestampColumn(rs.getTimestamp(i, Calendar.getInstance())); + if(!"org.apache.hive.jdbc.HiveDriver".equals(this.dataBaseType.getDriverClassName())){ + return new TimestampColumn(rs.getTimestamp(i, Calendar.getInstance())); + }else{ + //hive not support method(Timestamp getTimestamp(int columnIndex, Calendar cal)) + return new TimestampColumn(rs.getTimestamp(i)); + } case Types.BINARY: case Types.VARBINARY: diff --git a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/util/OriginalConfPretreatmentUtil.java b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/util/OriginalConfPretreatmentUtil.java index b087d228b..9180a57cb 100644 --- a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/util/OriginalConfPretreatmentUtil.java +++ b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/util/OriginalConfPretreatmentUtil.java @@ -35,6 +35,7 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import static com.wgzhao.addax.common.spi.ErrorCode.CONFIG_ERROR; @@ -118,7 +119,11 @@ private static void dealJdbcAndTable(Configuration originalConfig) String driverClass = connConf.getString(Key.JDBC_DRIVER, null); if (driverClass != null && !driverClass.isEmpty()) { LOG.warn("use specified driver class: {}", driverClass); - dataBaseType.setDriverClassName(driverClass); + + + Arrays.stream(DataBaseType.values()).filter( + d -> d.getDriverClassName().equals(driverClass)).findFirst().ifPresent(d -> + dataBaseType=d); } connConf.getNecessaryValue(Key.JDBC_URL, REQUIRED_VALUE); diff --git a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java index caac6b37c..8ebf65b60 100644 --- a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java +++ b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java @@ -50,7 +50,8 @@ public enum DataBaseType Sybase("sybase", "com.sybase.jdbc4.jdbc.SybDriver"), Databend("databend", "com.databend.jdbc.DatabendDriver"), Access("access","net.ucanaccess.jdbc.UcanaccessDriver"), - HANA("hana", "com.sap.db.jdbc.Driver"); + HANA("hana", "com.sap.db.jdbc.Driver"), + VERTICA("vertica", "com.vertica.jdbc.Driver"); private static final Pattern jdbcUrlPattern = Pattern.compile("jdbc:\\w+:(?:thin:url=|//|thin:@|)([\\w\\d.,]+).*"); @@ -162,8 +163,4 @@ public String getTypeName() return typeName; } - public void setDriverClassName(String driverClassName) - { - this.driverClassName = driverClassName; - } } diff --git a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/writer/util/OriginalConfPretreatmentUtil.java b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/writer/util/OriginalConfPretreatmentUtil.java index 0bc49d743..cc953df08 100644 --- a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/writer/util/OriginalConfPretreatmentUtil.java +++ b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/writer/util/OriginalConfPretreatmentUtil.java @@ -38,6 +38,7 @@ import java.sql.Connection; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import static com.wgzhao.addax.common.spi.ErrorCode.CONFIG_ERROR; @@ -92,7 +93,9 @@ public static void simplifyConf(Configuration originalConfig) String driverClass = connConf.getString(Key.JDBC_DRIVER, null); if (driverClass != null && !driverClass.isEmpty()) { LOG.warn("Use specified driver class [{}]", driverClass); - dataBaseType.setDriverClassName(driverClass); + Arrays.stream(DataBaseType.values()).filter( + d -> d.getDriverClassName().equals(driverClass)).findFirst().ifPresent(d -> + dataBaseType=d); } String jdbcUrl = connConf.getString(Key.JDBC_URL); if (StringUtils.isBlank(jdbcUrl)) { diff --git a/plugin/reader/rdbmsreader/src/main/java/com/wgzhao/addax/plugin/reader/rdbmsreader/RdbmsReader.java b/plugin/reader/rdbmsreader/src/main/java/com/wgzhao/addax/plugin/reader/rdbmsreader/RdbmsReader.java index 3e3d30fd7..c3e59fe0c 100644 --- a/plugin/reader/rdbmsreader/src/main/java/com/wgzhao/addax/plugin/reader/rdbmsreader/RdbmsReader.java +++ b/plugin/reader/rdbmsreader/src/main/java/com/wgzhao/addax/plugin/reader/rdbmsreader/RdbmsReader.java @@ -40,7 +40,7 @@ public class RdbmsReader extends Reader { - private static final DataBaseType DATABASE_TYPE = DataBaseType.RDBMS; + private static DataBaseType DATABASE_TYPE = DataBaseType.RDBMS; public static class Job extends Reader.Job @@ -71,11 +71,13 @@ public void init() final String jdbcType = connection.getString(Key.JDBC_URL).split(":")[1]; Arrays.stream(DataBaseType.values()).filter( dataBaseType -> dataBaseType.getTypeName().equals(jdbcType)).findFirst().ifPresent(dataBaseType -> - DATABASE_TYPE.setDriverClassName(dataBaseType.getDriverClassName())); + DATABASE_TYPE=dataBaseType); } else { // use custom jdbc driver - DATABASE_TYPE.setDriverClassName(jdbcDriver); + Arrays.stream(DataBaseType.values()).filter( + dataBaseType -> dataBaseType.getDriverClassName().equals(jdbcDriver)).findFirst().ifPresent(dataBaseType -> + DATABASE_TYPE=dataBaseType); } this.commonRdbmsReaderMaster = new SubCommonRdbmsReader.Job(DATABASE_TYPE); this.originalConfig = this.commonRdbmsReaderMaster.init(this.originalConfig); diff --git a/plugin/writer/rdbmswriter/src/main/java/com/wgzhao/addax/plugin/writer/rdbmswriter/RdbmsWriter.java b/plugin/writer/rdbmswriter/src/main/java/com/wgzhao/addax/plugin/writer/rdbmswriter/RdbmsWriter.java index 48112ac60..9c93ce99e 100644 --- a/plugin/writer/rdbmswriter/src/main/java/com/wgzhao/addax/plugin/writer/rdbmswriter/RdbmsWriter.java +++ b/plugin/writer/rdbmswriter/src/main/java/com/wgzhao/addax/plugin/writer/rdbmswriter/RdbmsWriter.java @@ -29,6 +29,7 @@ import com.wgzhao.addax.rdbms.writer.CommonRdbmsWriter; import org.apache.commons.lang3.StringUtils; +import java.util.Arrays; import java.util.List; import static com.wgzhao.addax.common.base.Key.JDBC_DRIVER; @@ -38,7 +39,7 @@ public class RdbmsWriter extends Writer { - private static final DataBaseType DATABASE_TYPE = DataBaseType.RDBMS; + private static DataBaseType DATABASE_TYPE = DataBaseType.RDBMS; public static class Job extends Writer.Job @@ -63,7 +64,9 @@ public void init() throw AddaxException.asAddaxException(REQUIRED_VALUE, "config 'driver' is required and must not be empty"); } // use special jdbc driver class - DATABASE_TYPE.setDriverClassName(jdbcDriver); + Arrays.stream(DataBaseType.values()).filter( + d -> d.getDriverClassName().equals(jdbcDriver)).findFirst().ifPresent(d -> + DATABASE_TYPE=d); this.commonRdbmsWriterJob = new CommonRdbmsWriter.Job(DATABASE_TYPE); commonRdbmsWriterJob.init(originalConfig); } From d150110ddc9e1d90a1da9bcb2475cc0a5868efa3 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 28 Nov 2024 13:39:56 +0800 Subject: [PATCH 09/40] add other dbs drivers --- .../java/com/wgzhao/addax/rdbms/util/DataBaseType.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java index 8ebf65b60..55e4bd727 100644 --- a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java +++ b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java @@ -51,7 +51,14 @@ public enum DataBaseType Databend("databend", "com.databend.jdbc.DatabendDriver"), Access("access","net.ucanaccess.jdbc.UcanaccessDriver"), HANA("hana", "com.sap.db.jdbc.Driver"), - VERTICA("vertica", "com.vertica.jdbc.Driver"); + VERTICA("vertica", "com.vertica.jdbc.Driver"), + DM("dm","dm.jdbc.driver.DmDriver"), + OSCAR("oscar","com.oscar.Driver"), + KINGBASE8("kingbase8","com.kingbase8.Driver"), + HIGHGO("highgo","com.highgo.jdbc.Driver"), + OCEANBASE("oceanbase","com.alipay.oceanbase.jdbc.Driver"), + GOLDENDB("goldendb","com.goldendb.jdbc.Driver"), + GBASEDBT("gbasedbt-sqli","com.gbasedbt.jdbc.Driver"); private static final Pattern jdbcUrlPattern = Pattern.compile("jdbc:\\w+:(?:thin:url=|//|thin:@|)([\\w\\d.,]+).*"); From b697d54b50785aba6f7cf591f4764c03ac9573f1 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Tue, 21 Jan 2025 13:26:42 +0800 Subject: [PATCH 10/40] [feature][plugin][paimonwriter] add paimon format file writer --- docs/writer/paimonwriter.md | 88 ++++++ package.xml | 8 + plugin/writer/paimonwriter/package.xml | 37 +++ plugin/writer/paimonwriter/pom.xml | 225 ++++++++++++++ .../writer/paimonwriter/PaimonHelper.java | 53 ++++ .../writer/paimonwriter/PaimonWriter.java | 291 ++++++++++++++++++ .../src/main/resources/plugin.json | 6 + .../main/resources/plugin_job_template.json | 13 + pom.xml | 1 + 9 files changed, 722 insertions(+) create mode 100644 docs/writer/paimonwriter.md create mode 100644 plugin/writer/paimonwriter/package.xml create mode 100644 plugin/writer/paimonwriter/pom.xml create mode 100644 plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonHelper.java create mode 100644 plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java create mode 100644 plugin/writer/paimonwriter/src/main/resources/plugin.json create mode 100644 plugin/writer/paimonwriter/src/main/resources/plugin_job_template.json diff --git a/docs/writer/paimonwriter.md b/docs/writer/paimonwriter.md new file mode 100644 index 000000000..a1973c1ea --- /dev/null +++ b/docs/writer/paimonwriter.md @@ -0,0 +1,88 @@ +# Paimon Writer + +Paimon Writer 提供向 已有的paimon表写入数据的能力。 + +## 配置样例 + +```json +--8<-- "jobs/paimonwriter.json" +``` + +## 参数说明 + +| 配置项 | 是否必须 | 数据类型 | 默认值 | 说明 | +|:-------------|:----:|--------|----|------------------------------------------------| +| dbName | 是 | string | 无 | 要写入的paimon数据库名 | +| tableName | 是 | string | 无 | 要写入的paimon表名 | +| writeMode | 是 | string | 无 | 写入模式,详述见下 | +| paimonConfig | 是 | json | {} | 里可以配置与 Paimon catalog和Hadoop 相关的一些高级参数,比如HA的配置 | + + + +### writeMode + +写入前数据清理处理模式: + +- append,写入前不做任何处理,直接写入,不清除原来的数据。 +- truncate 写入前先清空表,再写入。 + +### paimonConfig + +`paimonConfig` 里可以配置与 Paimon catalog和Hadoop 相关的一些高级参数,比如HA的配置 +```json +{ + "name": "paimonwriter", + "parameter": { + "dbName": "test", + "tableName": "test2", + "writeMode": "truncate", + "paimonConfig": { + "warehouse": "file:///g:/paimon", + "metastore": "filesystem" + } + } +} +``` +```json +{ + "paimonConfig": { + "warehouse": "hdfs://nameservice1/user/hive/paimon", + "metastore": "filesystem", + "fs.defaultFS":"hdfs://nameservice1", + "hadoop.security.authentication" : "kerberos", + "hadoop.kerberos.principal" : "hive/_HOST@XXXX.COM", + "hadoop.kerberos.keytab" : "/tmp/hive@XXXX.COM.keytab", + "ha.zookeeper.quorum" : "test-pr-nn1:2181,test-pr-nn2:2181,test-pr-nn3:2181", + "dfs.nameservices" : "nameservice1", + "dfs.namenode.rpc-address.nameservice1.namenode371" : "test-pr-nn2:8020", + "dfs.namenode.rpc-address.nameservice1.namenode265": "test-pr-nn1:8020", + "dfs.namenode.keytab.file" : "/tmp/hdfs@XXXX.COM.keytab", + "dfs.namenode.keytab.enabled" : "true", + "dfs.namenode.kerberos.principal" : "hdfs/_HOST@XXXX.COM", + "dfs.namenode.kerberos.internal.spnego.principal" : "HTTP/_HOST@XXXX.COM", + "dfs.ha.namenodes.nameservice1" : "namenode265,namenode371", + "dfs.datanode.keytab.file" : "/tmp/hdfs@XXXX.COM.keytab", + "dfs.datanode.keytab.enabled" : "true", + "dfs.datanode.kerberos.principal" : "hdfs/_HOST@XXXX.COM", + "dfs.client.use.datanode.hostname" : "false", + "dfs.client.failover.proxy.provider.nameservice1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.balancer.keytab.file" : "/tmp/hdfs@XXXX.COM.keytab", + "dfs.balancer.keytab.enabled" : "true", + "dfs.balancer.kerberos.principal" : "hdfs/_HOST@XXXX.COM" + } +} +``` + + +## 类型转换 + +| Addax 内部类型 | Paimon 数据类型 | +|------------|------------------------------| +| Integer | TINYINT,SMALLINT,INT,INTEGER | +| Long | BIGINT | +| Double | FLOAT,DOUBLE,DECIMAL | +| String | STRING,VARCHAR,CHAR | +| Boolean | BOOLEAN | +| Date | DATE,TIMESTAMP | +| Bytes | BINARY | + diff --git a/package.xml b/package.xml index f787ad819..b5a82b669 100644 --- a/package.xml +++ b/package.xml @@ -609,5 +609,13 @@ 0644 addax-${project.version} + + plugin/writer/paimonwriter/target/paimonwriter-${project.version}/ + + **/*.* + + 0644 + addax-${project.version} + diff --git a/plugin/writer/paimonwriter/package.xml b/plugin/writer/paimonwriter/package.xml new file mode 100644 index 000000000..8ecab9f9f --- /dev/null +++ b/plugin/writer/paimonwriter/package.xml @@ -0,0 +1,37 @@ + + release + + dir + + false + + + src/main/resources + + *.json + + plugin/writer/${project.artifactId} + + + target/ + + ${project.artifactId}-${project.version}.jar + + plugin/writer/${project.artifactId} + + + + + + false + plugin/writer/${project.artifactId}/libs + runtime + + com.wgzhao.addax:* + + + + diff --git a/plugin/writer/paimonwriter/pom.xml b/plugin/writer/paimonwriter/pom.xml new file mode 100644 index 000000000..3aa053801 --- /dev/null +++ b/plugin/writer/paimonwriter/pom.xml @@ -0,0 +1,225 @@ + + + 4.0.0 + + + com.wgzhao.addax + addax-all + 4.2.3-SNAPSHOT + ../../../pom.xml + + + paimonwriter + paimon-writer + PaimonWriter提供了本地写入paimon格式文件功能,建议开发、测试环境使用。 + jar + + + + com.wgzhao.addax + addax-common + ${project.version} + + + slf4j-log4j12 + org.slf4j + + + + + + com.wgzhao.addax + addax-storage + ${project.version} + + + + org.apache.paimon + paimon-bundle + 1.0.0 + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + + org.apache.hadoop + hadoop-aws + ${hadoop.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + + + com.fasterxml.jackson.core + jackson-databind + + + commons-codec + commons-codec + + + io.netty + netty + + + org.eclipse.jetty + jetty-util + + + + + + + com.fasterxml.woodstox + woodstox-core + ${woodstox.version} + + + + + + + maven-assembly-plugin + + + package.xml + + ${project.artifactId}-${project.version} + + + + release + package + + single + + + + + + + diff --git a/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonHelper.java b/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonHelper.java new file mode 100644 index 000000000..187080ead --- /dev/null +++ b/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonHelper.java @@ -0,0 +1,53 @@ +package com.wgzhao.addax.plugin.writer.paimonwriter; + +import com.wgzhao.addax.common.exception.AddaxException; +import com.wgzhao.addax.common.util.Configuration; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.options.Options; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.wgzhao.addax.common.spi.ErrorCode.LOGIN_ERROR; + +public class PaimonHelper { + private static final Logger LOG = LoggerFactory.getLogger(PaimonHelper.class); + + public static void kerberosAuthentication(org.apache.hadoop.conf.Configuration hadoopConf, String kerberosPrincipal, String kerberosKeytabFilePath) throws Exception { + if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) { + UserGroupInformation.setConfiguration(hadoopConf); + try { + UserGroupInformation.loginUserFromKeytab(kerberosPrincipal, kerberosKeytabFilePath); + } catch (Exception e) { + String message = String.format("kerberos authentication failed, keytab file: [%s], principal: [%s]", + kerberosKeytabFilePath, kerberosPrincipal); + LOG.error(message); + throw AddaxException.asAddaxException(LOGIN_ERROR, e); + } + } + } + + public static Options getOptions(Configuration conf){ + Options options = new Options(); + conf.getMap("paimonConfig").forEach((k, v) -> options.set(k, String.valueOf(v))); + return options; + } + + public static CatalogContext getCatalogContext(Options options) { + CatalogContext context = null; + if(options.get("warehouse").startsWith("hdfs://")){ + + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + options.toMap().forEach((k, v) -> hadoopConf.set(k, String.valueOf(v))); + UserGroupInformation.setConfiguration(hadoopConf); + context = CatalogContext.create(options,hadoopConf); + } else { + + context = CatalogContext.create(options); + } + + return context; + } + +} diff --git a/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java b/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java new file mode 100644 index 000000000..c02ba3542 --- /dev/null +++ b/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java @@ -0,0 +1,291 @@ +package com.wgzhao.addax.plugin.writer.paimonwriter; + +import com.alibaba.fastjson2.JSON; +import com.wgzhao.addax.common.element.Column; +import com.wgzhao.addax.common.element.Record; +import com.wgzhao.addax.common.plugin.RecordReceiver; +import com.wgzhao.addax.common.spi.Writer; +import com.wgzhao.addax.common.util.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.data.*; +import org.apache.paimon.options.Options; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.sink.BatchTableCommit; +import org.apache.paimon.table.sink.BatchTableWrite; +import org.apache.paimon.table.sink.BatchWriteBuilder; +import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypeRoot; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static com.wgzhao.addax.common.base.Key.KERBEROS_KEYTAB_FILE_PATH; +import static com.wgzhao.addax.common.base.Key.KERBEROS_PRINCIPAL; + + +public class PaimonWriter extends Writer { + public static class Job + extends Writer.Job { + private static final Logger LOG = LoggerFactory.getLogger(Job.class); + private Configuration conf = null; + private BatchWriteBuilder writeBuilder = null; + + @Override + public void init() { + this.conf = this.getPluginJobConf(); + + Options options = PaimonHelper.getOptions(this.conf); + CatalogContext context = PaimonHelper.getCatalogContext(options); + + + + if ("kerberos".equals(options.get("hadoop.security.authentication"))) { + String kerberosKeytabFilePath = options.get(KERBEROS_KEYTAB_FILE_PATH); + String kerberosPrincipal = options.get(KERBEROS_PRINCIPAL); + try { + PaimonHelper.kerberosAuthentication(context.hadoopConf(), kerberosPrincipal, kerberosKeytabFilePath); + LOG.info("kerberos Authentication success"); + + FileSystem fs = FileSystem.get(context.hadoopConf()); + fs.getStatus().getCapacity(); + } catch (Exception e) { + LOG.error("kerberos Authentication error", e); + throw new RuntimeException(e); + } + } + try (Catalog catalog = CatalogFactory.createCatalog(context)) { + + String dbName = this.conf.getString("dbName"); + String tableName = this.conf.getString("tableName"); + Identifier identifier = Identifier.create(dbName, tableName); + + Table table = catalog.getTable(identifier); + + writeBuilder = table.newBatchWriteBuilder(); + + + } catch (Exception e) { + LOG.error("init paimon error", e); + throw new RuntimeException(e); + } + + + } + + @Override + public List split(int mandatoryNumber) { + List configurations = new ArrayList<>(mandatoryNumber); + for (int i = 0; i < mandatoryNumber; i++) { + configurations.add(conf); + } + return configurations; + } + + @Override + public void prepare() { + String writeMode = this.conf.getString("writeMode"); + if ("truncate".equalsIgnoreCase(writeMode)) { + if (writeBuilder != null) { + LOG.info("You specify truncate writeMode, begin to clean history data."); + BatchTableWrite write = writeBuilder.withOverwrite().newWrite(); + BatchTableCommit commit = writeBuilder.newCommit(); + commit.commit(new ArrayList<>()); + try { + write.close(); + } catch (Exception e) { + LOG.error("close paimon write error", e); + throw new RuntimeException(e); + } + } + } + } + + @Override + public void destroy() { + + } + } + + public static class Task + extends Writer.Task { + + private static final Logger log = LoggerFactory.getLogger(Task.class); + private Configuration conf = null; + private BatchWriteBuilder writeBuilder = null; + private Integer batchSize = 1000; + private List columnList = new ArrayList<>(); + private List typeList = new ArrayList<>(); + + @Override + public void startWrite(RecordReceiver recordReceiver) { + + List writerBuffer = new ArrayList<>(this.batchSize); + Record record; + long total = 0; + while ((record = recordReceiver.getFromReader()) != null) { + writerBuffer.add(record); + if (writerBuffer.size() >= this.batchSize) { + total += doBatchInsert(writerBuffer); + writerBuffer.clear(); + } + } + + if (!writerBuffer.isEmpty()) { + total += doBatchInsert(writerBuffer); + writerBuffer.clear(); + } + + String msg = String.format("task end, write size :%d", total); + getTaskPluginCollector().collectMessage("writeSize", String.valueOf(total)); + log.info(msg); + + } + + @Override + public void init() { + this.conf = super.getPluginJobConf(); + + batchSize = conf.getInt("batchSize", 1000); + + Options options = PaimonHelper.getOptions(this.conf); + CatalogContext context = PaimonHelper.getCatalogContext(options); + + if ("kerberos".equals(options.get("hadoop.security.authentication"))) { + String kerberosKeytabFilePath = options.get(KERBEROS_KEYTAB_FILE_PATH); + String kerberosPrincipal = options.get(KERBEROS_PRINCIPAL); + try { + PaimonHelper.kerberosAuthentication(context.hadoopConf(), kerberosPrincipal, kerberosKeytabFilePath); + log.info("kerberos Authentication success"); + } catch (Exception e) { + log.error("kerberos Authentication error", e); + throw new RuntimeException(e); + } + } + + try (Catalog catalog = CatalogFactory.createCatalog(context)) { + + String dbName = this.conf.getString("dbName"); + String tableName = this.conf.getString("tableName"); + Identifier identifier = Identifier.create(dbName, tableName); + + Table table = catalog.getTable(identifier); + + columnList = table.rowType().getFields(); + typeList = table.rowType().getFieldTypes(); + writeBuilder = table.newBatchWriteBuilder(); + + + } catch (Exception e) { + log.error("init paimon error", e); + throw new RuntimeException(e); + } + } + + @Override + public void destroy() { + + } + + private long doBatchInsert(final List writerBuffer) { + BatchTableWrite write = writeBuilder.newWrite(); + GenericRow data; + for (Record record : writerBuffer) { + data = new GenericRow(columnList.size()); + StringBuilder id = new StringBuilder(); + for (int i = 0; i < record.getColumnNumber(); i++) { + Column column = record.getColumn(i); + if (i >= columnList.size()) { + throw new RuntimeException("columnList size is " + columnList.size() + ", but record column number is " + record.getColumnNumber()); + } + String columnName = columnList.get(i).name(); + DataType columnType = typeList.get(i); + //如果是数组类型,那它传入的必是字符串类型 + if (columnType.getTypeRoot().equals(DataTypeRoot.ARRAY)) { + if (null == column.asString()) { + data.setField(i, null); + } else { + String[] dataList = column.asString().split(","); + data.setField(i, new GenericArray(dataList)); + } + } else { + switch (columnType.getTypeRoot()) { + + case DATE: + try { + data.setField(i, Timestamp.fromEpochMillis(column.asLong())); + } catch (Exception e) { + getTaskPluginCollector().collectDirtyRecord(record, String.format("时间类型解析失败 [%s:%s] exception: %s", columnName, column.toString(), e)); + } + break; + case CHAR: + case VARCHAR: + data.setField(i, BinaryString.fromString(column.asString())); + break; + case BOOLEAN: + data.setField(i, column.asBoolean()); + break; + case VARBINARY: + case BINARY: + data.setField(i, column.asBytes()); + break; + case BIGINT: + data.setField(i, column.asLong()); + break; + case INTEGER: + case SMALLINT: + case TINYINT: + data.setField(i, column.asBigInteger().intValue()); + break; + case FLOAT: + case DOUBLE: + data.setField(i, column.asDouble()); + break; + case MAP: + try { + data.setField(i, new GenericMap(JSON.parseObject(column.asString(), Map.class))); + } catch (Exception e) { + getTaskPluginCollector().collectDirtyRecord(record, String.format("MAP类型解析失败 [%s:%s] exception: %s", columnName, column.toString(), e)); + } + break; + default: + getTaskPluginCollector().collectDirtyRecord(record, "类型错误:不支持的类型:" + columnType + " " + columnName); + } + } + } + + try { + write.write(data); + } catch (Exception e) { + throw new RuntimeException(e); + } + + } + + List messages = null; + try { + messages = write.prepareCommit(); + BatchTableCommit commit = writeBuilder.newCommit(); + commit.commit(messages); + + + write.close(); + + return messages.size(); + } catch (Exception e) { + throw new RuntimeException(e); + } + + + } + + } +} diff --git a/plugin/writer/paimonwriter/src/main/resources/plugin.json b/plugin/writer/paimonwriter/src/main/resources/plugin.json new file mode 100644 index 000000000..b1dd4c177 --- /dev/null +++ b/plugin/writer/paimonwriter/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "paimonwriter", + "class": "com.wgzhao.addax.plugin.writer.paimonwriter.PaimonWriter", + "description": "write data to paimon", + "developer": "wgzhao" +} diff --git a/plugin/writer/paimonwriter/src/main/resources/plugin_job_template.json b/plugin/writer/paimonwriter/src/main/resources/plugin_job_template.json new file mode 100644 index 000000000..681c940f9 --- /dev/null +++ b/plugin/writer/paimonwriter/src/main/resources/plugin_job_template.json @@ -0,0 +1,13 @@ +{ + "name": "paimonwriter", + "parameter": { + + "dbName": "test", + "tableName": "test", + "writeMode": "truncate", + "paimonConfig": { + "warehouse": "file:///tmp/paimon", + "metastore": "filesystem" + } + } +} diff --git a/pom.xml b/pom.xml index 98d64beaf..266a21b41 100644 --- a/pom.xml +++ b/pom.xml @@ -395,6 +395,7 @@ plugin/writer/sybasewriter plugin/writer/tdenginewriter plugin/writer/txtfilewriter + plugin/writer/paimonwriter From d5d53f3a29cc79d3cc61c139945d0c1fd93a519a Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Tue, 21 Jan 2025 15:34:10 +0800 Subject: [PATCH 11/40] fixed timestamp,decimal,null type value bug --- .../writer/paimonwriter/PaimonWriter.java | 22 +++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java b/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java index c02ba3542..116cf9b41 100644 --- a/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java +++ b/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java @@ -21,6 +21,7 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeRoot; +import org.apache.paimon.types.DecimalType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -203,6 +204,9 @@ private long doBatchInsert(final List writerBuffer) { StringBuilder id = new StringBuilder(); for (int i = 0; i < record.getColumnNumber(); i++) { Column column = record.getColumn(i); + if(column ==null ){ + continue; + } if (i >= columnList.size()) { throw new RuntimeException("columnList size is " + columnList.size() + ", but record column number is " + record.getColumnNumber()); } @@ -220,8 +224,14 @@ private long doBatchInsert(final List writerBuffer) { switch (columnType.getTypeRoot()) { case DATE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + case TIMESTAMP_WITHOUT_TIME_ZONE: try { - data.setField(i, Timestamp.fromEpochMillis(column.asLong())); + if(column.asLong()!=null) { + data.setField(i, Timestamp.fromEpochMillis(column.asLong())); + } else { + data.setField(i, null); + } } catch (Exception e) { getTaskPluginCollector().collectDirtyRecord(record, String.format("时间类型解析失败 [%s:%s] exception: %s", columnName, column.toString(), e)); } @@ -243,12 +253,20 @@ private long doBatchInsert(final List writerBuffer) { case INTEGER: case SMALLINT: case TINYINT: - data.setField(i, column.asBigInteger().intValue()); + data.setField(i, column.asBigInteger()==null?null:column.asBigInteger().intValue()); break; case FLOAT: case DOUBLE: + data.setField(i, column.asDouble()); break; + case DECIMAL: + if(column.asBigDecimal()!=null) { + data.setField(i, Decimal.fromBigDecimal(column.asBigDecimal(), ((DecimalType) columnType).getPrecision(), ((DecimalType) columnType).getScale())); + } else { + data.setField(i, null); + } + break; case MAP: try { data.setField(i, new GenericMap(JSON.parseObject(column.asString(), Map.class))); From e74ef7b1186f2c730b8e80771ace1de8cd3ecd04 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Wed, 22 Jan 2025 08:36:27 +0800 Subject: [PATCH 12/40] =?UTF-8?q?1.change=20author=202.Make=20modules=20?= =?UTF-8?q?=20ordered=20alphabetically=203.modify=20mkdocs.yml=20and=20add?= =?UTF-8?q?=20the=20guide=20into=20section=20=E5=86=99=E5=85=A5=E6=8F=92?= =?UTF-8?q?=E4=BB=B6=20ordered=20alphabetically?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- mkdocs.yml | 1 + package.xml | 16 ++++++++-------- .../paimonwriter/src/main/resources/plugin.json | 2 +- pom.xml | 2 +- 4 files changed, 11 insertions(+), 10 deletions(-) diff --git a/mkdocs.yml b/mkdocs.yml index 92d6c4a2a..18d142291 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -99,6 +99,7 @@ nav: - writer/mongodbwriter.md - writer/mysqlwriter.md - writer/oraclewriter.md + - writer/paimonwriter.md - writer/postgresqlwriter.md - writer/rdbmswriter.md - writer/rediswriter.md diff --git a/package.xml b/package.xml index b5a82b669..16ba1ce6d 100644 --- a/package.xml +++ b/package.xml @@ -521,6 +521,14 @@ 0644 addax-${project.version} + + plugin/writer/paimonwriter/target/paimonwriter-${project.version}/ + + **/*.* + + 0644 + addax-${project.version} + plugin/writer/postgresqlwriter/target/postgresqlwriter-${project.version}/ @@ -609,13 +617,5 @@ 0644 addax-${project.version} - - plugin/writer/paimonwriter/target/paimonwriter-${project.version}/ - - **/*.* - - 0644 - addax-${project.version} - diff --git a/plugin/writer/paimonwriter/src/main/resources/plugin.json b/plugin/writer/paimonwriter/src/main/resources/plugin.json index b1dd4c177..0eb2be570 100644 --- a/plugin/writer/paimonwriter/src/main/resources/plugin.json +++ b/plugin/writer/paimonwriter/src/main/resources/plugin.json @@ -2,5 +2,5 @@ "name": "paimonwriter", "class": "com.wgzhao.addax.plugin.writer.paimonwriter.PaimonWriter", "description": "write data to paimon", - "developer": "wgzhao" + "developer": "awol2005ex" } diff --git a/pom.xml b/pom.xml index 266a21b41..9acba56ae 100644 --- a/pom.xml +++ b/pom.xml @@ -384,6 +384,7 @@ plugin/writer/mongodbwriter plugin/writer/mysqlwriter plugin/writer/oraclewriter + plugin/writer/paimonwriter plugin/writer/postgresqlwriter plugin/writer/rdbmswriter plugin/writer/rediswriter @@ -395,7 +396,6 @@ plugin/writer/sybasewriter plugin/writer/tdenginewriter plugin/writer/txtfilewriter - plugin/writer/paimonwriter From 992d7a3c73f3dee26a29fb8f259501000cb67a56 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 23 Jan 2025 09:00:06 +0800 Subject: [PATCH 13/40] [add] add paimonwriter.json to docs --- docs/assets/jobs/paimonwriter.json | 48 ++++++++++++++++++++++++++++++ 1 file changed, 48 insertions(+) create mode 100644 docs/assets/jobs/paimonwriter.json diff --git a/docs/assets/jobs/paimonwriter.json b/docs/assets/jobs/paimonwriter.json new file mode 100644 index 000000000..84c97304a --- /dev/null +++ b/docs/assets/jobs/paimonwriter.json @@ -0,0 +1,48 @@ +{ + "job": { + "setting": { + "speed": { + "channel": 3 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + } + }, + "content": [ + { + "reader": { + "name": "rdbmsreader", + "parameter": { + "username": "root", + "password": "root", + "column": [ + "*" + ], + "connection": [ + { + "querySql": [ + "select 1+0 id ,'test1' as name" + ], + "jdbcUrl": ["jdbc:mysql://localhost:3306/ruoyi_vue_camunda?allowPublicKeyRetrieval=true",] + } + ], + "fetchSize": 1024 + } + }, + "writer": { + "name": "paimonwriter", + "parameter": { + "dbName": "test", + "tableName": "test2", + "writeMode": "truncate", + "paimonConfig": { + "warehouse": "file:///g:/paimon", + "metastore": "filesystem" + } + } + } + } + ] + } +} \ No newline at end of file From c0dedee176cf93b6be3944a58de50ef7936e5145 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Mon, 10 Feb 2025 16:46:07 +0800 Subject: [PATCH 14/40] fixed: timestamp time zone error --- .../wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java b/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java index ad514650b..caa11f0ad 100644 --- a/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java +++ b/plugin/writer/paimonwriter/src/main/java/com/wgzhao/addax/plugin/writer/paimonwriter/PaimonWriter.java @@ -237,7 +237,7 @@ private long doBatchInsert(final List writerBuffer) case TIMESTAMP_WITHOUT_TIME_ZONE: try { if (column.asLong() != null) { - data.setField(i, Timestamp.fromEpochMillis(column.asLong())); + data.setField(i, Timestamp.fromSQLTimestamp(column.asTimestamp())); } else { data.setField(i, null); From c017309304167dd65c335d8d459d454fb08c1169 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 13 Feb 2025 16:59:22 +0800 Subject: [PATCH 15/40] merge with master --- .../rdbms/reader/util/OriginalConfPretreatmentUtil.java | 7 +------ .../java/com/wgzhao/addax/rdbms/util/DataBaseType.java | 4 ++++ .../rdbms/writer/util/OriginalConfPretreatmentUtil.java | 5 +---- 3 files changed, 6 insertions(+), 10 deletions(-) diff --git a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/util/OriginalConfPretreatmentUtil.java b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/util/OriginalConfPretreatmentUtil.java index 9180a57cb..b087d228b 100644 --- a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/util/OriginalConfPretreatmentUtil.java +++ b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/util/OriginalConfPretreatmentUtil.java @@ -35,7 +35,6 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import static com.wgzhao.addax.common.spi.ErrorCode.CONFIG_ERROR; @@ -119,11 +118,7 @@ private static void dealJdbcAndTable(Configuration originalConfig) String driverClass = connConf.getString(Key.JDBC_DRIVER, null); if (driverClass != null && !driverClass.isEmpty()) { LOG.warn("use specified driver class: {}", driverClass); - - - Arrays.stream(DataBaseType.values()).filter( - d -> d.getDriverClassName().equals(driverClass)).findFirst().ifPresent(d -> - dataBaseType=d); + dataBaseType.setDriverClassName(driverClass); } connConf.getNecessaryValue(Key.JDBC_URL, REQUIRED_VALUE); diff --git a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java index d54438b59..1cc0b00f6 100644 --- a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java +++ b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java @@ -171,4 +171,8 @@ public String getTypeName() return typeName; } + public void setDriverClassName(String driverClassName) + { + this.driverClassName = driverClassName; + } } diff --git a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/writer/util/OriginalConfPretreatmentUtil.java b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/writer/util/OriginalConfPretreatmentUtil.java index cc953df08..0bc49d743 100644 --- a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/writer/util/OriginalConfPretreatmentUtil.java +++ b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/writer/util/OriginalConfPretreatmentUtil.java @@ -38,7 +38,6 @@ import java.sql.Connection; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import static com.wgzhao.addax.common.spi.ErrorCode.CONFIG_ERROR; @@ -93,9 +92,7 @@ public static void simplifyConf(Configuration originalConfig) String driverClass = connConf.getString(Key.JDBC_DRIVER, null); if (driverClass != null && !driverClass.isEmpty()) { LOG.warn("Use specified driver class [{}]", driverClass); - Arrays.stream(DataBaseType.values()).filter( - d -> d.getDriverClassName().equals(driverClass)).findFirst().ifPresent(d -> - dataBaseType=d); + dataBaseType.setDriverClassName(driverClass); } String jdbcUrl = connConf.getString(Key.JDBC_URL); if (StringUtils.isBlank(jdbcUrl)) { From ecc045bfb05522ee7af24a68fa6d1e3bd4633062 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Wed, 19 Feb 2025 13:08:25 +0800 Subject: [PATCH 16/40] [add][plugin][icebergwriter] Add support for writing files in Iceberg format --- plugin/writer/icebergwriter/package.xml | 37 + plugin/writer/icebergwriter/pom.xml | 871 ++++++++++++++++++ .../writer/icebergwriter/IcebergHelper.java | 102 ++ .../writer/icebergwriter/IcebergWriter.java | 317 +++++++ .../src/main/resources/plugin.json | 6 + .../main/resources/plugin_job_template.json | 17 + pom.xml | 16 + 7 files changed, 1366 insertions(+) create mode 100644 plugin/writer/icebergwriter/package.xml create mode 100644 plugin/writer/icebergwriter/pom.xml create mode 100644 plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergHelper.java create mode 100644 plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java create mode 100644 plugin/writer/icebergwriter/src/main/resources/plugin.json create mode 100644 plugin/writer/icebergwriter/src/main/resources/plugin_job_template.json diff --git a/plugin/writer/icebergwriter/package.xml b/plugin/writer/icebergwriter/package.xml new file mode 100644 index 000000000..8ecab9f9f --- /dev/null +++ b/plugin/writer/icebergwriter/package.xml @@ -0,0 +1,37 @@ + + release + + dir + + false + + + src/main/resources + + *.json + + plugin/writer/${project.artifactId} + + + target/ + + ${project.artifactId}-${project.version}.jar + + plugin/writer/${project.artifactId} + + + + + + false + plugin/writer/${project.artifactId}/libs + runtime + + com.wgzhao.addax:* + + + + diff --git a/plugin/writer/icebergwriter/pom.xml b/plugin/writer/icebergwriter/pom.xml new file mode 100644 index 000000000..f6cbe43b1 --- /dev/null +++ b/plugin/writer/icebergwriter/pom.xml @@ -0,0 +1,871 @@ + + + 4.0.0 + + + com.wgzhao.addax + addax-all + 4.2.4-SNAPSHOT + ../../../pom.xml + + + icebergwriter + iceberg-writer + Apache Iceberg writer plugin for Addax + jar + + + + com.wgzhao.addax + addax-common + ${project.version} + + + slf4j-log4j12 + org.slf4j + + + + + + com.wgzhao.addax + addax-storage + ${project.version} + + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + + org.apache.hadoop + hadoop-aws + ${hadoop.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + + + com.fasterxml.jackson.core + jackson-databind + + + commons-codec + commons-codec + + + io.netty + netty + + + org.eclipse.jetty + jetty-util + + + + + + + com.fasterxml.woodstox + woodstox-core + ${woodstox.version} + + + + org.apache.iceberg + iceberg-common + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-api + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-arrow + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-aws + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-core + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-parquet + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-orc + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-hive-metastore + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.parquet + parquet-hadoop-bundle + ${parquet.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.orc + orc-core + ${orc.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + org.apache.hadoop + hadoop-client-api + + + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + org.apache.avro + avro + ${avro.version} + + + com.fasterxml.jackson.core + jackson-databind + + + + + + + + + maven-assembly-plugin + + + package.xml + + ${project.artifactId}-${project.version} + + + + release + package + + single + + + + + + + diff --git a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergHelper.java b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergHelper.java new file mode 100644 index 000000000..e45d0db18 --- /dev/null +++ b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergHelper.java @@ -0,0 +1,102 @@ +package com.wgzhao.addax.plugin.writer.icebergwriter; + +import com.wgzhao.addax.common.exception.AddaxException; +import com.wgzhao.addax.common.util.Configuration; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hive.HiveCatalog; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +import static com.wgzhao.addax.common.base.Key.KERBEROS_KEYTAB_FILE_PATH; +import static com.wgzhao.addax.common.base.Key.KERBEROS_PRINCIPAL; +import static com.wgzhao.addax.common.spi.ErrorCode.LOGIN_ERROR; + +public class IcebergHelper { + private static final Logger LOG = LoggerFactory.getLogger(IcebergHelper.class); + + public static void kerberosAuthentication(org.apache.hadoop.conf.Configuration hadoopConf, String kerberosPrincipal, String kerberosKeytabFilePath) throws Exception { + if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) { + UserGroupInformation.setConfiguration(hadoopConf); + try { + UserGroupInformation.loginUserFromKeytab(kerberosPrincipal, kerberosKeytabFilePath); + } catch (Exception e) { + String message = String.format("kerberos authentication failed, keytab file: [%s], principal: [%s]", + kerberosKeytabFilePath, kerberosPrincipal); + LOG.error(message); + throw AddaxException.asAddaxException(LOGIN_ERROR, e); + } + } + } + + public static Catalog getCatalog(Configuration conf) throws Exception { + + String catalogType = conf.getString("catalogType"); + if (catalogType == null || catalogType.trim().isEmpty()) { + throw new RuntimeException("catalogType is not set"); + } + catalogType = catalogType.trim(); + + String warehouse = conf.getString("warehouse"); + if (warehouse == null || warehouse.trim().isEmpty()) { + throw new RuntimeException("warehouse is not set"); + } + + org.apache.hadoop.conf.Configuration hadoopConf = null; + + if (conf.getConfiguration("hadoopConfig") != null) { + Map hadoopConfig = conf.getMap("hadoopConfig"); + + hadoopConf = new org.apache.hadoop.conf.Configuration(); + + for (String key : hadoopConfig.keySet()) { + hadoopConf.set(key, (String)hadoopConfig.get(key)); + } + + + String authentication = (String)hadoopConfig.get("hadoop.security.authentication"); + + if ("kerberos".equals(authentication)) { + String kerberosKeytabFilePath = conf.getString(KERBEROS_KEYTAB_FILE_PATH); + if(kerberosKeytabFilePath ==null || kerberosKeytabFilePath.trim().isEmpty()){ + throw new RuntimeException("kerberosKeytabFilePath is not set"); + } else { + kerberosKeytabFilePath = kerberosKeytabFilePath.trim(); + } + + String kerberosPrincipal = conf.getString(KERBEROS_PRINCIPAL); + if(kerberosPrincipal ==null || kerberosPrincipal.trim().isEmpty()){ + throw new RuntimeException("kerberosPrincipal is not set"); + } else { + kerberosPrincipal = kerberosPrincipal.trim(); + } + IcebergHelper.kerberosAuthentication(hadoopConf, kerberosPrincipal, kerberosKeytabFilePath); + } + } + switch (catalogType) { + case "hadoop": + return new HadoopCatalog(hadoopConf, warehouse); + case "hive": + String uri = conf.getString("uri"); + if (uri == null || uri.trim().isEmpty()) { + throw new RuntimeException("uri is not set"); + } + HiveCatalog hiveCatalog = new HiveCatalog(); + hiveCatalog.setConf(hadoopConf); + Map properties = new HashMap(); + properties.put("warehouse", warehouse); + properties.put("uri", uri); + + hiveCatalog.initialize("hive", properties); + return hiveCatalog; + } + + throw new RuntimeException("not support catalogType:" + catalogType); + } + +} diff --git a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java new file mode 100644 index 000000000..820e1f219 --- /dev/null +++ b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java @@ -0,0 +1,317 @@ +package com.wgzhao.addax.plugin.writer.icebergwriter; + +import com.alibaba.fastjson2.JSON; +import com.google.common.collect.ImmutableList; +import com.wgzhao.addax.common.element.Column; +import com.wgzhao.addax.common.element.Record; +import com.wgzhao.addax.common.plugin.RecordReceiver; +import com.wgzhao.addax.common.spi.Writer; +import com.wgzhao.addax.common.util.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; + + +public class IcebergWriter extends Writer { + public static class Job + extends Writer.Job { + private static final Logger LOG = LoggerFactory.getLogger(Job.class); + private Configuration conf = null; + private Catalog catalog = null; + private String tableName = null; + + @Override + public void init() { + this.conf = this.getPluginJobConf(); + try { + this.catalog = IcebergHelper.getCatalog(conf); + } catch (Exception e) { + throw new RuntimeException(e); + } + + tableName = this.conf.getString("tableName"); + if (tableName == null || tableName.trim().isEmpty()) { + throw new RuntimeException("tableName is not set"); + } + + + } + + @Override + public List split(int mandatoryNumber) { + List configurations = new ArrayList<>(mandatoryNumber); + for (int i = 0; i < mandatoryNumber; i++) { + configurations.add(conf); + } + return configurations; + } + + @Override + public void prepare() { + String writeMode = this.conf.getString("writeMode"); + if ("truncate".equalsIgnoreCase(writeMode)) { + Table table = catalog.loadTable(TableIdentifier.of(tableName.split("\\."))); + table.newDelete().deleteFromRowFilter(org.apache.iceberg.expressions.Expressions.alwaysTrue()).commit(); + } + } + + @Override + public void destroy() { + if (this.catalog != null) { + try { + if (this.catalog instanceof HiveCatalog) { + ((HiveCatalog) this.catalog).close(); + } + if (this.catalog instanceof HadoopCatalog) { + ((HadoopCatalog) this.catalog).close(); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + } + } + + public static class Task + extends Writer.Task { + + private static final Logger log = LoggerFactory.getLogger(Task.class); + private Catalog catalog = null; + private Integer batchSize = 1000; + private Table table = null; + private org.apache.iceberg.Schema schema = null; + private String fileFormat = "parquet"; + private List columnList = null; + + @Override + public void startWrite(RecordReceiver recordReceiver) { + + List writerBuffer = new ArrayList<>(this.batchSize); + Record record; + long total = 0; + while ((record = recordReceiver.getFromReader()) != null) { + writerBuffer.add(record); + if (writerBuffer.size() >= this.batchSize) { + total += doBatchInsert(writerBuffer); + writerBuffer.clear(); + } + } + + if (!writerBuffer.isEmpty()) { + total += doBatchInsert(writerBuffer); + writerBuffer.clear(); + } + + String msg = String.format("task end, write size :%d", total); + getTaskPluginCollector().collectMessage("writeSize", String.valueOf(total)); + log.info(msg); + } + + @Override + public void init() { + Configuration conf = super.getPluginJobConf(); + + batchSize = conf.getInt("batchSize", 1000); + + try { + this.catalog = IcebergHelper.getCatalog(conf); + } catch (Exception e) { + throw new RuntimeException(e); + } + + String tableName = conf.getString("tableName"); + if (tableName == null || tableName.trim().isEmpty()) { + throw new RuntimeException("tableName is not set"); + } + + table = catalog.loadTable(TableIdentifier.of(tableName.split("\\."))); + schema = table.schema(); + + fileFormat = table.properties().get("write.format.default"); + if (fileFormat == null || fileFormat.trim().isEmpty()) { + fileFormat = "parquet"; + } + + + columnList = schema.columns(); + } + + @Override + public void destroy() { + if (this.catalog != null) { + try { + if (this.catalog instanceof HiveCatalog) { + ((HiveCatalog) this.catalog).close(); + } + if (this.catalog instanceof HadoopCatalog) { + ((HadoopCatalog) this.catalog).close(); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + } + + private long doBatchInsert(final List writerBuffer) { + ImmutableList.Builder builder = ImmutableList.builder(); + + + for (Record record : writerBuffer) { + GenericRecord data = GenericRecord.create(schema); + for (int i = 0; i < record.getColumnNumber(); i++) { + Column column = record.getColumn(i); + if (column == null) { + continue; + } + if (i >= columnList.size()) { + throw new RuntimeException("columnList size is " + columnList.size() + ", but record column number is " + record.getColumnNumber()); + } + Types.NestedField field = columnList.get(i); + org.apache.iceberg.types.Type columnType = field.type(); + //如果是数组类型,那它传入的必是字符串类型 + if (columnType.isListType()) { + if (null == column.asString()) { + data.setField(field.name(), null); + } else { + String[] dataList = column.asString().split(","); + data.setField(field.name(), dataList); + } + } else { + switch (columnType.typeId()) { + + case DATE: + try { + if (column.asLong() != null) { + data.setField(field.name(), column.asTimestamp().toLocalDateTime()); + } else { + data.setField(field.name(), null); + } + } catch (Exception e) { + getTaskPluginCollector().collectDirtyRecord(record, String.format("日期类型解析失败 [%s:%s] exception: %s", field.name(), column, e)); + } + break; + case TIME: + case TIMESTAMP: + case TIMESTAMP_NANO: + try { + if (column.asLong() != null) { + data.setField(field.name(), column.asTimestamp().toLocalDateTime()); + } else { + data.setField(field.name(), null); + } + } catch (Exception e) { + getTaskPluginCollector().collectDirtyRecord(record, String.format("时间类型解析失败 [%s:%s] exception: %s", field.name(), column, e)); + } + break; + case STRING: + data.setField(field.name(), column.asString()); + break; + case BOOLEAN: + data.setField(field.name(), column.asBoolean()); + break; + case FIXED: + case BINARY: + data.setField(field.name(), column.asBytes()); + break; + case LONG: + data.setField(field.name(), column.asLong()); + break; + case INTEGER: + data.setField(field.name(), column.asBigInteger() == null ? null : column.asBigInteger().intValue()); + break; + case FLOAT: + data.setField(field.name(), column.asDouble().floatValue()); + break; + case DOUBLE: + + data.setField(field.name(), column.asDouble()); + break; + case DECIMAL: + if (column.asBigDecimal() != null) { + data.setField(field.name(), column.asBigDecimal()); + } else { + data.setField(field.name(), null); + } + break; + case MAP: + try { + data.setField(field.name(), JSON.parseObject(column.asString(), Map.class)); + } catch (Exception e) { + getTaskPluginCollector().collectDirtyRecord(record, String.format("MAP类型解析失败 [%s:%s] exception: %s", field.name(), column, e)); + } + break; + case VARIANT: + try { + data.setField(field.name(), JSON.parseObject(column.asString(), Map.class)); + } catch (Exception e) { + getTaskPluginCollector().collectDirtyRecord(record, String.format("VARIANT类型解析失败 [%s:%s] exception: %s", field.name(), column, e)); + } + break; + default: + getTaskPluginCollector().collectDirtyRecord(record, "类型错误:不支持的类型:" + columnType + " " + field.name()); + } + } + + } + + + builder.add(data); + } + + String filepath = table.location() + "/" + UUID.randomUUID(); + OutputFile file = table.io().newOutputFile(filepath); + + DataWriter dataWriter = null; + + if ("parquet".equals(fileFormat)) { + try { + dataWriter = Parquet.writeData(file).overwrite().forTable(table).createWriterFunc(GenericParquetWriter::buildWriter).build(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } else if ("orc".equals(fileFormat)) { + dataWriter = ORC.writeData(file).overwrite().forTable(table).createWriterFunc(GenericOrcWriter::buildWriter).build(); + } else { + throw new RuntimeException("不支持的文件格式:" + fileFormat); + } + ImmutableList rows = builder.build(); + + if (dataWriter != null) { + dataWriter.write(rows); + } + + + if (dataWriter != null) { + try { + dataWriter.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + DataFile dataFile = dataWriter.toDataFile(); + table.newAppend().appendFile(dataFile).commit(); + return rows.size(); + } + } +} diff --git a/plugin/writer/icebergwriter/src/main/resources/plugin.json b/plugin/writer/icebergwriter/src/main/resources/plugin.json new file mode 100644 index 000000000..894f4e064 --- /dev/null +++ b/plugin/writer/icebergwriter/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "icebergwriter", + "class": "com.wgzhao.addax.plugin.writer.icebergwriter.IcebergWriter", + "description": "write data to iceberg", + "developer": "awol2005ex" +} diff --git a/plugin/writer/icebergwriter/src/main/resources/plugin_job_template.json b/plugin/writer/icebergwriter/src/main/resources/plugin_job_template.json new file mode 100644 index 000000000..831992aff --- /dev/null +++ b/plugin/writer/icebergwriter/src/main/resources/plugin_job_template.json @@ -0,0 +1,17 @@ +{ + "name": "icebergwriter", + "parameter": { + "tableName": "test.test1", + "catalogType" : "hadoop", + "writeMode": "truncate", + "warehouse": "s3a://pvc-91d1e2cd-4d25-45c9-8613-6c4f7bf0a4cc/iceberg", + "hadoopConfig": { + "fs.s3a.endpoint":"http://localhost:9000", + "fs.s3a.access.key":"gy0dX5lALP176g6c9fYf", + "fs.s3a.secret.key":"ReuUrCzzu5wKWAegtswoHIWV389BYl9AB1ZQbiKr", + "fs.s3a.connection.ssl.enabled":"false", + "fs.s3a.path.style.access":"true", + "fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem" + } + } +} diff --git a/pom.xml b/pom.xml index fd58aa49d..89b42906a 100644 --- a/pom.xml +++ b/pom.xml @@ -439,5 +439,21 @@ plugin/writer/txtfilewriter + + + iceberg + + + 1.8.0 + 11 + 11 + 11 + + + + + plugin/writer/icebergwriter + + From 8ac19e4e106cb3287f47a8515a69a03db5d2e87b Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Wed, 19 Feb 2025 14:01:13 +0800 Subject: [PATCH 17/40] [add][plugin][icebergwriter] fixed date type insert error --- .../writer/icebergwriter/IcebergWriter.java | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java index 820e1f219..f0731e2dd 100644 --- a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java +++ b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java @@ -25,10 +25,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.UUID; +import java.util.*; public class IcebergWriter extends Writer { @@ -195,7 +192,7 @@ private long doBatchInsert(final List writerBuffer) { data.setField(field.name(), null); } else { String[] dataList = column.asString().split(","); - data.setField(field.name(), dataList); + data.setField(field.name(), Arrays.asList(dataList)); } } else { switch (columnType.typeId()) { @@ -203,7 +200,7 @@ private long doBatchInsert(final List writerBuffer) { case DATE: try { if (column.asLong() != null) { - data.setField(field.name(), column.asTimestamp().toLocalDateTime()); + data.setField(field.name(), column.asTimestamp().toLocalDateTime().toLocalDate()); } else { data.setField(field.name(), null); } @@ -261,13 +258,6 @@ private long doBatchInsert(final List writerBuffer) { getTaskPluginCollector().collectDirtyRecord(record, String.format("MAP类型解析失败 [%s:%s] exception: %s", field.name(), column, e)); } break; - case VARIANT: - try { - data.setField(field.name(), JSON.parseObject(column.asString(), Map.class)); - } catch (Exception e) { - getTaskPluginCollector().collectDirtyRecord(record, String.format("VARIANT类型解析失败 [%s:%s] exception: %s", field.name(), column, e)); - } - break; default: getTaskPluginCollector().collectDirtyRecord(record, "类型错误:不支持的类型:" + columnType + " " + field.name()); } From b8d8aa0f875bf9a46b6575d40037b0f94e36a5df Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 20 Feb 2025 11:08:00 +0800 Subject: [PATCH 18/40] [add][plugin][icebergwriter] package.xml add fileSet --- package.xml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/package.xml b/package.xml index 16ba1ce6d..c4e67a5f8 100644 --- a/package.xml +++ b/package.xml @@ -453,6 +453,17 @@ 0644 addax-${project.version} + + plugin/writer/icebergwriter/target/icebergwriter-${project.version}/ + + **/*.* + + + **/addax-common*.jar + + 0644 + addax-${project.version} + plugin/writer/influxdbwriter/target/influxdbwriter-${project.version}/ From 0accdef86e91ada518811f792ae75bcb7262e390 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 20 Feb 2025 13:20:22 +0800 Subject: [PATCH 19/40] [add][plugin][icebergwriter] add docs --- docs/assets/jobs/icebergwriter.json | 56 +++ docs/writer/icebgerwriter.md | 611 ++++++++++++++++++++++++++++ 2 files changed, 667 insertions(+) create mode 100644 docs/assets/jobs/icebergwriter.json create mode 100644 docs/writer/icebgerwriter.md diff --git a/docs/assets/jobs/icebergwriter.json b/docs/assets/jobs/icebergwriter.json new file mode 100644 index 000000000..fedcebf72 --- /dev/null +++ b/docs/assets/jobs/icebergwriter.json @@ -0,0 +1,56 @@ +{ + "job": { + "setting": { + "speed": { + "channel": 3 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + } + }, + "content": [ + { + "reader": { + "name": "rdbmsreader", + "parameter": { + "username": "root", + "password": "root", + "column": [ + "*" + ], + "connection": [ + { + "querySql": [ + "select 1+0 id ,now() ts1,'test1' as name" + ], + "jdbcUrl": [ + "jdbc:mysql://localhost:3306/ruoyi_vue_camunda?allowPublicKeyRetrieval=true" + ] + } + ], + "fetchSize": 1024 + } + }, + "writer": { + "name": "icebergwriter", + "parameter": { + "tableName": "test.test1", + "writeMode": "truncate", + "catalogType":"hadoop", + "warehouse": "s3a://pvc-91d1e2cd-4d25-45c9-8613-6c4f7bf0a4cc/iceberg", + "hadoopConfig": { + + "fs.s3a.endpoint": "http://localhost:9000", + "fs.s3a.access.key": "gy0dX5lALP176g6c9fYf", + "fs.s3a.secret.key": "ReuUrCzzu5wKWAegtswoHIWV389BYl9AB1ZQbiKr", + "fs.s3a.connection.ssl.enabled": "false", + "fs.s3a.path.style.access": "true", + "fs.s3a.impl": "org.apache.hadoop.fs.s3a.S3AFileSystem" + } + } + } + } + ] + } +} \ No newline at end of file diff --git a/docs/writer/icebgerwriter.md b/docs/writer/icebgerwriter.md new file mode 100644 index 000000000..d8801ae1e --- /dev/null +++ b/docs/writer/icebgerwriter.md @@ -0,0 +1,611 @@ +# Paimon Writer + +Paimon Writer 提供向 已有的paimon表写入数据的能力。 + +## 配置样例 + +```json +--8<-- "jobs/icebergwriter.json" +``` + +## 参数说明 + +| 配置项 | 是否必须 | 数据类型 | 默认值 | 说明 | +|:-------------|:----:|--------|----|-------------------------------------------------| +| tableName | 是 | string | 无 | 要写入的iceberg表名 | +| catalogType | 是 | string | 无 | catalog类型, 目前支持 hive,hadoop | +| warehouse | 是 | string | 无 | 仓库地址 | +| writeMode | 是 | string | 无 | 写入模式,详述见下 | +| hadoopConfig | 是 | json | {} | 里可以配置与 Iceberg catalog和Hadoop 相关的一些高级参数,比如HA的配置 | + + + +### writeMode + +写入前数据清理处理模式: + +- append,写入前不做任何处理,直接写入,不清除原来的数据。 +- truncate 写入前先清空表,再写入。 + +### hadoopConfig + +`hadoopConfig` 里可以配置与 Iceberg catalog和Hadoop 相关的一些高级参数,比如HA的配置 + +创建表实例: + +依赖包设置: + +build.gradle + +```groovy +plugins { + id 'java' +} + +group = 'com.awol2005ex' +version = '1.0-SNAPSHOT' +ext["hadoop_version"] = "3.2.4" +ext["hive_version"] = "3.1.3" +ext["woodstox_version"] = "7.0.0" +ext["iceberg_version"] = "1.8.0" +repositories { + maven { url "https://maven.aliyun.com/repository/central" } + maven { url "https://maven.aliyun.com/repository/public" } + maven { + url 'https://repo.huaweicloud.com/repository/maven/' + } + maven { + url 'https://repo.spring.io/libs-milestone/' + } + + + maven { + url 'https://repo.spring.io/libs-snapshot' + } + mavenCentral() +} + +dependencies { + testImplementation platform('org.junit:junit-bom:5.10.0') + testImplementation 'org.junit.jupiter:junit-jupiter' + implementation("org.apache.hadoop:hadoop-common:${hadoop_version}") { + exclude group: 'com.fasterxml.jackson.core', module: 'jackson-databind' + exclude group: 'org.codehaus.jackson', module: 'jackson-core-asl' + exclude group: 'org.codehaus.jackson', module: 'jackson-mapper-asl' + exclude group: 'com.fasterxml.woodstox', module: 'woodstox-core' + exclude group: 'commons-codec', module: 'commons-codec' + exclude group: 'commons-net', module: 'commons-net' + exclude group: 'io.netty', module: 'netty' + exclude group: 'log4j', module: 'log4j' + exclude group: 'net.minidev', module: 'json-smart' + exclude group: 'org.codehaus.jettison', module: 'jettison' + exclude group: 'org.eclipse.jetty', module: 'jetty-server' + exclude group: 'org.xerial.snappy', module: 'snappy-java' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + exclude group: 'org.eclipse.jetty', module: 'jetty-util' + } + implementation("org.apache.hadoop:hadoop-aws:${hadoop_version}") { + exclude group: 'com.fasterxml.jackson.core', module: 'jackson-databind' + exclude group: 'org.codehaus.jackson', module: 'jackson-core-asl' + exclude group: 'org.codehaus.jackson', module: 'jackson-mapper-asl' + exclude group: 'com.fasterxml.woodstox', module: 'woodstox-core' + exclude group: 'commons-codec', module: 'commons-codec' + exclude group: 'commons-net', module: 'commons-net' + exclude group: 'io.netty', module: 'netty' + exclude group: 'log4j', module: 'log4j' + exclude group: 'net.minidev', module: 'json-smart' + exclude group: 'org.codehaus.jettison', module: 'jettison' + exclude group: 'org.eclipse.jetty', module: 'jetty-server' + exclude group: 'org.xerial.snappy', module: 'snappy-java' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + exclude group: 'org.eclipse.jetty', module: 'jetty-util' + } + implementation("org.apache.hadoop:hadoop-mapreduce-client-core:${hadoop_version}") { + exclude group: 'com.fasterxml.jackson.core', module: 'jackson-databind' + exclude group: 'org.codehaus.jackson', module: 'jackson-core-asl' + exclude group: 'org.codehaus.jackson', module: 'jackson-mapper-asl' + exclude group: 'com.fasterxml.woodstox', module: 'woodstox-core' + exclude group: 'commons-codec', module: 'commons-codec' + exclude group: 'commons-net', module: 'commons-net' + exclude group: 'io.netty', module: 'netty' + exclude group: 'log4j', module: 'log4j' + exclude group: 'net.minidev', module: 'json-smart' + exclude group: 'org.codehaus.jettison', module: 'jettison' + exclude group: 'org.eclipse.jetty', module: 'jetty-server' + exclude group: 'org.xerial.snappy', module: 'snappy-java' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + exclude group: 'org.eclipse.jetty', module: 'jetty-util' + } + implementation("org.apache.hive:hive-metastore:${hive_version}"){ + exclude group: 'com.fasterxml.jackson.core', module: 'jackson-databind' + exclude group: 'org.codehaus.jackson', module: 'jackson-core-asl' + exclude group: 'org.codehaus.jackson', module: 'jackson-mapper-asl' + exclude group: 'com.fasterxml.woodstox', module: 'woodstox-core' + exclude group: 'commons-codec', module: 'commons-codec' + exclude group: 'commons-net', module: 'commons-net' + exclude group: 'io.netty', module: 'netty' + exclude group: 'log4j', module: 'log4j' + exclude group: 'net.minidev', module: 'json-smart' + exclude group: 'org.codehaus.jettison', module: 'jettison' + exclude group: 'org.eclipse.jetty', module: 'jetty-server' + exclude group: 'org.xerial.snappy', module: 'snappy-java' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + exclude group: 'org.eclipse.jetty', module: 'jetty-util' + } + implementation("com.fasterxml.woodstox:woodstox-core:${woodstox_version}") + + implementation("org.apache.iceberg:iceberg-common:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-api:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-arrow:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-aws:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-core:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-parquet:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-orc:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-hive-metastore:${iceberg_version}") +} + +test { + useJUnitPlatform() +} +``` + +创建存储在minio,catalogType是hadoop的iceberg表 + +```java +package com.test; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; + +import java.io.IOException; + +public class CreateMinioTable { + public static void main(String[] args) throws IOException { + + Configuration hadoopConf = new Configuration(); + "fs.s3a.endpoint", "http://localhost:9000"); + "fs.s3a.access.key", "gy0dX5lALP176g6c9fYf"); + "fs.s3a.secret.key", "ReuUrCzzu5wKWAegtswoHIWV389BYl9AB1ZQbiKr"); + "fs.s3a.connection.ssl.enabled", "false"); + "fs.s3a.path.style.access", "true"); + "fs.s3a.impl","org.apache.hadoop.fs.s3a.S3AFileSystem"); + String warehousePath = "s3a://pvc-91d1e2cd-4d25-45c9-8613-6c4f7bf0a4cc/iceberg"; + HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehousePath); + + TableIdentifier name = TableIdentifier.of("test", "test1"); + + Schema schema = new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "ts1", Types.TimestampType.withoutZone()), + Types.NestedField.required(3, "name", Types.StringType.get()) + ); + Table table = catalog.createTable(name, schema); + System.out.println(table.location()); + + catalog.close(); + } +} + +``` + +创建存储在hdfs,catalogType是hadoop的iceberg表 +```java +package com.test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.types.Types; + +import java.io.IOException; + +public class CreateHdfsTable { + public static void main(String[] args) throws IOException { + + System.setProperty("java.security.krb5.conf","D:/MIT/krb5.ini"); + + Configuration hadoopConf = new Configuration(); + "fs.defaultFS", "hdfs://nameservice1"); + "hadoop.security.authentication", "kerberos"); + "hadoop.kerberos.principal", "hive/_HOST@XXX.COM"); + "hadoop.kerberos.keytab", "/tmp/hive@XXX.COM.keytab"); + "ha.zookeeper.quorum", "nn1:2181,nn2:2181,nn3:2181"); + "dfs.nameservices", "nameservice1"); + "dfs.namenode.rpc-address.nameservice1.namenode371", "nn2:8020"); + "dfs.namenode.rpc-address.nameservice1.namenode265", "nn1:8020"); + "dfs.namenode.keytab.file", "/tmp/hdfs@XXX.COM.keytab"); + "dfs.namenode.keytab.enabled", "true"); + "dfs.namenode.kerberos.principal", "hdfs/_HOST@XXX.COM"); + "dfs.namenode.kerberos.internal.spnego.principal", "HTTP/_HOST@XXX.COM"); + "dfs.ha.namenodes.nameservice1", "namenode265,namenode371"); + "dfs.datanode.keytab.file", "/tmp/hdfs@XXX.COM.keytab"); + "dfs.datanode.keytab.enabled", "true"); + "dfs.datanode.kerberos.principal", "hdfs/_HOST@XXX.COM"); + "dfs.client.use.datanode.hostname", "false"); + "dfs.client.failover.proxy.provider.nameservice1", "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider"); + "dfs.balancer.keytab.file", "/tmp/hdfs@XXX.COM.keytab"); + "dfs.balancer.keytab.enabled", "true"); + "dfs.balancer.kerberos.principal", "hdfs/_HOST@XXX.COM"); + + UserGroupInformation.setConfiguration(hadoopConf); + try { + UserGroupInformation.loginUserFromKeytab("hive@XXX.COM", "/tmp/hive@XXX.COM.keytab"); + } catch (Exception e) { + e.printStackTrace(); + } + + + + String warehousePath = "hdfs://nameservice1/user/hive/iceberg"; + HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehousePath); + + TableIdentifier name = TableIdentifier.of("test1", "test20250219"); + + Schema schema = new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "ts1", Types.TimestampType.withoutZone()), + Types.NestedField.required(3, "dec1", Types.DecimalType.of(12,2)), + Types.NestedField.required(4, "bool1", Types.BooleanType.get()), + Types.NestedField.required(5, "map1", Types.MapType.ofRequired(11,12,Types.StringType.get(),Types.StringType.get())), + Types.NestedField.required(6, "date1", Types.DateType.get()), + Types.NestedField.required(7, "float1", Types.FloatType.get()), + Types.NestedField.required(8, "double1", Types.DoubleType.get()), + Types.NestedField.required(9, "array1", Types.ListType.ofRequired(13,Types.StringType.get())), + Types.NestedField.required(10, "name", Types.StringType.get()) + ); + catalog.dropTable(name,true); + Table table = catalog.createTable(name, schema); + System.out.println(table.location()); + + catalog.close(); + } +} +``` +创建hive表 + +```java +package com.test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.types.Types; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class CreateHiveTable { + public static void main(String[] args) throws IOException { + + System.setProperty("java.security.krb5.conf","D:/MIT/krb5.ini"); + + Configuration hadoopConf = new Configuration(); + "fs.defaultFS", "hdfs,//nameservice1"); + "hadoop.security.authentication", "kerberos"); + "hadoop.kerberos.principal", "hive/_HOST@XXX.COM"); + "hadoop.kerberos.keytab", "/tmp/hive@XXX.COM.keytab"); + "ha.zookeeper.quorum", "nn1:2181,nn2:2181,nn3:2181"); + "dfs.nameservices", "nameservice1"); + "dfs.namenode.rpc-address.nameservice1.namenode371", "nn2:8020"); + "dfs.namenode.rpc-address.nameservice1.namenode265", "nn1:8020"); + "dfs.namenode.keytab.file", "/tmp/hdfs@XXX.COM.keytab"); + "dfs.namenode.keytab.enabled", "true"); + "dfs.namenode.kerberos.principal", "hdfs/_HOST@XXX.COM"); + "dfs.namenode.kerberos.internal.spnego.principal", "HTTP/_HOST@XXX.COM"); + "dfs.ha.namenodes.nameservice1", "namenode265,namenode371"); + "dfs.datanode.keytab.file", "/tmp/hdfs@XXX.COM.keytab"); + "dfs.datanode.keytab.enabled", "true"); + "dfs.datanode.kerberos.principal", "hdfs/_HOST@XXX.COM"); + "dfs.client.use.datanode.hostname", "false"); + "dfs.client.failover.proxy.provider.nameservice1", "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider"); + "dfs.balancer.keytab.file", "/tmp/hdfs@XXX.COM.keytab"); + "dfs.balancer.keytab.enabled", "true"); + "dfs.balancer.kerberos.principal", "hdfs/_HOST@XXX.COM"); + + "hive.metastore.uris", "thrift://nn1:9083,thrift://nn2:9083"); + "hive.server2.authentication","kerberos"); + "hive.metastore.kerberos.principal","hive/_HOST@XXX.COM"); + + "hive.metastore.sasl.enabled", "true"); + + UserGroupInformation.setConfiguration(hadoopConf); + try { + UserGroupInformation.loginUserFromKeytab("hive@XXX.COM", "/tmp/hive@XXX.COM.keytab"); + } catch (Exception e) { + e.printStackTrace();; + } + + + + + HiveCatalog catalog = new HiveCatalog(); + catalog.setConf(hadoopConf); + Map properties = new HashMap(); + properties.put("warehouse", "/warehouse/tablespace/managed/hive"); + properties.put("uri", "thrift://nn1:9083,thrift://nn2:9083"); + + catalog.initialize("hive", properties); + + TableIdentifier name = TableIdentifier.of("test1", "test20250218"); + + Schema schema = new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "ts1", Types.TimestampType.withoutZone()), + Types.NestedField.required(3, "name", Types.StringType.get()) + ); + Table table = catalog.createTable(name, schema); + System.out.println(table.location()); + + catalog.close(); + } +} + + +``` + + + +Spark 或者 flink 环境创建表 + +```sql +CREATE TABLE if not exists test1.test1_iceberg1 USING ICEBERG + TBLPROPERTIES( + 'format-version'='2', + 'write.metadata.delete-after-commit.enabled'=true, + 'write.metadata.previous-versions-max'=1, + 'target-file-size-bytes'=268435456 + ) + as select * from test1.test1 limit 0; + +``` + +s3 或者 minio hadoop catalog例子 +```json +{ + "job": { + "setting": { + "speed": { + "channel": 3 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + } + }, + "content": [ + { + "reader": { + "name": "rdbmsreader", + "parameter": { + "username": "root", + "password": "root", + "column": [ + "*" + ], + "connection": [ + { + "querySql": [ + "select 1+0 id ,now() ts1,'test1' as name" + ], + "jdbcUrl": [ + "jdbc:mysql://localhost:3306/ruoyi_vue_camunda?allowPublicKeyRetrieval=true" + ] + } + ], + "fetchSize": 1024 + } + }, + "writer": { + "name": "icebergwriter", + "parameter": { + "tableName": "test.test1", + "writeMode": "truncate", + "catalogType":"hadoop", + "warehouse": "s3a://pvc-91d1e2cd-4d25-45c9-8613-6c4f7bf0a4cc/iceberg", + "hadoopConfig": { + + "fs.s3a.endpoint": "http://localhost:9000", + "fs.s3a.access.key": "gy0dX5lALP176g6c9fYf", + "fs.s3a.secret.key": "ReuUrCzzu5wKWAegtswoHIWV389BYl9AB1ZQbiKr", + "fs.s3a.connection.ssl.enabled": "false", + "fs.s3a.path.style.access": "true", + "fs.s3a.impl": "org.apache.hadoop.fs.s3a.S3AFileSystem" + } + } + } + } + ] + } +} +``` + + +hdfs hadoop catalog例子 + +```json +{ + "job": { + "setting": { + "speed": { + "channel": 3 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + } + }, + "content": [ + { + "reader": { + "name": "rdbmsreader", + "parameter": { + "username": "root", + "password": "root", + "column": [ + "*" + ], + "connection": [ + { + "querySql": [ + "select 1+0 id ,now() ts1,CAST(1.2 AS DECIMAL(12,2)) dec1,true bool1,'{\"a\":\"1\"}' map1,now() date1,1.3 float1,1.4 double1,'a,b,c' array1,'test1' as name" + ], + "jdbcUrl": [ + "jdbc:mysql://localhost:3306/ruoyi_vue_camunda?allowPublicKeyRetrieval=true" + ] + } + ], + "fetchSize": 1024 + } + }, + "writer": { + "name": "icebergwriter", + "parameter": { + "tableName": "test1.test20250219", + "writeMode": "truncate", + "catalogType": "hadoop", + "warehouse": "hdfs://nameservice1/user/hive/iceberg", + "kerberosKeytabFilePath":"/tmp/hive@XXX.COM.keytab", + "kerberosPrincipal":"hive@XXX.COM", + "hadoopConfig": { + "fs.defaultFS": "hdfs://nameservice1", + "hadoop.security.authentication": "kerberos", + "hadoop.kerberos.principal": "hive/_HOST@XXX.COM", + "hadoop.kerberos.keytab": "/tmp/hive@XXX.COM.keytab", + "ha.zookeeper.quorum": "nn1:2181,nn2:2181,nn3:2181", + "dfs.nameservices": "nameservice1", + "dfs.namenode.rpc-address.nameservice1.namenode371": "nn2:8020", + "dfs.namenode.rpc-address.nameservice1.namenode265": "nn1:8020", + "dfs.namenode.keytab.file": "/tmp/hdfs@XXX.COM.keytab", + "dfs.namenode.keytab.enabled": "true", + "dfs.namenode.kerberos.principal": "hdfs/_HOST@XXX.COM", + "dfs.namenode.kerberos.internal.spnego.principal": "HTTP/_HOST@XXX.COM", + "dfs.ha.namenodes.nameservice1": "namenode265,namenode371", + "dfs.datanode.keytab.file": "/tmp/hdfs@XXX.COM.keytab", + "dfs.datanode.keytab.enabled": "true", + "dfs.datanode.kerberos.principal": "hdfs/_HOST@XXX.COM", + "dfs.client.use.datanode.hostname": "false", + "dfs.client.failover.proxy.provider.nameservice1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.balancer.keytab.file": "/tmp/hdfs@XXX.COM.keytab", + "dfs.balancer.keytab.enabled": "true", + "dfs.balancer.kerberos.principal": "hdfs/_HOST@XXX.COM" + } + } + } + } + ] + } +} +``` + + +hive catalog例子 + +```json +{ + "job": { + "setting": { + "speed": { + "channel": 3 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + } + }, + "content": [ + { + "reader": { + "name": "rdbmsreader", + "parameter": { + "username": "root", + "password": "root", + "column": [ + "*" + ], + "connection": [ + { + "querySql": [ + "select 1+0 id ,now() ts1,CAST(1.2 AS DECIMAL(12,2)) dec1,true bool1,'{\"a\":\"1\"}' map1,now() date1,1.3 float1,1.4 double1,'a,b,c' array1,'test1' as name" + ], + "jdbcUrl": [ + "jdbc:mysql://localhost:3306/ruoyi_vue_camunda?allowPublicKeyRetrieval=true" + ] + } + ], + "fetchSize": 1024 + } + }, + "writer": { + "name": "icebergwriter", + "parameter": { + "tableName": "test1.test20250219", + "writeMode": "truncate", + "catalogType": "hive", + "uri": "thrift://nn1:9083,thrift://nn2:9083", + "warehouse": "/warehouse/tablespace/managed/hive", + "kerberosKeytabFilePath":"/tmp/hive@XXX.COM.keytab", + "kerberosPrincipal":"hive@XXX.COM", + "hadoopConfig": { + "fs.defaultFS": "hdfs://nameservice1", + "hadoop.security.authentication": "kerberos", + "hadoop.kerberos.principal": "hive/_HOST@XXX.COM", + "hadoop.kerberos.keytab": "/tmp/hive@XXX.COM.keytab", + "ha.zookeeper.quorum": "nn1:2181,nn2:2181,nn3:2181", + "dfs.nameservices": "nameservice1", + "dfs.namenode.rpc-address.nameservice1.namenode371": "nn2:8020", + "dfs.namenode.rpc-address.nameservice1.namenode265": "nn1:8020", + "dfs.namenode.keytab.file": "/tmp/hdfs@XXX.COM.keytab", + "dfs.namenode.keytab.enabled": "true", + "dfs.namenode.kerberos.principal": "hdfs/_HOST@XXX.COM", + "dfs.namenode.kerberos.internal.spnego.principal": "HTTP/_HOST@XXX.COM", + "dfs.ha.namenodes.nameservice1": "namenode265,namenode371", + "dfs.datanode.keytab.file": "/tmp/hdfs@XXX.COM.keytab", + "dfs.datanode.keytab.enabled": "true", + "dfs.datanode.kerberos.principal": "hdfs/_HOST@XXX.COM", + "dfs.client.use.datanode.hostname": "false", + "dfs.client.failover.proxy.provider.nameservice1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.balancer.keytab.file": "/tmp/hdfs@XXX.COM.keytab", + "dfs.balancer.keytab.enabled": "true", + "dfs.balancer.kerberos.principal": "hdfs/_HOST@XXX.COM", + "hive.metastore.uris":"thrift://nn1:9083,thrift://nn2:9083", + "hive.server2.authentication":"kerberos", + "hive.metastore.kerberos.principal":"hive/_HOST@XXX.COM", + "hive.metastore.sasl.enabled":"true" + } + } + } + } + ] + } +} +``` + + +## 类型转换 + +| Addax 内部类型 | Iceberg 数据类型 | +|------------------------------|--------------| +| Integer | INTEGER | +| Long | LONG | +| Double | DOUBLE | +| Float | FLOAT | +| Decimal | DECIMAL | +| String | STRING | +| Boolean | BOOLEAN | +| Date | DATE | +| TIMESTAMP | TIMESTAMP | +| Bytes | BINARY | +| STRING(逗号分隔如'a,b,c') | ARRAY | +| STRING(json格式如'{"a":"1"}') | MAP | + + From b21da1be21f3aaf3e90acb86cd8f679ddb2a53be Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 20 Feb 2025 13:24:57 +0800 Subject: [PATCH 20/40] [add][plugin][icebergwriter] add build scripts to docs --- docs/writer/icebgerwriter.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/writer/icebgerwriter.md b/docs/writer/icebgerwriter.md index d8801ae1e..58665fcb2 100644 --- a/docs/writer/icebgerwriter.md +++ b/docs/writer/icebgerwriter.md @@ -608,4 +608,11 @@ hive catalog例子 | STRING(逗号分隔如'a,b,c') | ARRAY | | STRING(json格式如'{"a":"1"}') | MAP | +##插件构建 + +```shell +set JAVA_HOME=E:\jdk\openlogic-openjdk-17.0.13+11-windows-x64 +mvn package install -Pdefault -Piceberg -pl plugin/writer/icebergwriter +``` + From c1b876b5bbc2f1e653a368aec6c2d48686a29a8d Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 20 Feb 2025 13:25:56 +0800 Subject: [PATCH 21/40] [add][plugin][icebergwriter] docs fixed --- docs/writer/icebgerwriter.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/writer/icebgerwriter.md b/docs/writer/icebgerwriter.md index 58665fcb2..e7dce4214 100644 --- a/docs/writer/icebgerwriter.md +++ b/docs/writer/icebgerwriter.md @@ -1,6 +1,6 @@ -# Paimon Writer +# Iceberg Writer -Paimon Writer 提供向 已有的paimon表写入数据的能力。 +Iceberg Writer 提供向 已有的iceberg表写入数据的能力。 ## 配置样例 From dd744a757ee6530ad1e72ce8943d1001ca254090 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 20 Feb 2025 13:45:16 +0800 Subject: [PATCH 22/40] [add][plugin][icebergwriter] docs --- docs/writer/icebergwriter.md | 618 +++++++++++++++++++++++++++++++++++ 1 file changed, 618 insertions(+) create mode 100644 docs/writer/icebergwriter.md diff --git a/docs/writer/icebergwriter.md b/docs/writer/icebergwriter.md new file mode 100644 index 000000000..e7dce4214 --- /dev/null +++ b/docs/writer/icebergwriter.md @@ -0,0 +1,618 @@ +# Iceberg Writer + +Iceberg Writer 提供向 已有的iceberg表写入数据的能力。 + +## 配置样例 + +```json +--8<-- "jobs/icebergwriter.json" +``` + +## 参数说明 + +| 配置项 | 是否必须 | 数据类型 | 默认值 | 说明 | +|:-------------|:----:|--------|----|-------------------------------------------------| +| tableName | 是 | string | 无 | 要写入的iceberg表名 | +| catalogType | 是 | string | 无 | catalog类型, 目前支持 hive,hadoop | +| warehouse | 是 | string | 无 | 仓库地址 | +| writeMode | 是 | string | 无 | 写入模式,详述见下 | +| hadoopConfig | 是 | json | {} | 里可以配置与 Iceberg catalog和Hadoop 相关的一些高级参数,比如HA的配置 | + + + +### writeMode + +写入前数据清理处理模式: + +- append,写入前不做任何处理,直接写入,不清除原来的数据。 +- truncate 写入前先清空表,再写入。 + +### hadoopConfig + +`hadoopConfig` 里可以配置与 Iceberg catalog和Hadoop 相关的一些高级参数,比如HA的配置 + +创建表实例: + +依赖包设置: + +build.gradle + +```groovy +plugins { + id 'java' +} + +group = 'com.awol2005ex' +version = '1.0-SNAPSHOT' +ext["hadoop_version"] = "3.2.4" +ext["hive_version"] = "3.1.3" +ext["woodstox_version"] = "7.0.0" +ext["iceberg_version"] = "1.8.0" +repositories { + maven { url "https://maven.aliyun.com/repository/central" } + maven { url "https://maven.aliyun.com/repository/public" } + maven { + url 'https://repo.huaweicloud.com/repository/maven/' + } + maven { + url 'https://repo.spring.io/libs-milestone/' + } + + + maven { + url 'https://repo.spring.io/libs-snapshot' + } + mavenCentral() +} + +dependencies { + testImplementation platform('org.junit:junit-bom:5.10.0') + testImplementation 'org.junit.jupiter:junit-jupiter' + implementation("org.apache.hadoop:hadoop-common:${hadoop_version}") { + exclude group: 'com.fasterxml.jackson.core', module: 'jackson-databind' + exclude group: 'org.codehaus.jackson', module: 'jackson-core-asl' + exclude group: 'org.codehaus.jackson', module: 'jackson-mapper-asl' + exclude group: 'com.fasterxml.woodstox', module: 'woodstox-core' + exclude group: 'commons-codec', module: 'commons-codec' + exclude group: 'commons-net', module: 'commons-net' + exclude group: 'io.netty', module: 'netty' + exclude group: 'log4j', module: 'log4j' + exclude group: 'net.minidev', module: 'json-smart' + exclude group: 'org.codehaus.jettison', module: 'jettison' + exclude group: 'org.eclipse.jetty', module: 'jetty-server' + exclude group: 'org.xerial.snappy', module: 'snappy-java' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + exclude group: 'org.eclipse.jetty', module: 'jetty-util' + } + implementation("org.apache.hadoop:hadoop-aws:${hadoop_version}") { + exclude group: 'com.fasterxml.jackson.core', module: 'jackson-databind' + exclude group: 'org.codehaus.jackson', module: 'jackson-core-asl' + exclude group: 'org.codehaus.jackson', module: 'jackson-mapper-asl' + exclude group: 'com.fasterxml.woodstox', module: 'woodstox-core' + exclude group: 'commons-codec', module: 'commons-codec' + exclude group: 'commons-net', module: 'commons-net' + exclude group: 'io.netty', module: 'netty' + exclude group: 'log4j', module: 'log4j' + exclude group: 'net.minidev', module: 'json-smart' + exclude group: 'org.codehaus.jettison', module: 'jettison' + exclude group: 'org.eclipse.jetty', module: 'jetty-server' + exclude group: 'org.xerial.snappy', module: 'snappy-java' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + exclude group: 'org.eclipse.jetty', module: 'jetty-util' + } + implementation("org.apache.hadoop:hadoop-mapreduce-client-core:${hadoop_version}") { + exclude group: 'com.fasterxml.jackson.core', module: 'jackson-databind' + exclude group: 'org.codehaus.jackson', module: 'jackson-core-asl' + exclude group: 'org.codehaus.jackson', module: 'jackson-mapper-asl' + exclude group: 'com.fasterxml.woodstox', module: 'woodstox-core' + exclude group: 'commons-codec', module: 'commons-codec' + exclude group: 'commons-net', module: 'commons-net' + exclude group: 'io.netty', module: 'netty' + exclude group: 'log4j', module: 'log4j' + exclude group: 'net.minidev', module: 'json-smart' + exclude group: 'org.codehaus.jettison', module: 'jettison' + exclude group: 'org.eclipse.jetty', module: 'jetty-server' + exclude group: 'org.xerial.snappy', module: 'snappy-java' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + exclude group: 'org.eclipse.jetty', module: 'jetty-util' + } + implementation("org.apache.hive:hive-metastore:${hive_version}"){ + exclude group: 'com.fasterxml.jackson.core', module: 'jackson-databind' + exclude group: 'org.codehaus.jackson', module: 'jackson-core-asl' + exclude group: 'org.codehaus.jackson', module: 'jackson-mapper-asl' + exclude group: 'com.fasterxml.woodstox', module: 'woodstox-core' + exclude group: 'commons-codec', module: 'commons-codec' + exclude group: 'commons-net', module: 'commons-net' + exclude group: 'io.netty', module: 'netty' + exclude group: 'log4j', module: 'log4j' + exclude group: 'net.minidev', module: 'json-smart' + exclude group: 'org.codehaus.jettison', module: 'jettison' + exclude group: 'org.eclipse.jetty', module: 'jetty-server' + exclude group: 'org.xerial.snappy', module: 'snappy-java' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + exclude group: 'org.eclipse.jetty', module: 'jetty-util' + } + implementation("com.fasterxml.woodstox:woodstox-core:${woodstox_version}") + + implementation("org.apache.iceberg:iceberg-common:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-api:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-arrow:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-aws:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-core:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-parquet:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-orc:${iceberg_version}") + implementation("org.apache.iceberg:iceberg-hive-metastore:${iceberg_version}") +} + +test { + useJUnitPlatform() +} +``` + +创建存储在minio,catalogType是hadoop的iceberg表 + +```java +package com.test; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; + +import java.io.IOException; + +public class CreateMinioTable { + public static void main(String[] args) throws IOException { + + Configuration hadoopConf = new Configuration(); + "fs.s3a.endpoint", "http://localhost:9000"); + "fs.s3a.access.key", "gy0dX5lALP176g6c9fYf"); + "fs.s3a.secret.key", "ReuUrCzzu5wKWAegtswoHIWV389BYl9AB1ZQbiKr"); + "fs.s3a.connection.ssl.enabled", "false"); + "fs.s3a.path.style.access", "true"); + "fs.s3a.impl","org.apache.hadoop.fs.s3a.S3AFileSystem"); + String warehousePath = "s3a://pvc-91d1e2cd-4d25-45c9-8613-6c4f7bf0a4cc/iceberg"; + HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehousePath); + + TableIdentifier name = TableIdentifier.of("test", "test1"); + + Schema schema = new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "ts1", Types.TimestampType.withoutZone()), + Types.NestedField.required(3, "name", Types.StringType.get()) + ); + Table table = catalog.createTable(name, schema); + System.out.println(table.location()); + + catalog.close(); + } +} + +``` + +创建存储在hdfs,catalogType是hadoop的iceberg表 +```java +package com.test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.types.Types; + +import java.io.IOException; + +public class CreateHdfsTable { + public static void main(String[] args) throws IOException { + + System.setProperty("java.security.krb5.conf","D:/MIT/krb5.ini"); + + Configuration hadoopConf = new Configuration(); + "fs.defaultFS", "hdfs://nameservice1"); + "hadoop.security.authentication", "kerberos"); + "hadoop.kerberos.principal", "hive/_HOST@XXX.COM"); + "hadoop.kerberos.keytab", "/tmp/hive@XXX.COM.keytab"); + "ha.zookeeper.quorum", "nn1:2181,nn2:2181,nn3:2181"); + "dfs.nameservices", "nameservice1"); + "dfs.namenode.rpc-address.nameservice1.namenode371", "nn2:8020"); + "dfs.namenode.rpc-address.nameservice1.namenode265", "nn1:8020"); + "dfs.namenode.keytab.file", "/tmp/hdfs@XXX.COM.keytab"); + "dfs.namenode.keytab.enabled", "true"); + "dfs.namenode.kerberos.principal", "hdfs/_HOST@XXX.COM"); + "dfs.namenode.kerberos.internal.spnego.principal", "HTTP/_HOST@XXX.COM"); + "dfs.ha.namenodes.nameservice1", "namenode265,namenode371"); + "dfs.datanode.keytab.file", "/tmp/hdfs@XXX.COM.keytab"); + "dfs.datanode.keytab.enabled", "true"); + "dfs.datanode.kerberos.principal", "hdfs/_HOST@XXX.COM"); + "dfs.client.use.datanode.hostname", "false"); + "dfs.client.failover.proxy.provider.nameservice1", "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider"); + "dfs.balancer.keytab.file", "/tmp/hdfs@XXX.COM.keytab"); + "dfs.balancer.keytab.enabled", "true"); + "dfs.balancer.kerberos.principal", "hdfs/_HOST@XXX.COM"); + + UserGroupInformation.setConfiguration(hadoopConf); + try { + UserGroupInformation.loginUserFromKeytab("hive@XXX.COM", "/tmp/hive@XXX.COM.keytab"); + } catch (Exception e) { + e.printStackTrace(); + } + + + + String warehousePath = "hdfs://nameservice1/user/hive/iceberg"; + HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehousePath); + + TableIdentifier name = TableIdentifier.of("test1", "test20250219"); + + Schema schema = new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "ts1", Types.TimestampType.withoutZone()), + Types.NestedField.required(3, "dec1", Types.DecimalType.of(12,2)), + Types.NestedField.required(4, "bool1", Types.BooleanType.get()), + Types.NestedField.required(5, "map1", Types.MapType.ofRequired(11,12,Types.StringType.get(),Types.StringType.get())), + Types.NestedField.required(6, "date1", Types.DateType.get()), + Types.NestedField.required(7, "float1", Types.FloatType.get()), + Types.NestedField.required(8, "double1", Types.DoubleType.get()), + Types.NestedField.required(9, "array1", Types.ListType.ofRequired(13,Types.StringType.get())), + Types.NestedField.required(10, "name", Types.StringType.get()) + ); + catalog.dropTable(name,true); + Table table = catalog.createTable(name, schema); + System.out.println(table.location()); + + catalog.close(); + } +} +``` +创建hive表 + +```java +package com.test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.types.Types; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class CreateHiveTable { + public static void main(String[] args) throws IOException { + + System.setProperty("java.security.krb5.conf","D:/MIT/krb5.ini"); + + Configuration hadoopConf = new Configuration(); + "fs.defaultFS", "hdfs,//nameservice1"); + "hadoop.security.authentication", "kerberos"); + "hadoop.kerberos.principal", "hive/_HOST@XXX.COM"); + "hadoop.kerberos.keytab", "/tmp/hive@XXX.COM.keytab"); + "ha.zookeeper.quorum", "nn1:2181,nn2:2181,nn3:2181"); + "dfs.nameservices", "nameservice1"); + "dfs.namenode.rpc-address.nameservice1.namenode371", "nn2:8020"); + "dfs.namenode.rpc-address.nameservice1.namenode265", "nn1:8020"); + "dfs.namenode.keytab.file", "/tmp/hdfs@XXX.COM.keytab"); + "dfs.namenode.keytab.enabled", "true"); + "dfs.namenode.kerberos.principal", "hdfs/_HOST@XXX.COM"); + "dfs.namenode.kerberos.internal.spnego.principal", "HTTP/_HOST@XXX.COM"); + "dfs.ha.namenodes.nameservice1", "namenode265,namenode371"); + "dfs.datanode.keytab.file", "/tmp/hdfs@XXX.COM.keytab"); + "dfs.datanode.keytab.enabled", "true"); + "dfs.datanode.kerberos.principal", "hdfs/_HOST@XXX.COM"); + "dfs.client.use.datanode.hostname", "false"); + "dfs.client.failover.proxy.provider.nameservice1", "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider"); + "dfs.balancer.keytab.file", "/tmp/hdfs@XXX.COM.keytab"); + "dfs.balancer.keytab.enabled", "true"); + "dfs.balancer.kerberos.principal", "hdfs/_HOST@XXX.COM"); + + "hive.metastore.uris", "thrift://nn1:9083,thrift://nn2:9083"); + "hive.server2.authentication","kerberos"); + "hive.metastore.kerberos.principal","hive/_HOST@XXX.COM"); + + "hive.metastore.sasl.enabled", "true"); + + UserGroupInformation.setConfiguration(hadoopConf); + try { + UserGroupInformation.loginUserFromKeytab("hive@XXX.COM", "/tmp/hive@XXX.COM.keytab"); + } catch (Exception e) { + e.printStackTrace();; + } + + + + + HiveCatalog catalog = new HiveCatalog(); + catalog.setConf(hadoopConf); + Map properties = new HashMap(); + properties.put("warehouse", "/warehouse/tablespace/managed/hive"); + properties.put("uri", "thrift://nn1:9083,thrift://nn2:9083"); + + catalog.initialize("hive", properties); + + TableIdentifier name = TableIdentifier.of("test1", "test20250218"); + + Schema schema = new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "ts1", Types.TimestampType.withoutZone()), + Types.NestedField.required(3, "name", Types.StringType.get()) + ); + Table table = catalog.createTable(name, schema); + System.out.println(table.location()); + + catalog.close(); + } +} + + +``` + + + +Spark 或者 flink 环境创建表 + +```sql +CREATE TABLE if not exists test1.test1_iceberg1 USING ICEBERG + TBLPROPERTIES( + 'format-version'='2', + 'write.metadata.delete-after-commit.enabled'=true, + 'write.metadata.previous-versions-max'=1, + 'target-file-size-bytes'=268435456 + ) + as select * from test1.test1 limit 0; + +``` + +s3 或者 minio hadoop catalog例子 +```json +{ + "job": { + "setting": { + "speed": { + "channel": 3 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + } + }, + "content": [ + { + "reader": { + "name": "rdbmsreader", + "parameter": { + "username": "root", + "password": "root", + "column": [ + "*" + ], + "connection": [ + { + "querySql": [ + "select 1+0 id ,now() ts1,'test1' as name" + ], + "jdbcUrl": [ + "jdbc:mysql://localhost:3306/ruoyi_vue_camunda?allowPublicKeyRetrieval=true" + ] + } + ], + "fetchSize": 1024 + } + }, + "writer": { + "name": "icebergwriter", + "parameter": { + "tableName": "test.test1", + "writeMode": "truncate", + "catalogType":"hadoop", + "warehouse": "s3a://pvc-91d1e2cd-4d25-45c9-8613-6c4f7bf0a4cc/iceberg", + "hadoopConfig": { + + "fs.s3a.endpoint": "http://localhost:9000", + "fs.s3a.access.key": "gy0dX5lALP176g6c9fYf", + "fs.s3a.secret.key": "ReuUrCzzu5wKWAegtswoHIWV389BYl9AB1ZQbiKr", + "fs.s3a.connection.ssl.enabled": "false", + "fs.s3a.path.style.access": "true", + "fs.s3a.impl": "org.apache.hadoop.fs.s3a.S3AFileSystem" + } + } + } + } + ] + } +} +``` + + +hdfs hadoop catalog例子 + +```json +{ + "job": { + "setting": { + "speed": { + "channel": 3 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + } + }, + "content": [ + { + "reader": { + "name": "rdbmsreader", + "parameter": { + "username": "root", + "password": "root", + "column": [ + "*" + ], + "connection": [ + { + "querySql": [ + "select 1+0 id ,now() ts1,CAST(1.2 AS DECIMAL(12,2)) dec1,true bool1,'{\"a\":\"1\"}' map1,now() date1,1.3 float1,1.4 double1,'a,b,c' array1,'test1' as name" + ], + "jdbcUrl": [ + "jdbc:mysql://localhost:3306/ruoyi_vue_camunda?allowPublicKeyRetrieval=true" + ] + } + ], + "fetchSize": 1024 + } + }, + "writer": { + "name": "icebergwriter", + "parameter": { + "tableName": "test1.test20250219", + "writeMode": "truncate", + "catalogType": "hadoop", + "warehouse": "hdfs://nameservice1/user/hive/iceberg", + "kerberosKeytabFilePath":"/tmp/hive@XXX.COM.keytab", + "kerberosPrincipal":"hive@XXX.COM", + "hadoopConfig": { + "fs.defaultFS": "hdfs://nameservice1", + "hadoop.security.authentication": "kerberos", + "hadoop.kerberos.principal": "hive/_HOST@XXX.COM", + "hadoop.kerberos.keytab": "/tmp/hive@XXX.COM.keytab", + "ha.zookeeper.quorum": "nn1:2181,nn2:2181,nn3:2181", + "dfs.nameservices": "nameservice1", + "dfs.namenode.rpc-address.nameservice1.namenode371": "nn2:8020", + "dfs.namenode.rpc-address.nameservice1.namenode265": "nn1:8020", + "dfs.namenode.keytab.file": "/tmp/hdfs@XXX.COM.keytab", + "dfs.namenode.keytab.enabled": "true", + "dfs.namenode.kerberos.principal": "hdfs/_HOST@XXX.COM", + "dfs.namenode.kerberos.internal.spnego.principal": "HTTP/_HOST@XXX.COM", + "dfs.ha.namenodes.nameservice1": "namenode265,namenode371", + "dfs.datanode.keytab.file": "/tmp/hdfs@XXX.COM.keytab", + "dfs.datanode.keytab.enabled": "true", + "dfs.datanode.kerberos.principal": "hdfs/_HOST@XXX.COM", + "dfs.client.use.datanode.hostname": "false", + "dfs.client.failover.proxy.provider.nameservice1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.balancer.keytab.file": "/tmp/hdfs@XXX.COM.keytab", + "dfs.balancer.keytab.enabled": "true", + "dfs.balancer.kerberos.principal": "hdfs/_HOST@XXX.COM" + } + } + } + } + ] + } +} +``` + + +hive catalog例子 + +```json +{ + "job": { + "setting": { + "speed": { + "channel": 3 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + } + }, + "content": [ + { + "reader": { + "name": "rdbmsreader", + "parameter": { + "username": "root", + "password": "root", + "column": [ + "*" + ], + "connection": [ + { + "querySql": [ + "select 1+0 id ,now() ts1,CAST(1.2 AS DECIMAL(12,2)) dec1,true bool1,'{\"a\":\"1\"}' map1,now() date1,1.3 float1,1.4 double1,'a,b,c' array1,'test1' as name" + ], + "jdbcUrl": [ + "jdbc:mysql://localhost:3306/ruoyi_vue_camunda?allowPublicKeyRetrieval=true" + ] + } + ], + "fetchSize": 1024 + } + }, + "writer": { + "name": "icebergwriter", + "parameter": { + "tableName": "test1.test20250219", + "writeMode": "truncate", + "catalogType": "hive", + "uri": "thrift://nn1:9083,thrift://nn2:9083", + "warehouse": "/warehouse/tablespace/managed/hive", + "kerberosKeytabFilePath":"/tmp/hive@XXX.COM.keytab", + "kerberosPrincipal":"hive@XXX.COM", + "hadoopConfig": { + "fs.defaultFS": "hdfs://nameservice1", + "hadoop.security.authentication": "kerberos", + "hadoop.kerberos.principal": "hive/_HOST@XXX.COM", + "hadoop.kerberos.keytab": "/tmp/hive@XXX.COM.keytab", + "ha.zookeeper.quorum": "nn1:2181,nn2:2181,nn3:2181", + "dfs.nameservices": "nameservice1", + "dfs.namenode.rpc-address.nameservice1.namenode371": "nn2:8020", + "dfs.namenode.rpc-address.nameservice1.namenode265": "nn1:8020", + "dfs.namenode.keytab.file": "/tmp/hdfs@XXX.COM.keytab", + "dfs.namenode.keytab.enabled": "true", + "dfs.namenode.kerberos.principal": "hdfs/_HOST@XXX.COM", + "dfs.namenode.kerberos.internal.spnego.principal": "HTTP/_HOST@XXX.COM", + "dfs.ha.namenodes.nameservice1": "namenode265,namenode371", + "dfs.datanode.keytab.file": "/tmp/hdfs@XXX.COM.keytab", + "dfs.datanode.keytab.enabled": "true", + "dfs.datanode.kerberos.principal": "hdfs/_HOST@XXX.COM", + "dfs.client.use.datanode.hostname": "false", + "dfs.client.failover.proxy.provider.nameservice1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.balancer.keytab.file": "/tmp/hdfs@XXX.COM.keytab", + "dfs.balancer.keytab.enabled": "true", + "dfs.balancer.kerberos.principal": "hdfs/_HOST@XXX.COM", + "hive.metastore.uris":"thrift://nn1:9083,thrift://nn2:9083", + "hive.server2.authentication":"kerberos", + "hive.metastore.kerberos.principal":"hive/_HOST@XXX.COM", + "hive.metastore.sasl.enabled":"true" + } + } + } + } + ] + } +} +``` + + +## 类型转换 + +| Addax 内部类型 | Iceberg 数据类型 | +|------------------------------|--------------| +| Integer | INTEGER | +| Long | LONG | +| Double | DOUBLE | +| Float | FLOAT | +| Decimal | DECIMAL | +| String | STRING | +| Boolean | BOOLEAN | +| Date | DATE | +| TIMESTAMP | TIMESTAMP | +| Bytes | BINARY | +| STRING(逗号分隔如'a,b,c') | ARRAY | +| STRING(json格式如'{"a":"1"}') | MAP | + +##插件构建 + +```shell +set JAVA_HOME=E:\jdk\openlogic-openjdk-17.0.13+11-windows-x64 +mvn package install -Pdefault -Piceberg -pl plugin/writer/icebergwriter +``` + + From ab8bd116cf14768f930708abfffdd467b985a018 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 20 Feb 2025 13:46:25 +0800 Subject: [PATCH 23/40] [add][plugin][icebergwriter] docs name error --- docs/writer/{icebgerwriter.md => icebergwriter.md} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/writer/{icebgerwriter.md => icebergwriter.md} (100%) diff --git a/docs/writer/icebgerwriter.md b/docs/writer/icebergwriter.md similarity index 100% rename from docs/writer/icebgerwriter.md rename to docs/writer/icebergwriter.md From a9b2fe2247001ae3266f4900e240af1c198623bd Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 20 Feb 2025 13:54:10 +0800 Subject: [PATCH 24/40] [add][plugin][icebergwriter] add icebergwriter code --- docs/assets/jobs/icebergwriter.json | 56 ++ package.xml | 11 + plugin/writer/icebergwriter/package.xml | 37 + plugin/writer/icebergwriter/pom.xml | 871 ++++++++++++++++++ .../writer/icebergwriter/IcebergHelper.java | 102 ++ .../writer/icebergwriter/IcebergWriter.java | 307 ++++++ .../src/main/resources/plugin.json | 6 + .../main/resources/plugin_job_template.json | 17 + pom.xml | 16 + 9 files changed, 1423 insertions(+) create mode 100644 docs/assets/jobs/icebergwriter.json create mode 100644 plugin/writer/icebergwriter/package.xml create mode 100644 plugin/writer/icebergwriter/pom.xml create mode 100644 plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergHelper.java create mode 100644 plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java create mode 100644 plugin/writer/icebergwriter/src/main/resources/plugin.json create mode 100644 plugin/writer/icebergwriter/src/main/resources/plugin_job_template.json diff --git a/docs/assets/jobs/icebergwriter.json b/docs/assets/jobs/icebergwriter.json new file mode 100644 index 000000000..fedcebf72 --- /dev/null +++ b/docs/assets/jobs/icebergwriter.json @@ -0,0 +1,56 @@ +{ + "job": { + "setting": { + "speed": { + "channel": 3 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + } + }, + "content": [ + { + "reader": { + "name": "rdbmsreader", + "parameter": { + "username": "root", + "password": "root", + "column": [ + "*" + ], + "connection": [ + { + "querySql": [ + "select 1+0 id ,now() ts1,'test1' as name" + ], + "jdbcUrl": [ + "jdbc:mysql://localhost:3306/ruoyi_vue_camunda?allowPublicKeyRetrieval=true" + ] + } + ], + "fetchSize": 1024 + } + }, + "writer": { + "name": "icebergwriter", + "parameter": { + "tableName": "test.test1", + "writeMode": "truncate", + "catalogType":"hadoop", + "warehouse": "s3a://pvc-91d1e2cd-4d25-45c9-8613-6c4f7bf0a4cc/iceberg", + "hadoopConfig": { + + "fs.s3a.endpoint": "http://localhost:9000", + "fs.s3a.access.key": "gy0dX5lALP176g6c9fYf", + "fs.s3a.secret.key": "ReuUrCzzu5wKWAegtswoHIWV389BYl9AB1ZQbiKr", + "fs.s3a.connection.ssl.enabled": "false", + "fs.s3a.path.style.access": "true", + "fs.s3a.impl": "org.apache.hadoop.fs.s3a.S3AFileSystem" + } + } + } + } + ] + } +} \ No newline at end of file diff --git a/package.xml b/package.xml index 16ba1ce6d..c4e67a5f8 100644 --- a/package.xml +++ b/package.xml @@ -453,6 +453,17 @@ 0644 addax-${project.version} + + plugin/writer/icebergwriter/target/icebergwriter-${project.version}/ + + **/*.* + + + **/addax-common*.jar + + 0644 + addax-${project.version} + plugin/writer/influxdbwriter/target/influxdbwriter-${project.version}/ diff --git a/plugin/writer/icebergwriter/package.xml b/plugin/writer/icebergwriter/package.xml new file mode 100644 index 000000000..8ecab9f9f --- /dev/null +++ b/plugin/writer/icebergwriter/package.xml @@ -0,0 +1,37 @@ + + release + + dir + + false + + + src/main/resources + + *.json + + plugin/writer/${project.artifactId} + + + target/ + + ${project.artifactId}-${project.version}.jar + + plugin/writer/${project.artifactId} + + + + + + false + plugin/writer/${project.artifactId}/libs + runtime + + com.wgzhao.addax:* + + + + diff --git a/plugin/writer/icebergwriter/pom.xml b/plugin/writer/icebergwriter/pom.xml new file mode 100644 index 000000000..f6cbe43b1 --- /dev/null +++ b/plugin/writer/icebergwriter/pom.xml @@ -0,0 +1,871 @@ + + + 4.0.0 + + + com.wgzhao.addax + addax-all + 4.2.4-SNAPSHOT + ../../../pom.xml + + + icebergwriter + iceberg-writer + Apache Iceberg writer plugin for Addax + jar + + + + com.wgzhao.addax + addax-common + ${project.version} + + + slf4j-log4j12 + org.slf4j + + + + + + com.wgzhao.addax + addax-storage + ${project.version} + + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + + org.apache.hadoop + hadoop-aws + ${hadoop.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + + + com.fasterxml.jackson.core + jackson-databind + + + commons-codec + commons-codec + + + io.netty + netty + + + org.eclipse.jetty + jetty-util + + + + + + + com.fasterxml.woodstox + woodstox-core + ${woodstox.version} + + + + org.apache.iceberg + iceberg-common + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-api + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-arrow + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-aws + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-core + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-parquet + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-orc + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.iceberg + iceberg-hive-metastore + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.parquet + parquet-hadoop-bundle + ${parquet.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + + + org.apache.orc + orc-core + ${orc.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + org.apache.hadoop + hadoop-client-api + + + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + org.apache.avro + avro + ${avro.version} + + + com.fasterxml.jackson.core + jackson-databind + + + + + + + + + maven-assembly-plugin + + + package.xml + + ${project.artifactId}-${project.version} + + + + release + package + + single + + + + + + + diff --git a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergHelper.java b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergHelper.java new file mode 100644 index 000000000..e45d0db18 --- /dev/null +++ b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergHelper.java @@ -0,0 +1,102 @@ +package com.wgzhao.addax.plugin.writer.icebergwriter; + +import com.wgzhao.addax.common.exception.AddaxException; +import com.wgzhao.addax.common.util.Configuration; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hive.HiveCatalog; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +import static com.wgzhao.addax.common.base.Key.KERBEROS_KEYTAB_FILE_PATH; +import static com.wgzhao.addax.common.base.Key.KERBEROS_PRINCIPAL; +import static com.wgzhao.addax.common.spi.ErrorCode.LOGIN_ERROR; + +public class IcebergHelper { + private static final Logger LOG = LoggerFactory.getLogger(IcebergHelper.class); + + public static void kerberosAuthentication(org.apache.hadoop.conf.Configuration hadoopConf, String kerberosPrincipal, String kerberosKeytabFilePath) throws Exception { + if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) { + UserGroupInformation.setConfiguration(hadoopConf); + try { + UserGroupInformation.loginUserFromKeytab(kerberosPrincipal, kerberosKeytabFilePath); + } catch (Exception e) { + String message = String.format("kerberos authentication failed, keytab file: [%s], principal: [%s]", + kerberosKeytabFilePath, kerberosPrincipal); + LOG.error(message); + throw AddaxException.asAddaxException(LOGIN_ERROR, e); + } + } + } + + public static Catalog getCatalog(Configuration conf) throws Exception { + + String catalogType = conf.getString("catalogType"); + if (catalogType == null || catalogType.trim().isEmpty()) { + throw new RuntimeException("catalogType is not set"); + } + catalogType = catalogType.trim(); + + String warehouse = conf.getString("warehouse"); + if (warehouse == null || warehouse.trim().isEmpty()) { + throw new RuntimeException("warehouse is not set"); + } + + org.apache.hadoop.conf.Configuration hadoopConf = null; + + if (conf.getConfiguration("hadoopConfig") != null) { + Map hadoopConfig = conf.getMap("hadoopConfig"); + + hadoopConf = new org.apache.hadoop.conf.Configuration(); + + for (String key : hadoopConfig.keySet()) { + hadoopConf.set(key, (String)hadoopConfig.get(key)); + } + + + String authentication = (String)hadoopConfig.get("hadoop.security.authentication"); + + if ("kerberos".equals(authentication)) { + String kerberosKeytabFilePath = conf.getString(KERBEROS_KEYTAB_FILE_PATH); + if(kerberosKeytabFilePath ==null || kerberosKeytabFilePath.trim().isEmpty()){ + throw new RuntimeException("kerberosKeytabFilePath is not set"); + } else { + kerberosKeytabFilePath = kerberosKeytabFilePath.trim(); + } + + String kerberosPrincipal = conf.getString(KERBEROS_PRINCIPAL); + if(kerberosPrincipal ==null || kerberosPrincipal.trim().isEmpty()){ + throw new RuntimeException("kerberosPrincipal is not set"); + } else { + kerberosPrincipal = kerberosPrincipal.trim(); + } + IcebergHelper.kerberosAuthentication(hadoopConf, kerberosPrincipal, kerberosKeytabFilePath); + } + } + switch (catalogType) { + case "hadoop": + return new HadoopCatalog(hadoopConf, warehouse); + case "hive": + String uri = conf.getString("uri"); + if (uri == null || uri.trim().isEmpty()) { + throw new RuntimeException("uri is not set"); + } + HiveCatalog hiveCatalog = new HiveCatalog(); + hiveCatalog.setConf(hadoopConf); + Map properties = new HashMap(); + properties.put("warehouse", warehouse); + properties.put("uri", uri); + + hiveCatalog.initialize("hive", properties); + return hiveCatalog; + } + + throw new RuntimeException("not support catalogType:" + catalogType); + } + +} diff --git a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java new file mode 100644 index 000000000..f0731e2dd --- /dev/null +++ b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java @@ -0,0 +1,307 @@ +package com.wgzhao.addax.plugin.writer.icebergwriter; + +import com.alibaba.fastjson2.JSON; +import com.google.common.collect.ImmutableList; +import com.wgzhao.addax.common.element.Column; +import com.wgzhao.addax.common.element.Record; +import com.wgzhao.addax.common.plugin.RecordReceiver; +import com.wgzhao.addax.common.spi.Writer; +import com.wgzhao.addax.common.util.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.*; + + +public class IcebergWriter extends Writer { + public static class Job + extends Writer.Job { + private static final Logger LOG = LoggerFactory.getLogger(Job.class); + private Configuration conf = null; + private Catalog catalog = null; + private String tableName = null; + + @Override + public void init() { + this.conf = this.getPluginJobConf(); + try { + this.catalog = IcebergHelper.getCatalog(conf); + } catch (Exception e) { + throw new RuntimeException(e); + } + + tableName = this.conf.getString("tableName"); + if (tableName == null || tableName.trim().isEmpty()) { + throw new RuntimeException("tableName is not set"); + } + + + } + + @Override + public List split(int mandatoryNumber) { + List configurations = new ArrayList<>(mandatoryNumber); + for (int i = 0; i < mandatoryNumber; i++) { + configurations.add(conf); + } + return configurations; + } + + @Override + public void prepare() { + String writeMode = this.conf.getString("writeMode"); + if ("truncate".equalsIgnoreCase(writeMode)) { + Table table = catalog.loadTable(TableIdentifier.of(tableName.split("\\."))); + table.newDelete().deleteFromRowFilter(org.apache.iceberg.expressions.Expressions.alwaysTrue()).commit(); + } + } + + @Override + public void destroy() { + if (this.catalog != null) { + try { + if (this.catalog instanceof HiveCatalog) { + ((HiveCatalog) this.catalog).close(); + } + if (this.catalog instanceof HadoopCatalog) { + ((HadoopCatalog) this.catalog).close(); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + } + } + + public static class Task + extends Writer.Task { + + private static final Logger log = LoggerFactory.getLogger(Task.class); + private Catalog catalog = null; + private Integer batchSize = 1000; + private Table table = null; + private org.apache.iceberg.Schema schema = null; + private String fileFormat = "parquet"; + private List columnList = null; + + @Override + public void startWrite(RecordReceiver recordReceiver) { + + List writerBuffer = new ArrayList<>(this.batchSize); + Record record; + long total = 0; + while ((record = recordReceiver.getFromReader()) != null) { + writerBuffer.add(record); + if (writerBuffer.size() >= this.batchSize) { + total += doBatchInsert(writerBuffer); + writerBuffer.clear(); + } + } + + if (!writerBuffer.isEmpty()) { + total += doBatchInsert(writerBuffer); + writerBuffer.clear(); + } + + String msg = String.format("task end, write size :%d", total); + getTaskPluginCollector().collectMessage("writeSize", String.valueOf(total)); + log.info(msg); + } + + @Override + public void init() { + Configuration conf = super.getPluginJobConf(); + + batchSize = conf.getInt("batchSize", 1000); + + try { + this.catalog = IcebergHelper.getCatalog(conf); + } catch (Exception e) { + throw new RuntimeException(e); + } + + String tableName = conf.getString("tableName"); + if (tableName == null || tableName.trim().isEmpty()) { + throw new RuntimeException("tableName is not set"); + } + + table = catalog.loadTable(TableIdentifier.of(tableName.split("\\."))); + schema = table.schema(); + + fileFormat = table.properties().get("write.format.default"); + if (fileFormat == null || fileFormat.trim().isEmpty()) { + fileFormat = "parquet"; + } + + + columnList = schema.columns(); + } + + @Override + public void destroy() { + if (this.catalog != null) { + try { + if (this.catalog instanceof HiveCatalog) { + ((HiveCatalog) this.catalog).close(); + } + if (this.catalog instanceof HadoopCatalog) { + ((HadoopCatalog) this.catalog).close(); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + } + + private long doBatchInsert(final List writerBuffer) { + ImmutableList.Builder builder = ImmutableList.builder(); + + + for (Record record : writerBuffer) { + GenericRecord data = GenericRecord.create(schema); + for (int i = 0; i < record.getColumnNumber(); i++) { + Column column = record.getColumn(i); + if (column == null) { + continue; + } + if (i >= columnList.size()) { + throw new RuntimeException("columnList size is " + columnList.size() + ", but record column number is " + record.getColumnNumber()); + } + Types.NestedField field = columnList.get(i); + org.apache.iceberg.types.Type columnType = field.type(); + //如果是数组类型,那它传入的必是字符串类型 + if (columnType.isListType()) { + if (null == column.asString()) { + data.setField(field.name(), null); + } else { + String[] dataList = column.asString().split(","); + data.setField(field.name(), Arrays.asList(dataList)); + } + } else { + switch (columnType.typeId()) { + + case DATE: + try { + if (column.asLong() != null) { + data.setField(field.name(), column.asTimestamp().toLocalDateTime().toLocalDate()); + } else { + data.setField(field.name(), null); + } + } catch (Exception e) { + getTaskPluginCollector().collectDirtyRecord(record, String.format("日期类型解析失败 [%s:%s] exception: %s", field.name(), column, e)); + } + break; + case TIME: + case TIMESTAMP: + case TIMESTAMP_NANO: + try { + if (column.asLong() != null) { + data.setField(field.name(), column.asTimestamp().toLocalDateTime()); + } else { + data.setField(field.name(), null); + } + } catch (Exception e) { + getTaskPluginCollector().collectDirtyRecord(record, String.format("时间类型解析失败 [%s:%s] exception: %s", field.name(), column, e)); + } + break; + case STRING: + data.setField(field.name(), column.asString()); + break; + case BOOLEAN: + data.setField(field.name(), column.asBoolean()); + break; + case FIXED: + case BINARY: + data.setField(field.name(), column.asBytes()); + break; + case LONG: + data.setField(field.name(), column.asLong()); + break; + case INTEGER: + data.setField(field.name(), column.asBigInteger() == null ? null : column.asBigInteger().intValue()); + break; + case FLOAT: + data.setField(field.name(), column.asDouble().floatValue()); + break; + case DOUBLE: + + data.setField(field.name(), column.asDouble()); + break; + case DECIMAL: + if (column.asBigDecimal() != null) { + data.setField(field.name(), column.asBigDecimal()); + } else { + data.setField(field.name(), null); + } + break; + case MAP: + try { + data.setField(field.name(), JSON.parseObject(column.asString(), Map.class)); + } catch (Exception e) { + getTaskPluginCollector().collectDirtyRecord(record, String.format("MAP类型解析失败 [%s:%s] exception: %s", field.name(), column, e)); + } + break; + default: + getTaskPluginCollector().collectDirtyRecord(record, "类型错误:不支持的类型:" + columnType + " " + field.name()); + } + } + + } + + + builder.add(data); + } + + String filepath = table.location() + "/" + UUID.randomUUID(); + OutputFile file = table.io().newOutputFile(filepath); + + DataWriter dataWriter = null; + + if ("parquet".equals(fileFormat)) { + try { + dataWriter = Parquet.writeData(file).overwrite().forTable(table).createWriterFunc(GenericParquetWriter::buildWriter).build(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } else if ("orc".equals(fileFormat)) { + dataWriter = ORC.writeData(file).overwrite().forTable(table).createWriterFunc(GenericOrcWriter::buildWriter).build(); + } else { + throw new RuntimeException("不支持的文件格式:" + fileFormat); + } + ImmutableList rows = builder.build(); + + if (dataWriter != null) { + dataWriter.write(rows); + } + + + if (dataWriter != null) { + try { + dataWriter.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + DataFile dataFile = dataWriter.toDataFile(); + table.newAppend().appendFile(dataFile).commit(); + return rows.size(); + } + } +} diff --git a/plugin/writer/icebergwriter/src/main/resources/plugin.json b/plugin/writer/icebergwriter/src/main/resources/plugin.json new file mode 100644 index 000000000..894f4e064 --- /dev/null +++ b/plugin/writer/icebergwriter/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "icebergwriter", + "class": "com.wgzhao.addax.plugin.writer.icebergwriter.IcebergWriter", + "description": "write data to iceberg", + "developer": "awol2005ex" +} diff --git a/plugin/writer/icebergwriter/src/main/resources/plugin_job_template.json b/plugin/writer/icebergwriter/src/main/resources/plugin_job_template.json new file mode 100644 index 000000000..831992aff --- /dev/null +++ b/plugin/writer/icebergwriter/src/main/resources/plugin_job_template.json @@ -0,0 +1,17 @@ +{ + "name": "icebergwriter", + "parameter": { + "tableName": "test.test1", + "catalogType" : "hadoop", + "writeMode": "truncate", + "warehouse": "s3a://pvc-91d1e2cd-4d25-45c9-8613-6c4f7bf0a4cc/iceberg", + "hadoopConfig": { + "fs.s3a.endpoint":"http://localhost:9000", + "fs.s3a.access.key":"gy0dX5lALP176g6c9fYf", + "fs.s3a.secret.key":"ReuUrCzzu5wKWAegtswoHIWV389BYl9AB1ZQbiKr", + "fs.s3a.connection.ssl.enabled":"false", + "fs.s3a.path.style.access":"true", + "fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem" + } + } +} diff --git a/pom.xml b/pom.xml index bc84fe3d2..847cc09d3 100644 --- a/pom.xml +++ b/pom.xml @@ -439,5 +439,21 @@ plugin/writer/txtfilewriter + + + iceberg + + + 1.8.0 + 11 + 11 + 11 + + + + + plugin/writer/icebergwriter + + From bab8a0cd4eef67700d1075014ffaa1617d836a82 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Tue, 25 Feb 2025 10:58:46 +0800 Subject: [PATCH 25/40] [add][plugin][icebergwriter] change to streamreader at example --- docs/assets/jobs/icebergwriter.json | 28 +++++++++++++--------------- 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/docs/assets/jobs/icebergwriter.json b/docs/assets/jobs/icebergwriter.json index fedcebf72..ce0ab0d39 100644 --- a/docs/assets/jobs/icebergwriter.json +++ b/docs/assets/jobs/icebergwriter.json @@ -12,24 +12,23 @@ "content": [ { "reader": { - "name": "rdbmsreader", + "name": "streamreader", "parameter": { - "username": "root", - "password": "root", "column": [ - "*" - ], - "connection": [ { - "querySql": [ - "select 1+0 id ,now() ts1,'test1' as name" - ], - "jdbcUrl": [ - "jdbc:mysql://localhost:3306/ruoyi_vue_camunda?allowPublicKeyRetrieval=true" - ] + "value": "1", + "type": "long" + }, + { + "value": "1989-06-04 00:00:00", + "type": "timestamp" + }, + { + "value": "test1", + "type": "string" } ], - "fetchSize": 1024 + "sliceRecordCount": 1000 } }, "writer": { @@ -37,10 +36,9 @@ "parameter": { "tableName": "test.test1", "writeMode": "truncate", - "catalogType":"hadoop", + "catalogType": "hadoop", "warehouse": "s3a://pvc-91d1e2cd-4d25-45c9-8613-6c4f7bf0a4cc/iceberg", "hadoopConfig": { - "fs.s3a.endpoint": "http://localhost:9000", "fs.s3a.access.key": "gy0dX5lALP176g6c9fYf", "fs.s3a.secret.key": "ReuUrCzzu5wKWAegtswoHIWV389BYl9AB1ZQbiKr", From 65831744a04acf1e6a9f4cdb0c9a683b080a24de Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Mon, 3 Mar 2025 12:32:14 +0800 Subject: [PATCH 26/40] [add][plugin][icebergwriter] fixed bug when partition iceberg table --- plugin/writer/icebergwriter/pom.xml | 63 ++++++++++ .../writer/icebergwriter/IcebergWriter.java | 117 ++++++++++++++---- 2 files changed, 157 insertions(+), 23 deletions(-) diff --git a/plugin/writer/icebergwriter/pom.xml b/plugin/writer/icebergwriter/pom.xml index f6cbe43b1..cbb8e9b6c 100644 --- a/plugin/writer/icebergwriter/pom.xml +++ b/plugin/writer/icebergwriter/pom.xml @@ -383,6 +383,69 @@ + + org.apache.iceberg + iceberg-data + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + org.apache.iceberg iceberg-aws diff --git a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java index f0731e2dd..f7a66a2a0 100644 --- a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java +++ b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java @@ -2,29 +2,34 @@ import com.alibaba.fastjson2.JSON; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.common.primitives.Ints; import com.wgzhao.addax.common.element.Column; import com.wgzhao.addax.common.element.Record; import com.wgzhao.addax.common.plugin.RecordReceiver; import com.wgzhao.addax.common.spi.Writer; import com.wgzhao.addax.common.util.Configuration; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.Table; +import org.apache.iceberg.*; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderFactory; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.orc.GenericOrcWriter; import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.io.DataWriter; -import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.*; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.*; @@ -268,39 +273,105 @@ private long doBatchInsert(final List writerBuffer) { builder.add(data); } + ImmutableList rows = builder.build(); String filepath = table.location() + "/" + UUID.randomUUID(); OutputFile file = table.io().newOutputFile(filepath); - DataWriter dataWriter = null; + if(table.spec()==null) { + DataWriter dataWriter = null; - if ("parquet".equals(fileFormat)) { - try { - dataWriter = Parquet.writeData(file).overwrite().forTable(table).createWriterFunc(GenericParquetWriter::buildWriter).build(); - } catch (IOException e) { - throw new RuntimeException(e); + if ("parquet".equals(fileFormat)) { + try { + dataWriter = Parquet.writeData(file).overwrite().forTable(table).createWriterFunc(GenericParquetWriter::buildWriter).build(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } else if ("orc".equals(fileFormat)) { + dataWriter = ORC.writeData(file).overwrite().forTable(table).createWriterFunc(GenericOrcWriter::buildWriter).build(); + } else { + throw new RuntimeException("不支持的文件格式:" + fileFormat); } - } else if ("orc".equals(fileFormat)) { - dataWriter = ORC.writeData(file).overwrite().forTable(table).createWriterFunc(GenericOrcWriter::buildWriter).build(); - } else { - throw new RuntimeException("不支持的文件格式:" + fileFormat); - } - ImmutableList rows = builder.build(); - if (dataWriter != null) { - dataWriter.write(rows); - } + + if (dataWriter != null) { + dataWriter.write(rows); + } - if (dataWriter != null) { + if (dataWriter != null) { + try { + dataWriter.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + DataFile dataFile = dataWriter.toDataFile(); + table.newAppend().appendFile(dataFile).commit(); + } else { + Map tableProps = Maps.newHashMap(table.properties()); + long targetFileSize = + PropertyUtil.propertyAsLong( + tableProps, + TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, + TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); + + int partitionId = 1, taskId = 1; + FileFormat fileFormat= FileFormat.PARQUET; + if("orc".equals(fileFormat)){ + fileFormat= FileFormat.ORC; + } + Set identifierFieldIds = table.schema().identifierFieldIds(); + FileAppenderFactory appenderFactory; + if (identifierFieldIds == null || identifierFieldIds.isEmpty()) { + appenderFactory = + new GenericAppenderFactory(table.schema(), table.spec(), null, null, null) + .setAll(tableProps); + } else { + appenderFactory = + new GenericAppenderFactory( + table.schema(), + table.spec(), + Ints.toArray(identifierFieldIds), + TypeUtil.select(table.schema(), Sets.newHashSet(identifierFieldIds)), + null) + .setAll(tableProps); + } + OutputFileFactory outputFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId).format(fileFormat).build(); + final PartitionKey partitionKey = new PartitionKey(table.spec(), table.spec().schema()); + // partitionedFanoutWriter will auto partitioned record and create the partitioned writer + PartitionedFanoutWriter partitionedFanoutWriter = new PartitionedFanoutWriter(table.spec(), fileFormat, appenderFactory, outputFileFactory, table.io(), targetFileSize) { + @Override + protected PartitionKey partition(org.apache.iceberg.data.Record record) { + partitionKey.partition(record); + return partitionKey; + } + }; + + rows.forEach( + row -> { + try { + partitionedFanoutWriter.write(row); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }); + try { + WriteResult writeResult =partitionedFanoutWriter.complete(); + + AppendFiles appends = table.newAppend(); + Arrays.stream(writeResult.dataFiles()).forEach(appends::appendFile); + appends.commit(); + + } catch (IOException e) { + throw new RuntimeException(e); + } try { - dataWriter.close(); + partitionedFanoutWriter.close(); } catch (IOException e) { throw new RuntimeException(e); } } - DataFile dataFile = dataWriter.toDataFile(); - table.newAppend().appendFile(dataFile).commit(); return rows.size(); } } From 34f7e6d6c345fc6b8c1a7cd7bcc38a66eb0d8df2 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Mon, 3 Mar 2025 13:05:28 +0800 Subject: [PATCH 27/40] [add][plugin][icebergwriter] fixed bug when partition iceberg table --- plugin/writer/icebergwriter/pom.xml | 63 +++++++++++++++++++ .../writer/icebergwriter/IcebergWriter.java | 3 + 2 files changed, 66 insertions(+) diff --git a/plugin/writer/icebergwriter/pom.xml b/plugin/writer/icebergwriter/pom.xml index f6cbe43b1..cbb8e9b6c 100644 --- a/plugin/writer/icebergwriter/pom.xml +++ b/plugin/writer/icebergwriter/pom.xml @@ -383,6 +383,69 @@ + + org.apache.iceberg + iceberg-data + ${iceberg.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.woodstox + woodstox-core + + + commons-codec + commons-codec + + + commons-net + commons-net + + + io.netty + netty + + + log4j + log4j + + + net.minidev + json-smart + + + org.codehaus.jettison + jettison + + + org.eclipse.jetty + jetty-server + + + org.xerial.snappy + snappy-java + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-util + + + org.apache.iceberg iceberg-aws diff --git a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java index f0731e2dd..240b7b251 100644 --- a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java +++ b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java @@ -2,6 +2,9 @@ import com.alibaba.fastjson2.JSON; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.common.primitives.Ints; import com.wgzhao.addax.common.element.Column; import com.wgzhao.addax.common.element.Record; import com.wgzhao.addax.common.plugin.RecordReceiver; From 323cf31482a01b01364e2968dd3b0d51ce05eb50 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Wed, 26 Mar 2025 16:00:45 +0800 Subject: [PATCH 28/40] merge master --- plugin/reader/hivereader/pom.xml | 100 +++++++++--------- plugin/writer/s3writer/pom.xml | 21 ++-- .../plugin/writer/s3writer/S3Writer.java | 2 +- .../s3writer/formatwriter/IFormatWriter.java | 6 +- .../s3writer/formatwriter/OrcWriter.java | 20 ++-- .../s3writer/formatwriter/ParquetWriter.java | 14 +-- .../s3writer/formatwriter/TextWriter.java | 14 +-- 7 files changed, 91 insertions(+), 86 deletions(-) diff --git a/plugin/reader/hivereader/pom.xml b/plugin/reader/hivereader/pom.xml index e2d47528b..7d5bbbbe3 100644 --- a/plugin/reader/hivereader/pom.xml +++ b/plugin/reader/hivereader/pom.xml @@ -47,55 +47,6 @@ avro - - org.apache.hive - hive-jdbc - ${hive.version} - - - org.apache.hive - hive-llap-server - - - org.apache.hbase - hbase-client - - - org.apache.hadoop - hadoop-yarn-api - - - org.apache.hadoop - hadoop-yarn-common - - - org.apache.hadoop - hadoop-yarn-server-resourcemanager - - - org.apache.hadoop - hadoop-yarn-server-common - - - org.apache.hadoop - hadoop-yarn-server-applicationhistoryservice - - - org.apache.hadoop - hadoop-yarn-server-web-proxy - - - org.apache.zookeeper - zookeeper - - - - - - org.xerial.snappy - snappy-java - - org.apache.curator curator-framework @@ -184,6 +135,57 @@ + + org.apache.hive + hive-jdbc + ${hive.version} + + + org.apache.hive + hive-llap-server + + + org.apache.hbase + hbase-client + + + org.apache.hadoop + hadoop-yarn-api + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-yarn-server-resourcemanager + + + org.apache.hadoop + hadoop-yarn-server-common + + + org.apache.hadoop + hadoop-yarn-server-applicationhistoryservice + + + org.apache.hadoop + hadoop-yarn-server-web-proxy + + + org.apache.zookeeper + zookeeper + + + + + + org.xerial.snappy + snappy-java + + + + diff --git a/plugin/writer/s3writer/pom.xml b/plugin/writer/s3writer/pom.xml index b184dcaf0..0ad62972b 100644 --- a/plugin/writer/s3writer/pom.xml +++ b/plugin/writer/s3writer/pom.xml @@ -27,10 +27,14 @@ - software.amazon.awssdk - s3 + com.fasterxml.woodstox + woodstox-core + ${woodstox.version} + + + com.google.protobuf protobuf-java @@ -48,10 +52,9 @@ - org.apache.hadoop - hadoop-common + hadoop-aws ${hadoop.version} @@ -112,10 +115,9 @@ - org.apache.hadoop - hadoop-aws + hadoop-common ${hadoop.version} @@ -177,6 +179,8 @@ + + org.apache.hadoop hadoop-mapreduce-client-core @@ -321,9 +325,8 @@ - com.fasterxml.woodstox - woodstox-core - ${woodstox.version} + software.amazon.awssdk + s3 diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java index c1853d4a4..8a6c7a6c8 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/S3Writer.java @@ -17,7 +17,7 @@ import java.util.*; -import static com.wgzhao.addax.common.spi.ErrorCode.*; +import static com.wgzhao.addax.core.spi.ErrorCode.*; public class S3Writer extends Writer { diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java index e021ad4b6..c46281345 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/IFormatWriter.java @@ -1,8 +1,8 @@ package com.wgzhao.addax.plugin.writer.s3writer.formatwriter; -import com.wgzhao.addax.common.plugin.RecordReceiver; -import com.wgzhao.addax.common.plugin.TaskPluginCollector; -import com.wgzhao.addax.common.util.Configuration; +import com.wgzhao.addax.core.plugin.RecordReceiver; +import com.wgzhao.addax.core.plugin.TaskPluginCollector; +import com.wgzhao.addax.core.util.Configuration; public interface IFormatWriter { void init(Configuration config); diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/OrcWriter.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/OrcWriter.java index 90a87fcea..94518d112 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/OrcWriter.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/OrcWriter.java @@ -1,14 +1,14 @@ package com.wgzhao.addax.plugin.writer.s3writer.formatwriter; -import com.wgzhao.addax.common.base.Constant; -import com.wgzhao.addax.common.base.Key; -import com.wgzhao.addax.common.element.Column; -import com.wgzhao.addax.common.element.DateColumn; -import com.wgzhao.addax.common.element.Record; -import com.wgzhao.addax.common.exception.AddaxException; -import com.wgzhao.addax.common.plugin.RecordReceiver; -import com.wgzhao.addax.common.plugin.TaskPluginCollector; -import com.wgzhao.addax.common.util.Configuration; +import com.wgzhao.addax.core.base.Constant; +import com.wgzhao.addax.core.base.Key; +import com.wgzhao.addax.core.element.Column; +import com.wgzhao.addax.core.element.DateColumn; +import com.wgzhao.addax.core.element.Record; +import com.wgzhao.addax.core.exception.AddaxException; +import com.wgzhao.addax.core.plugin.RecordReceiver; +import com.wgzhao.addax.core.plugin.TaskPluginCollector; +import com.wgzhao.addax.core.util.Configuration; import com.wgzhao.addax.plugin.writer.s3writer.S3Key; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.type.HiveDecimal; @@ -33,7 +33,7 @@ import java.util.List; import java.util.StringJoiner; -import static com.wgzhao.addax.common.spi.ErrorCode.*; +import static com.wgzhao.addax.core.spi.ErrorCode.*; public class OrcWriter implements IFormatWriter { diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/ParquetWriter.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/ParquetWriter.java index a455be265..361d77b92 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/ParquetWriter.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/ParquetWriter.java @@ -19,13 +19,13 @@ package com.wgzhao.addax.plugin.writer.s3writer.formatwriter; -import com.wgzhao.addax.common.base.Constant; -import com.wgzhao.addax.common.base.Key; -import com.wgzhao.addax.common.element.Column; -import com.wgzhao.addax.common.element.Record; -import com.wgzhao.addax.common.plugin.RecordReceiver; -import com.wgzhao.addax.common.plugin.TaskPluginCollector; -import com.wgzhao.addax.common.util.Configuration; +import com.wgzhao.addax.core.base.Constant; +import com.wgzhao.addax.core.base.Key; +import com.wgzhao.addax.core.element.Column; +import com.wgzhao.addax.core.element.Record; +import com.wgzhao.addax.core.plugin.RecordReceiver; +import com.wgzhao.addax.core.plugin.TaskPluginCollector; +import com.wgzhao.addax.core.util.Configuration; import com.wgzhao.addax.plugin.writer.s3writer.S3Key; import org.apache.avro.Conversions; import org.apache.avro.generic.GenericData; diff --git a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/TextWriter.java b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/TextWriter.java index bf4223380..b7a343c34 100644 --- a/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/TextWriter.java +++ b/plugin/writer/s3writer/src/main/java/com/wgzhao/addax/plugin/writer/s3writer/formatwriter/TextWriter.java @@ -1,11 +1,11 @@ package com.wgzhao.addax.plugin.writer.s3writer.formatwriter; -import com.wgzhao.addax.common.element.Column; -import com.wgzhao.addax.common.element.Record; -import com.wgzhao.addax.common.exception.AddaxException; -import com.wgzhao.addax.common.plugin.RecordReceiver; -import com.wgzhao.addax.common.plugin.TaskPluginCollector; -import com.wgzhao.addax.common.util.Configuration; +import com.wgzhao.addax.core.element.Column; +import com.wgzhao.addax.core.element.Record; +import com.wgzhao.addax.core.exception.AddaxException; +import com.wgzhao.addax.core.plugin.RecordReceiver; +import com.wgzhao.addax.core.plugin.TaskPluginCollector; +import com.wgzhao.addax.core.util.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.core.sync.RequestBody; @@ -20,7 +20,7 @@ import java.util.List; import java.util.StringJoiner; -import static com.wgzhao.addax.common.spi.ErrorCode.IO_ERROR; +import static com.wgzhao.addax.core.spi.ErrorCode.IO_ERROR; public class TextWriter implements IFormatWriter { private static final Logger LOG = LoggerFactory.getLogger(TextWriter.class); From 4c8cd9ce65c95cf98ccd472c43c28115566dc505 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Wed, 26 Mar 2025 16:52:40 +0800 Subject: [PATCH 29/40] merge master --- plugin/writer/icebergwriter/pom.xml | 107 +++++++++--------- .../writer/icebergwriter/IcebergHelper.java | 10 +- .../writer/icebergwriter/IcebergWriter.java | 10 +- 3 files changed, 63 insertions(+), 64 deletions(-) diff --git a/plugin/writer/icebergwriter/pom.xml b/plugin/writer/icebergwriter/pom.xml index cbb8e9b6c..363c7a259 100644 --- a/plugin/writer/icebergwriter/pom.xml +++ b/plugin/writer/icebergwriter/pom.xml @@ -5,7 +5,7 @@ com.wgzhao.addax addax-all - 4.2.4-SNAPSHOT + 5.0.1-SNAPSHOT ../../../pom.xml @@ -17,27 +17,38 @@ com.wgzhao.addax - addax-common - ${project.version} - - - slf4j-log4j12 - org.slf4j - - + addax-core com.wgzhao.addax addax-storage - ${project.version} - - + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + com.fasterxml.woodstox + woodstox-core + ${woodstox.version} + + + org.apache.avro + avro + ${avro.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.apache.hadoop - hadoop-common + hadoop-aws ${hadoop.version} @@ -98,10 +109,9 @@ - org.apache.hadoop - hadoop-aws + hadoop-common ${hadoop.version} @@ -163,6 +173,8 @@ + + org.apache.hadoop hadoop-mapreduce-client-core @@ -188,15 +200,12 @@ - - com.fasterxml.woodstox - woodstox-core - ${woodstox.version} - + + org.apache.iceberg - iceberg-common + iceberg-api ${iceberg.version} @@ -259,7 +268,7 @@ org.apache.iceberg - iceberg-api + iceberg-arrow ${iceberg.version} @@ -320,9 +329,10 @@ + org.apache.iceberg - iceberg-arrow + iceberg-aws ${iceberg.version} @@ -385,7 +395,7 @@ org.apache.iceberg - iceberg-data + iceberg-common ${iceberg.version} @@ -448,7 +458,7 @@ org.apache.iceberg - iceberg-aws + iceberg-core ${iceberg.version} @@ -511,7 +521,7 @@ org.apache.iceberg - iceberg-core + iceberg-data ${iceberg.version} @@ -574,7 +584,7 @@ org.apache.iceberg - iceberg-parquet + iceberg-hive-metastore ${iceberg.version} @@ -700,7 +710,7 @@ org.apache.iceberg - iceberg-hive-metastore + iceberg-parquet ${iceberg.version} @@ -761,10 +771,12 @@ + + - org.apache.parquet - parquet-hadoop-bundle - ${parquet.version} + org.apache.orc + orc-core + ${orc.version} com.fasterxml.jackson.core @@ -822,12 +834,17 @@ org.eclipse.jetty jetty-util + + org.apache.hadoop + hadoop-client-api + + - org.apache.orc - orc-core - ${orc.version} + org.apache.parquet + parquet-hadoop-bundle + ${parquet.version} com.fasterxml.jackson.core @@ -885,28 +902,10 @@ org.eclipse.jetty jetty-util - - org.apache.hadoop - hadoop-client-api - - - - - com.fasterxml.jackson.core - jackson-databind - ${jackson.version} - - - org.apache.avro - avro - ${avro.version} - - - com.fasterxml.jackson.core - jackson-databind - + + diff --git a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergHelper.java b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergHelper.java index e45d0db18..77da24ca7 100644 --- a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergHelper.java +++ b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergHelper.java @@ -1,7 +1,7 @@ package com.wgzhao.addax.plugin.writer.icebergwriter; -import com.wgzhao.addax.common.exception.AddaxException; -import com.wgzhao.addax.common.util.Configuration; +import com.wgzhao.addax.core.exception.AddaxException; +import com.wgzhao.addax.core.util.Configuration; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.iceberg.catalog.Catalog; @@ -13,9 +13,9 @@ import java.util.HashMap; import java.util.Map; -import static com.wgzhao.addax.common.base.Key.KERBEROS_KEYTAB_FILE_PATH; -import static com.wgzhao.addax.common.base.Key.KERBEROS_PRINCIPAL; -import static com.wgzhao.addax.common.spi.ErrorCode.LOGIN_ERROR; +import static com.wgzhao.addax.core.base.Key.KERBEROS_KEYTAB_FILE_PATH; +import static com.wgzhao.addax.core.base.Key.KERBEROS_PRINCIPAL; +import static com.wgzhao.addax.core.spi.ErrorCode.LOGIN_ERROR; public class IcebergHelper { private static final Logger LOG = LoggerFactory.getLogger(IcebergHelper.class); diff --git a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java index f7a66a2a0..dc5ecf7c6 100644 --- a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java +++ b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java @@ -5,11 +5,11 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.primitives.Ints; -import com.wgzhao.addax.common.element.Column; -import com.wgzhao.addax.common.element.Record; -import com.wgzhao.addax.common.plugin.RecordReceiver; -import com.wgzhao.addax.common.spi.Writer; -import com.wgzhao.addax.common.util.Configuration; +import com.wgzhao.addax.core.element.Column; +import com.wgzhao.addax.core.element.Record; +import com.wgzhao.addax.core.plugin.RecordReceiver; +import com.wgzhao.addax.core.spi.Writer; +import com.wgzhao.addax.core.util.Configuration; import org.apache.iceberg.*; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; From a83e9ece9e1b8f0e7c5ee46ebcf05bacaeb46e74 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 15 May 2025 15:31:41 +0800 Subject: [PATCH 30/40] merge master --- package.xml | 2 +- plugin/reader/hivereader/pom.xml | 105 ++++------------------------ plugin/writer/icebergwriter/pom.xml | 2 +- 3 files changed, 16 insertions(+), 93 deletions(-) diff --git a/package.xml b/package.xml index 48e15e221..bb795593c 100644 --- a/package.xml +++ b/package.xml @@ -446,7 +446,7 @@ **/addax-common*.jar 0644 - addax-${project.version} + / plugin/writer/influxdbwriter/target/influxdbwriter-${project.version}/ diff --git a/plugin/reader/hivereader/pom.xml b/plugin/reader/hivereader/pom.xml index ec3c24060..d8a2c6195 100644 --- a/plugin/reader/hivereader/pom.xml +++ b/plugin/reader/hivereader/pom.xml @@ -38,29 +38,9 @@ 2.6 - - org.apache.hadoop - hadoop-common - - - * - * - - - - - org.apache.hive - hive-common - ${hive.version} - runtime - - - * - * - - - + + org.apache.curator @@ -73,79 +53,25 @@ curator-recipes ${curator.version} - org.apache.hadoop hadoop-common - ${hadoop.version} - com.fasterxml.jackson.core - jackson-databind - - - org.codehaus.jackson - jackson-core-asl - - - org.codehaus.jackson - jackson-mapper-asl - - - com.fasterxml.woodstox - woodstox-core - - - commons-codec - commons-codec - - - commons-net - commons-net - - - io.netty - netty - - - log4j - log4j - - - net.minidev - json-smart - - - org.codehaus.jettison - jettison - - - org.eclipse.jetty - jetty-server - - - com.google.protobuf - protobuf-java - - - com.google.guava - guava - - - org.apache.avro - avro - - - org.xerial.snappy - snappy-java - - - org.apache.zookeeper - zookeeper + * + * + + + + org.apache.hive + hive-common + ${hive.version} + runtime + - org.apache.curator - curator-client + * + * @@ -223,9 +149,6 @@ - - - diff --git a/plugin/writer/icebergwriter/pom.xml b/plugin/writer/icebergwriter/pom.xml index 363c7a259..46724eedd 100644 --- a/plugin/writer/icebergwriter/pom.xml +++ b/plugin/writer/icebergwriter/pom.xml @@ -5,7 +5,7 @@ com.wgzhao.addax addax-all - 5.0.1-SNAPSHOT + 5.1.3-SNAPSHOT ../../../pom.xml From 408fa57c5cceca4de227a5287fbe7e2960cd825a Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 15 May 2025 16:21:08 +0800 Subject: [PATCH 31/40] [bugfix][writer][icebergwriter] fix dependency errors --- plugin/writer/icebergwriter/pom.xml | 66 ++++++++++++++++++++++++++++- 1 file changed, 65 insertions(+), 1 deletion(-) diff --git a/plugin/writer/icebergwriter/pom.xml b/plugin/writer/icebergwriter/pom.xml index 46724eedd..d15bef2c8 100644 --- a/plugin/writer/icebergwriter/pom.xml +++ b/plugin/writer/icebergwriter/pom.xml @@ -24,17 +24,31 @@ com.wgzhao.addax addax-storage - + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + com.fasterxml.jackson.core jackson-databind ${jackson.version} + + + com.fasterxml.jackson.core + jackson-core + + com.fasterxml.woodstox woodstox-core ${woodstox.version} + + com.google.protobuf + protobuf-java + org.apache.avro avro @@ -44,6 +58,10 @@ com.fasterxml.jackson.core jackson-databind + + com.fasterxml.jackson.core + jackson-core + @@ -55,6 +73,10 @@ com.fasterxml.jackson.core jackson-databind + + com.fasterxml.jackson.core + jackson-core + org.codehaus.jackson jackson-core-asl @@ -184,6 +206,10 @@ com.fasterxml.jackson.core jackson-databind + + com.fasterxml.jackson.core + jackson-core + commons-codec commons-codec @@ -212,6 +238,10 @@ com.fasterxml.jackson.core jackson-databind + + com.fasterxml.jackson.core + jackson-core + org.codehaus.jackson jackson-core-asl @@ -275,6 +305,10 @@ com.fasterxml.jackson.core jackson-databind + + com.fasterxml.jackson.core + jackson-core + org.codehaus.jackson jackson-core-asl @@ -339,6 +373,10 @@ com.fasterxml.jackson.core jackson-databind + + com.fasterxml.jackson.core + jackson-core + org.codehaus.jackson jackson-core-asl @@ -402,6 +440,10 @@ com.fasterxml.jackson.core jackson-databind + + com.fasterxml.jackson.core + jackson-core + org.codehaus.jackson jackson-core-asl @@ -465,6 +507,10 @@ com.fasterxml.jackson.core jackson-databind + + com.fasterxml.jackson.core + jackson-core + org.codehaus.jackson jackson-core-asl @@ -528,6 +574,10 @@ com.fasterxml.jackson.core jackson-databind + + com.fasterxml.jackson.core + jackson-core + org.codehaus.jackson jackson-core-asl @@ -591,6 +641,10 @@ com.fasterxml.jackson.core jackson-databind + + com.fasterxml.jackson.core + jackson-core + org.codehaus.jackson jackson-core-asl @@ -654,6 +708,10 @@ com.fasterxml.jackson.core jackson-databind + + com.fasterxml.jackson.core + jackson-core + org.codehaus.jackson jackson-core-asl @@ -717,6 +775,10 @@ com.fasterxml.jackson.core jackson-databind + + com.fasterxml.jackson.core + jackson-core + org.codehaus.jackson jackson-core-asl @@ -906,6 +968,8 @@ + + From 104dfae567045c7adeb395187abc95ad01e80fde Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Fri, 16 May 2025 11:57:22 +0800 Subject: [PATCH 32/40] [bugfix][writer][icebergwriter] remove profile for the jdk is upgrade to version 17 --- pom.xml | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/pom.xml b/pom.xml index 904914cdb..c6ce25029 100644 --- a/pom.xml +++ b/pom.xml @@ -112,6 +112,7 @@ 3.2.1 351 5.0.1 + 1.8.0 UTF-8 @@ -183,6 +184,7 @@ plugin/writer/hbase11xwriter plugin/writer/hbase20xsqlwriter plugin/writer/hdfswriter + plugin/writer/icebergwriter plugin/writer/influxdb2writer plugin/writer/influxdbwriter plugin/writer/kafkawriter @@ -1493,21 +1495,5 @@ - - - iceberg - - - 1.8.0 - 11 - 11 - 11 - - - - - plugin/writer/icebergwriter - - From 0e29fda4742bd7b8e5c12ee1a4988f1765e57d41 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Mon, 19 May 2025 11:46:05 +0800 Subject: [PATCH 33/40] [bugfix][writer][s3writer] fix dependency error --- plugin/writer/s3writer/pom.xml | 13 +++++++++++++ pom.xml | 1 + 2 files changed, 14 insertions(+) diff --git a/plugin/writer/s3writer/pom.xml b/plugin/writer/s3writer/pom.xml index 3f34c2f86..440877990 100644 --- a/plugin/writer/s3writer/pom.xml +++ b/plugin/writer/s3writer/pom.xml @@ -52,6 +52,11 @@ + + org.apache.commons + commons-text + ${commons.text.version} + org.apache.hadoop hadoop-aws @@ -290,6 +295,12 @@ org.apache.orc orc-core ${orc.version} + + + org.apache.hadoop + hadoop-client-api + + @@ -329,6 +340,8 @@ s3 + + diff --git a/pom.xml b/pom.xml index c6ce25029..1a32af72f 100644 --- a/pom.xml +++ b/pom.xml @@ -45,6 +45,7 @@ 1.3.5 3.11.1 1.6 + 1.13.1 4.2.0 3.7.3 2.13.0 From 064e1b9255732d0569c034a298dbf0bb6051e365 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Mon, 19 May 2025 11:57:29 +0800 Subject: [PATCH 34/40] [add][writer][s3writer] add orc ,parquet file type support --- docs/writer/s3writer.md | 36 ++++++++++++++++++------------------ 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/docs/writer/s3writer.md b/docs/writer/s3writer.md index e759b5384..0c431016d 100644 --- a/docs/writer/s3writer.md +++ b/docs/writer/s3writer.md @@ -15,21 +15,24 @@ html) 编写。 ## 参数说明 -| 配置项 | 是否必须 | 数据类型 | 默认值 | 描述 | -| :------------- | :------: | -------- | -------- | ----------------------------------------------------- | -| endpoint | 是 | string | 无 | S3 Server的 EndPoint地址,例如 `s3.xx.amazonaws.com` | -| region | 是 | string | 无 | S3 Server的 Region 地址,例如 `ap-southeast-1` | -| accessId | 是 | string | 无 | 访问 ID | -| accessKey | 是 | string | 无 | 访问 Key | -| bucket | 是 | string | 无 | 要写入的 bucket | -| object | 是 | string | 无 | 要写入的 object,注意事项见下 | -| fieldDelimiter | 否 | char | `','` | 字段的分隔符 | -| nullFormat | 否 | char | `\N` | 当值为空时,用什么字符表示 | -| header | 否 | list | 无 | 写入文件头信息,比如 `["id","title","url"]` | -| maxFileSize | 否 | int | `100000` | 单个 object 的大小,单位为 MB | -| encoding | 否 | string | `utf-8` | 文件编码格式 | -| writeMode | 否 | string | `append` | 写入模式,详见 [hdfswriter](../hdfswriter) 中相关描述 | - +| 配置项 | 是否必须 | 数据类型 | 默认值 | 描述 | +|:-----------------------| :------: |--------|----------|------------------------------------------------| +| endpoint | 是 | string | 无 | S3 Server的 EndPoint地址,例如 `s3.xx.amazonaws.com` | +| region | 是 | string | 无 | S3 Server的 Region 地址,例如 `ap-southeast-1` | +| accessId | 是 | string | 无 | 访问 ID | +| accessKey | 是 | string | 无 | 访问 Key | +| bucket | 是 | string | 无 | 要写入的 bucket | +| object | 是 | string | 无 | 要写入的 object,注意事项见下 | +| fieldDelimiter | 否 | char | `','` | 字段的分隔符 | +| nullFormat | 否 | char | `\N` | 当值为空时,用什么字符表示 | +| header | 否 | list | 无 | 写入文件头信息,比如 `["id","title","url"]` | +| maxFileSize | 否 | int | `100000` | 单个 object 的大小,单位为 MB | +| encoding | 否 | string | `utf-8` | 文件编码格式 | +| writeMode | 否 | string | `append` | 写入模式,详见 [hdfswriter](../hdfswriter) 中相关描述 | +| pathStyleAccessEnabled | 否 | bool | false | 是否使用path access方式访问 | +| sslEnabled | 否 | bool | false | 是否使用ssl方式访问 | +| fileType | 否 | string | `text` | 文件类型 text, orc ,parquet | +| compress | 否 | string | `NONE` | orc或parquet文件的压缩方式,默认不压缩 | ### object 上述配置中的 `object` 配置的虽然是 `upload.csv` 文件,实际上在 S3 写入的文件名会在指定的文件名后面加上 `uuid` 后缀, @@ -47,7 +50,4 @@ html) 编写。 | Boolean | bit, bool | | Bytes | tinyblob, mediumblob, blob, longblob, varbinary | -## 限制说明 -1. 仅支持写入文本文件格式 -2. 暂不支持压缩 From 3b72c4d4e0e767dea244a9cef1f618d82c398b4b Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Thu, 5 Jun 2025 15:32:39 +0800 Subject: [PATCH 35/40] [improve][writer][kuduwriter] add kerberos support --- plugin/writer/kuduwriter/pom.xml | 14 +++- .../plugin/writer/kuduwriter/KuduHelper.java | 75 ++++++++++--------- .../plugin/writer/kuduwriter/KuduWriter.java | 16 +++- .../writer/kuduwriter/KuduWriterTask.java | 68 ++++++++--------- 4 files changed, 102 insertions(+), 71 deletions(-) diff --git a/plugin/writer/kuduwriter/pom.xml b/plugin/writer/kuduwriter/pom.xml index 61b5c2968..626452730 100644 --- a/plugin/writer/kuduwriter/pom.xml +++ b/plugin/writer/kuduwriter/pom.xml @@ -1,5 +1,6 @@ - + 4.0.0 @@ -25,6 +26,15 @@ com.alibaba.fastjson2 fastjson2 + + com.fasterxml.woodstox + woodstox-core + + + + org.apache.hadoop + hadoop-common + org.apache.kudu @@ -37,7 +47,7 @@ - org.apache.maven.plugins + org.apache.maven.plugins maven-assembly-plugin diff --git a/plugin/writer/kuduwriter/src/main/java/com/wgzhao/addax/plugin/writer/kuduwriter/KuduHelper.java b/plugin/writer/kuduwriter/src/main/java/com/wgzhao/addax/plugin/writer/kuduwriter/KuduHelper.java index bcc664dea..d675d69ed 100644 --- a/plugin/writer/kuduwriter/src/main/java/com/wgzhao/addax/plugin/writer/kuduwriter/KuduHelper.java +++ b/plugin/writer/kuduwriter/src/main/java/com/wgzhao/addax/plugin/writer/kuduwriter/KuduHelper.java @@ -20,6 +20,8 @@ package com.wgzhao.addax.plugin.writer.kuduwriter; import com.wgzhao.addax.core.exception.AddaxException; +import com.wgzhao.addax.core.util.Configuration; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.kudu.ColumnSchema; import org.apache.kudu.Schema; import org.apache.kudu.client.KuduClient; @@ -29,90 +31,96 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.List; +import static com.wgzhao.addax.core.base.Key.*; import static com.wgzhao.addax.core.spi.ErrorCode.CONNECT_ERROR; import static com.wgzhao.addax.core.spi.ErrorCode.RUNTIME_ERROR; -public class KuduHelper -{ +public class KuduHelper { private static final Logger LOG = LoggerFactory.getLogger(KuduHelper.class); private final KuduClient kuduClient; private KuduTable kuduTable; - public KuduHelper(String masterAddress) - { - this(masterAddress, 100 * 1000L); + public KuduHelper(Configuration config, String masterAddress) { + this(config, masterAddress, 100 * 1000L); } - public KuduHelper(String masterAddress, long timeout) - { + public KuduHelper(Configuration config, String masterAddress, long timeout) { try { - this.kuduClient = new KuduClient.KuduClientBuilder(masterAddress) - .defaultOperationTimeoutMs(timeout) - .build(); - } - catch (Exception e) { + boolean haveKerberos = config.getBool(HAVE_KERBEROS, false); + + if (!haveKerberos) { + this.kuduClient = new KuduClient.KuduClientBuilder(masterAddress) + .defaultOperationTimeoutMs(timeout) + .build(); + } else { + org.apache.hadoop.conf.Configuration configuration = new org.apache.hadoop.conf.Configuration(); + UserGroupInformation.setConfiguration(configuration); + + String kerberosKeytabFilePath = config.getString(KERBEROS_KEYTAB_FILE_PATH); + String kerberosPrincipal = config.getString(KERBEROS_PRINCIPAL); + UserGroupInformation.loginUserFromKeytab(kerberosPrincipal, kerberosKeytabFilePath); + this.kuduClient = UserGroupInformation.getLoginUser().doAs( + new PrivilegedExceptionAction() { + @Override + public KuduClient run() throws Exception { + return new KuduClient.KuduClientBuilder(masterAddress).defaultOperationTimeoutMs(timeout).build(); + } + }); + } + } catch (Exception e) { throw AddaxException.asAddaxException(CONNECT_ERROR, e); } } - public KuduTable getKuduTable(String tableName) - { + public KuduTable getKuduTable(String tableName) { if (tableName == null) { return null; - } - else { + } else { try { kuduTable = kuduClient.openTable(tableName); return kuduTable; - } - catch (KuduException e) { + } catch (KuduException e) { throw AddaxException.asAddaxException(RUNTIME_ERROR, e); } } } - public boolean isTableExists(String tableName) - { + public boolean isTableExists(String tableName) { if (tableName == null) { return false; } try { return kuduClient.tableExists(tableName); - } - catch (KuduException e) { + } catch (KuduException e) { throw AddaxException.asAddaxException(RUNTIME_ERROR, e); } } - public void closeClient() - { + public void closeClient() { try { if (kuduClient != null) { kuduClient.close(); } - } - catch (KuduException e) { + } catch (KuduException e) { LOG.warn("The kudu client was not stopped gracefully. !"); } } - public Schema getSchema(String tableName) - { + public Schema getSchema(String tableName) { if (kuduTable != null) { return kuduTable.getSchema(); - } - else { + } else { kuduTable = getKuduTable(tableName); return kuduTable.getSchema(); } } - public List getAllColumns(String tableName) - { + public List getAllColumns(String tableName) { List columns = new ArrayList<>(); Schema schema = getSchema(tableName); for (ColumnSchema column : schema.getColumns()) { @@ -121,8 +129,7 @@ public List getAllColumns(String tableName) return columns; } - public KuduSession getSession() - { + public KuduSession getSession() { return kuduClient.newSession(); } } diff --git a/plugin/writer/kuduwriter/src/main/java/com/wgzhao/addax/plugin/writer/kuduwriter/KuduWriter.java b/plugin/writer/kuduwriter/src/main/java/com/wgzhao/addax/plugin/writer/kuduwriter/KuduWriter.java index 54ad96c49..21d91a12c 100644 --- a/plugin/writer/kuduwriter/src/main/java/com/wgzhao/addax/plugin/writer/kuduwriter/KuduWriter.java +++ b/plugin/writer/kuduwriter/src/main/java/com/wgzhao/addax/plugin/writer/kuduwriter/KuduWriter.java @@ -19,6 +19,7 @@ package com.wgzhao.addax.plugin.writer.kuduwriter; +import com.wgzhao.addax.core.base.Key; import com.wgzhao.addax.core.exception.AddaxException; import com.wgzhao.addax.core.plugin.RecordReceiver; import com.wgzhao.addax.core.spi.Writer; @@ -63,7 +64,7 @@ private void validateParameter() this.config.set(KuduKey.KUDU_TIMEOUT, timeout); LOG.info("Try to connect kudu with {}", masterAddress); - KuduHelper kuduHelper = new KuduHelper(masterAddress, timeout); + KuduHelper kuduHelper = new KuduHelper(this.config,masterAddress, timeout); // check table exists or not if (!kuduHelper.isTableExists(tableName)) { throw AddaxException.asAddaxException(CONFIG_ERROR, "table '" + tableName + "' does not exists"); @@ -95,6 +96,19 @@ private void validateParameter() // writeMode check String writeMode = this.config.getString(KuduKey.WRITE_MODE, INSERT_MODE); this.config.set(KuduKey.WRITE_MODE, writeMode); + + + //Kerberos check + validateKerberos(); + } + + private void validateKerberos() + { + boolean haveKerberos = this.config.getBool(Key.HAVE_KERBEROS, false); + if (haveKerberos) { + this.config.getNecessaryValue(Key.KERBEROS_KEYTAB_FILE_PATH, REQUIRED_VALUE); + this.config.getNecessaryValue(Key.KERBEROS_PRINCIPAL, REQUIRED_VALUE); + } } @Override diff --git a/plugin/writer/kuduwriter/src/main/java/com/wgzhao/addax/plugin/writer/kuduwriter/KuduWriterTask.java b/plugin/writer/kuduwriter/src/main/java/com/wgzhao/addax/plugin/writer/kuduwriter/KuduWriterTask.java index 0feb52aad..8b4583650 100644 --- a/plugin/writer/kuduwriter/src/main/java/com/wgzhao/addax/plugin/writer/kuduwriter/KuduWriterTask.java +++ b/plugin/writer/kuduwriter/src/main/java/com/wgzhao/addax/plugin/writer/kuduwriter/KuduWriterTask.java @@ -28,13 +28,7 @@ import com.wgzhao.addax.core.util.Configuration; import org.apache.kudu.Schema; import org.apache.kudu.Type; -import org.apache.kudu.client.Insert; -import org.apache.kudu.client.KuduException; -import org.apache.kudu.client.KuduSession; -import org.apache.kudu.client.KuduTable; -import org.apache.kudu.client.PartialRow; -import org.apache.kudu.client.SessionConfiguration; -import org.apache.kudu.client.Upsert; +import org.apache.kudu.client.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,27 +38,23 @@ import java.util.List; import static com.wgzhao.addax.core.base.Constant.DEFAULT_BATCH_SIZE; -import static com.wgzhao.addax.core.spi.ErrorCode.CONFIG_ERROR; -import static com.wgzhao.addax.core.spi.ErrorCode.NOT_SUPPORT_TYPE; -import static com.wgzhao.addax.core.spi.ErrorCode.RUNTIME_ERROR; +import static com.wgzhao.addax.core.spi.ErrorCode.*; public class KuduWriterTask - extends Writer -{ + extends Writer { private static final Logger LOG = LoggerFactory.getLogger(KuduWriterTask.class); - public KuduSession session; private final Double batchSize; private final List columns; private final Boolean isUpsert; private final Boolean isSkipFail; private final KuduTable table; private final KuduHelper kuduHelper; + public KuduSession session; - public KuduWriterTask(Configuration configuration) - { + public KuduWriterTask(Configuration configuration) { String masterAddress = configuration.getString(KuduKey.KUDU_MASTER_ADDRESSES); - this.kuduHelper = new KuduHelper(masterAddress, configuration.getLong(KuduKey.KUDU_TIMEOUT)); + this.kuduHelper = new KuduHelper(configuration, masterAddress, configuration.getLong(KuduKey.KUDU_TIMEOUT)); this.columns = configuration.getList(KuduKey.COLUMN, String.class); this.batchSize = configuration.getDouble(KuduKey.BATCH_SIZE, DEFAULT_BATCH_SIZE); @@ -79,13 +69,14 @@ public KuduWriterTask(Configuration configuration) session.setMutationBufferSpace((int) mutationBufferSpace); } - public void startWriter(RecordReceiver lineReceiver, TaskPluginCollector taskPluginCollector) - { + public void startWriter(RecordReceiver lineReceiver, TaskPluginCollector taskPluginCollector) { LOG.info("Begin to write"); Record record; + Record lastRecord = null; int commit = 0; final Schema schema = this.table.getSchema(); while ((record = lineReceiver.getFromReader()) != null) { + lastRecord = record; if (record.getColumnNumber() != columns.size()) { throw AddaxException.asAddaxException(CONFIG_ERROR, "The number of record fields (" + record.getColumnNumber() @@ -97,8 +88,7 @@ public void startWriter(RecordReceiver lineReceiver, TaskPluginCollector taskPlu if (isUpsert) { //override update row = upsert.getRow(); - } - else { + } else { //incremental update row = insert.getRow(); } @@ -154,23 +144,27 @@ public void startWriter(RecordReceiver lineReceiver, TaskPluginCollector taskPlu try { if (isUpsert) { session.apply(upsert); - } - else { + } else { session.apply(insert); } commit++; if (commit % batchSize == 0) { // flush - session.flush(); + List operationResponseList = session.flush(); + for (OperationResponse operationResponse : operationResponseList) { + if (operationResponse.hasRowError()) { + throw AddaxException.asAddaxException(RUNTIME_ERROR, operationResponse.getRowError().getErrorStatus().toString()); + } + } } - } - catch (KuduException e) { + } catch (KuduException e) { LOG.error("Failed to write a record: ", e); + if (isSkipFail) { - LOG.warn("Since you have configured 'skipFail' to be true, this record will be skipped."); taskPluginCollector.collectDirtyRecord(record, e.getMessage()); - } - else { + LOG.warn("Since you have configured 'skipFail' to be true, this record will be skipped."); + + } else { throw AddaxException.asAddaxException(RUNTIME_ERROR, e.getMessage()); } } @@ -178,15 +172,21 @@ public void startWriter(RecordReceiver lineReceiver, TaskPluginCollector taskPlu try { // try to flush last upsert/insert - session.flush(); - } - catch (KuduException e) { + List operationResponseList = session.flush(); + for (OperationResponse operationResponse : operationResponseList) { + if (operationResponse.hasRowError()) { + throw AddaxException.asAddaxException(RUNTIME_ERROR, operationResponse.getRowError().getErrorStatus().toString()); + } + + } + } catch (KuduException e) { LOG.error("Failed to write a record: ", e); + if (isSkipFail) { + taskPluginCollector.collectDirtyRecord(lastRecord, e.getMessage()); LOG.warn("Since you have configured 'skipFail' to be true, this record will be skipped !"); - taskPluginCollector.collectDirtyRecord(record, e.getMessage()); - } - else { + + } else { throw AddaxException.asAddaxException(RUNTIME_ERROR, e.getMessage()); } } From 291d5ca84cb5ac2f9338854a715bc91a9e62e8b7 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Fri, 13 Jun 2025 10:57:15 +0800 Subject: [PATCH 36/40] [improve][writer][kudureader] add kerberos support --- docs/reader/kudureader.md | 3 ++ docs/writer/kuduwriter.md | 3 ++ plugin/reader/kudureader/pom.xml | 10 ++++-- .../plugin/reader/kudureader/KuduReader.java | 32 +++++++++++++++++-- .../main/resources/plugin_job_template.json | 5 ++- .../main/resources/plugin_job_template.json | 5 ++- 6 files changed, 51 insertions(+), 7 deletions(-) diff --git a/docs/reader/kudureader.md b/docs/reader/kudureader.md index 8d9273bd2..f2055888a 100644 --- a/docs/reader/kudureader.md +++ b/docs/reader/kudureader.md @@ -45,6 +45,9 @@ bin/addax.sh job/kudu2stream.json | scanTimeout | 否 | int | 20 | 数据扫描请求超时(秒) | | column | 否 | list | 无 | 指定要获取的字段 | | where | 否 | list | 无 | 指定其他过滤条件,详见下面描述 | +| haveKerberos | 否 | boolean | false | 是否启用 Kerberos 认证,如果启用,则需要同时配置以下两项 | +| kerberosKeytabFilePath | 否 | string | 无 | 用于 Kerberos 认证的凭证文件路径, 比如 `/your/path/addax.service.keytab` | +| kerberosPrincipal | 否 | string | 无 | 用于 Kerberos 认证的凭证主体, 比如 `addax/node1@WGZHAO.COM` | ### where diff --git a/docs/writer/kuduwriter.md b/docs/writer/kuduwriter.md index d7deb9aa1..762c62c49 100644 --- a/docs/writer/kuduwriter.md +++ b/docs/writer/kuduwriter.md @@ -50,6 +50,9 @@ bin/addax.sh job/stream2kudu.json | timeout | 否 | int | 100 | 写入数据超时时间(秒), 0 表示不受限制 | | column | 是 | list | 无 | 要写入的表字段,配置方式见上示例 | | skipFail | 否 | boolean | false | 是否跳过插入失败的记录,如果设置为true,则插件不会把插入失败的当作异常 | +| haveKerberos | 否 | boolean | false | 是否启用 Kerberos 认证,如果启用,则需要同时配置以下两项 | +| kerberosKeytabFilePath | 否 | string | 无 | 用于 Kerberos 认证的凭证文件路径, 比如 `/your/path/addax.service.keytab` | +| kerberosPrincipal | 否 | string | 无 | 用于 Kerberos 认证的凭证主体, 比如 `addax/node1@WGZHAO.COM` | ## column diff --git a/plugin/reader/kudureader/pom.xml b/plugin/reader/kudureader/pom.xml index ee940031b..3d0372a32 100644 --- a/plugin/reader/kudureader/pom.xml +++ b/plugin/reader/kudureader/pom.xml @@ -31,12 +31,18 @@ ch.qos.logback logback-classic - + + com.fasterxml.woodstox + woodstox-core + com.google.guava guava - + + org.apache.hadoop + hadoop-common + org.apache.kudu diff --git a/plugin/reader/kudureader/src/main/java/com/wgzhao/addax/plugin/reader/kudureader/KuduReader.java b/plugin/reader/kudureader/src/main/java/com/wgzhao/addax/plugin/reader/kudureader/KuduReader.java index 106077ea3..86a2a731f 100644 --- a/plugin/reader/kudureader/src/main/java/com/wgzhao/addax/plugin/reader/kudureader/KuduReader.java +++ b/plugin/reader/kudureader/src/main/java/com/wgzhao/addax/plugin/reader/kudureader/KuduReader.java @@ -30,6 +30,7 @@ import com.wgzhao.addax.core.plugin.RecordSender; import com.wgzhao.addax.core.spi.Reader; import com.wgzhao.addax.core.util.Configuration; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.kudu.ColumnSchema; import org.apache.kudu.Schema; import org.apache.kudu.Type; @@ -41,8 +42,10 @@ import org.apache.kudu.client.RowResult; import org.apache.kudu.client.RowResultIterator; +import java.io.IOException; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; +import java.security.PrivilegedExceptionAction; import java.sql.Date; import java.text.ParseException; import java.text.SimpleDateFormat; @@ -55,6 +58,9 @@ import static com.wgzhao.addax.core.base.Constant.DEFAULT_DATE_FORMAT; import static com.wgzhao.addax.core.base.Key.COLUMN; +import static com.wgzhao.addax.core.base.Key.HAVE_KERBEROS; +import static com.wgzhao.addax.core.base.Key.KERBEROS_KEYTAB_FILE_PATH; +import static com.wgzhao.addax.core.base.Key.KERBEROS_PRINCIPAL; import static com.wgzhao.addax.core.base.Key.WHERE; import static com.wgzhao.addax.core.spi.ErrorCode.CONFIG_ERROR; import static com.wgzhao.addax.core.spi.ErrorCode.ILLEGAL_VALUE; @@ -211,9 +217,29 @@ public void init() long socketReadTimeoutMs = readerSliceConfig.getLong(KuduKey.SOCKET_READ_TIMEOUT, 10) * 1000L; this.scanRequestTimeout = readerSliceConfig.getLong(KuduKey.SCAN_REQUEST_TIMEOUT, 20L) * 1000L; - this.kuduClient = new KuduClient.KuduClientBuilder(masterAddresses) - .defaultOperationTimeoutMs(socketReadTimeoutMs) - .build(); + boolean haveKerberos = readerSliceConfig.getBool(HAVE_KERBEROS, false); + + if (!haveKerberos) { + this.kuduClient = new KuduClient.KuduClientBuilder(masterAddresses) + .defaultOperationTimeoutMs(socketReadTimeoutMs) + .build(); + } + else { + org.apache.hadoop.conf.Configuration configuration = new org.apache.hadoop.conf.Configuration(); + UserGroupInformation.setConfiguration(configuration); + + String kerberosKeytabFilePath = readerSliceConfig.getString(KERBEROS_KEYTAB_FILE_PATH); + String kerberosPrincipal = readerSliceConfig.getString(KERBEROS_PRINCIPAL); + try { + UserGroupInformation.loginUserFromKeytab(kerberosPrincipal, kerberosKeytabFilePath); + this.kuduClient = UserGroupInformation.getLoginUser().doAs( + (PrivilegedExceptionAction) () -> + new KuduClient.KuduClientBuilder(masterAddresses).defaultOperationTimeoutMs(socketReadTimeoutMs).build()); + } + catch (IOException | InterruptedException e) { + throw new RuntimeException(e); + } + } this.lowerBound = readerSliceConfig.getString(KuduKey.SPLIT_LOWER_BOUND); this.upperBound = readerSliceConfig.getString(KuduKey.SPLIT_UPPER_BOUND); diff --git a/plugin/reader/kudureader/src/main/resources/plugin_job_template.json b/plugin/reader/kudureader/src/main/resources/plugin_job_template.json index bff5aaa6b..2a24e0d14 100644 --- a/plugin/reader/kudureader/src/main/resources/plugin_job_template.json +++ b/plugin/reader/kudureader/src/main/resources/plugin_job_template.json @@ -12,6 +12,9 @@ "col1", "col2", "col3" - ] + ], + "haveKerberos": "true", + "kerberosPrincipal": "hive@COMPANY.COM", + "kerberosKeytabFilePath": "/etc/security/keytabs/hive.headless.keytab" } } diff --git a/plugin/writer/kuduwriter/src/main/resources/plugin_job_template.json b/plugin/writer/kuduwriter/src/main/resources/plugin_job_template.json index ca513093e..e8c779aa9 100644 --- a/plugin/writer/kuduwriter/src/main/resources/plugin_job_template.json +++ b/plugin/writer/kuduwriter/src/main/resources/plugin_job_template.json @@ -13,6 +13,9 @@ "batchSize": 1024, "bufferSize": 2048, "skipFail": false, - "encoding": "UTF-8" + "encoding": "UTF-8", + "haveKerberos": "true", + "kerberosPrincipal": "hive@COMPANY.COM", + "kerberosKeytabFilePath": "/etc/security/keytabs/hive.headless.keytab" } } \ No newline at end of file From a5f9ea17163e0b445f655399eea0e10a16de595c Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Fri, 20 Jun 2025 13:35:00 +0800 Subject: [PATCH 37/40] [bugfix][plugin][icebergwriter] fix error when write orc format table --- .../writer/icebergwriter/IcebergWriter.java | 19 ++++++++++++++----- pom.xml | 4 ++-- 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java index a04bc335e..bcd3c2271 100644 --- a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java +++ b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java @@ -188,6 +188,7 @@ public void init() schema = table.schema(); fileFormat = table.properties().get("write.format.default"); + log.info("fileFormat: {}", fileFormat); if (fileFormat == null || fileFormat.trim().isEmpty()) { fileFormat = "parquet"; } @@ -326,6 +327,7 @@ private long doBatchInsert(final List writerBuffer) DataWriter dataWriter = null; if ("parquet".equals(fileFormat)) { + log.info("start writing fileFormat: {}", fileFormat); try { dataWriter = Parquet.writeData(file).overwrite().forTable(table).createWriterFunc(GenericParquetWriter::buildWriter).build(); } @@ -334,7 +336,14 @@ private long doBatchInsert(final List writerBuffer) } } else if ("orc".equals(fileFormat)) { - dataWriter = ORC.writeData(file).overwrite().forTable(table).createWriterFunc(GenericOrcWriter::buildWriter).build(); + + log.info("start writing fileFormat: {}", fileFormat); + try { + dataWriter = ORC.writeData(file).overwrite().forTable(table).createWriterFunc(GenericOrcWriter::buildWriter).build(); + } + catch (Exception e) { + throw new RuntimeException(e); + } } else { throw new RuntimeException("不支持的文件格式:" + fileFormat); @@ -364,9 +373,9 @@ else if ("orc".equals(fileFormat)) { TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); int partitionId = 1, taskId = 1; - FileFormat fileFormat = FileFormat.PARQUET; + FileFormat fileFormatIntance = FileFormat.PARQUET; if ("orc".equals(fileFormat)) { - fileFormat = FileFormat.ORC; + fileFormatIntance = FileFormat.ORC; } Set identifierFieldIds = table.schema().identifierFieldIds(); FileAppenderFactory appenderFactory; @@ -385,10 +394,10 @@ else if ("orc".equals(fileFormat)) { null) .setAll(tableProps); } - OutputFileFactory outputFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId).format(fileFormat).build(); + OutputFileFactory outputFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId).format(fileFormatIntance).build(); final PartitionKey partitionKey = new PartitionKey(table.spec(), table.spec().schema()); // partitionedFanoutWriter will auto partitioned record and create the partitioned writer - PartitionedFanoutWriter partitionedFanoutWriter = new PartitionedFanoutWriter(table.spec(), fileFormat, appenderFactory, outputFileFactory, table.io(), targetFileSize) + PartitionedFanoutWriter partitionedFanoutWriter = new PartitionedFanoutWriter(table.spec(), fileFormatIntance, appenderFactory, outputFileFactory, table.io(), targetFileSize) { @Override protected PartitionKey partition(org.apache.iceberg.data.Record record) diff --git a/pom.xml b/pom.xml index 8d3276f5a..2dedd639b 100644 --- a/pom.xml +++ b/pom.xml @@ -31,7 +31,7 @@ 0.27 2.29.20 - 1.11.4 + 1.12.0 1.10.1 1.10 2.11.0 @@ -83,7 +83,7 @@ 2.0.2-beta 4.1.93.Final 3.4.7 - 1.9.2 + 1.9.5 1.15.1 5.4.1 3.25.5 From b5ab2ac371bfd6a4aa1007519e201cfd155c8870 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Fri, 20 Jun 2025 13:40:31 +0800 Subject: [PATCH 38/40] [bugfix][plugin][icebergwriter] fix error when write orc format table --- .../addax/plugin/writer/icebergwriter/IcebergWriter.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java index bcd3c2271..f2a0ce11a 100644 --- a/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java +++ b/plugin/writer/icebergwriter/src/main/java/com/wgzhao/addax/plugin/writer/icebergwriter/IcebergWriter.java @@ -59,7 +59,12 @@ import java.io.IOException; import java.io.UncheckedIOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; public class IcebergWriter extends Writer From a78c77eb7aab19632132b993aa3aea79fa433799 Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Fri, 25 Jul 2025 16:39:39 +0800 Subject: [PATCH 39/40] [bugfixed] bug for kerberos error when jdk24 --- package.xml | 8 + patch/addax-kerberos-jdk24-patch/package.xml | 48 + patch/addax-kerberos-jdk24-patch/pom.xml | 72 + .../hadoop/security/UserGroupInformation.java | 2259 +++++++++++++++++ pom.xml | 40 + 5 files changed, 2427 insertions(+) create mode 100644 patch/addax-kerberos-jdk24-patch/package.xml create mode 100644 patch/addax-kerberos-jdk24-patch/pom.xml create mode 100644 patch/addax-kerberos-jdk24-patch/src/main/java/org/apache/hadoop/security/UserGroupInformation.java diff --git a/package.xml b/package.xml index fd9d0b246..3d2db5981 100644 --- a/package.xml +++ b/package.xml @@ -34,6 +34,14 @@ / + + patch/addax-kerberos-jdk24-patch/target/addax-kerberos-jdk24-patch-${project.version}/lib/ + + *.* + + 0644 + lib + lib/addax-rdbms/target/addax-rdbms-${project.version}/lib/ diff --git a/patch/addax-kerberos-jdk24-patch/package.xml b/patch/addax-kerberos-jdk24-patch/package.xml new file mode 100644 index 000000000..c2bc008e4 --- /dev/null +++ b/patch/addax-kerberos-jdk24-patch/package.xml @@ -0,0 +1,48 @@ + + + + release + + dir + + false + + + target/ + + ${project.artifactId}-${project.version}.jar + + lib + + + + + false + lib + runtime + + com.wgzhao.addax:addax-core + + + + diff --git a/patch/addax-kerberos-jdk24-patch/pom.xml b/patch/addax-kerberos-jdk24-patch/pom.xml new file mode 100644 index 000000000..52be55c72 --- /dev/null +++ b/patch/addax-kerberos-jdk24-patch/pom.xml @@ -0,0 +1,72 @@ + + + + 4.0.0 + + + com.wgzhao.addax + addax-all + 6.0.3-SNAPSHOT + ../../pom.xml + + + addax-kerberos-jdk24-patch + + addax-kerberos-jdk24-patch + Fix for Error When kerberos Auth running on jdk24 + jar + + + + + com.fasterxml.woodstox + woodstox-core + ${woodstox.version} + + + com.google.protobuf + protobuf-java + ${protobuf.version} + + + org.apache.hadoop + hadoop-common + + + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + 24 + 24 + + + + + diff --git a/patch/addax-kerberos-jdk24-patch/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/patch/addax-kerberos-jdk24-patch/src/main/java/org/apache/hadoop/security/UserGroupInformation.java new file mode 100644 index 000000000..7eae75ca2 --- /dev/null +++ b/patch/addax-kerberos-jdk24-patch/src/main/java/org/apache/hadoop/security/UserGroupInformation.java @@ -0,0 +1,2259 @@ +/** + * 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.hadoop.security; + +import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN_DEFAULT; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_KEYTAB_LOGIN_AUTORENEWAL_ENABLED; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_KEYTAB_LOGIN_AUTORENEWAL_ENABLED_DEFAULT; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKENS; +import static org.apache.hadoop.security.UGIExceptionMessages.*; +import static org.apache.hadoop.util.PlatformName.IBM_JAVA; +import static org.apache.hadoop.util.StringUtils.getTrimmedStringCollection; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.UndeclaredThrowableException; +import java.security.Principal; +import java.security.PrivilegedAction; +import java.security.PrivilegedActionException; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumMap; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import javax.security.auth.DestroyFailedException; +import javax.security.auth.Subject; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.kerberos.KerberosPrincipal; +import javax.security.auth.kerberos.KerberosTicket; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag; +import javax.security.auth.login.Configuration.Parameters; +import javax.security.auth.login.LoginContext; +import javax.security.auth.login.LoginException; +import javax.security.auth.spi.LoginModule; + +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.metrics2.annotation.Metric; +import org.apache.hadoop.metrics2.annotation.Metrics; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.metrics2.lib.MetricsRegistry; +import org.apache.hadoop.metrics2.lib.MutableGaugeInt; +import org.apache.hadoop.metrics2.lib.MutableGaugeLong; +import org.apache.hadoop.metrics2.lib.MutableQuantiles; +import org.apache.hadoop.metrics2.lib.MutableRate; +import org.apache.hadoop.security.SaslRpcServer.AuthMethod; +import org.apache.hadoop.security.authentication.util.KerberosUtil; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.hadoop.util.Shell; +import org.apache.hadoop.util.Time; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * User and group information for Hadoop. + * This class wraps around a JAAS Subject and provides methods to determine the + * user's username and groups. It supports both the Windows, Unix and Kerberos + * login modules. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class UserGroupInformation { + @VisibleForTesting + static final Logger LOG = LoggerFactory.getLogger( + UserGroupInformation.class); + + /** + * Percentage of the ticket window to use before we renew ticket. + */ + private static final float TICKET_RENEW_WINDOW = 0.80f; + private static boolean shouldRenewImmediatelyForTests = false; + static final String HADOOP_USER_NAME = "HADOOP_USER_NAME"; + static final String HADOOP_PROXY_USER = "HADOOP_PROXY_USER"; + + /** + * For the purposes of unit tests, we want to test login + * from keytab and don't want to wait until the renew + * window (controlled by TICKET_RENEW_WINDOW). + * @param immediate true if we should login without waiting for ticket window + */ + @VisibleForTesting + public static void setShouldRenewImmediatelyForTests(boolean immediate) { + shouldRenewImmediatelyForTests = immediate; + } + + /** + * UgiMetrics maintains UGI activity statistics + * and publishes them through the metrics interfaces. + */ + @Metrics(about="User and group related metrics", context="ugi") + static class UgiMetrics { + final MetricsRegistry registry = new MetricsRegistry("UgiMetrics"); + + @Metric("Rate of successful kerberos logins and latency (milliseconds)") + MutableRate loginSuccess; + @Metric("Rate of failed kerberos logins and latency (milliseconds)") + MutableRate loginFailure; + @Metric("GetGroups") MutableRate getGroups; + MutableQuantiles[] getGroupsQuantiles; + @Metric("Renewal failures since startup") + private MutableGaugeLong renewalFailuresTotal; + @Metric("Renewal failures since last successful login") + private MutableGaugeInt renewalFailures; + + static UgiMetrics create() { + return DefaultMetricsSystem.instance().register(new UgiMetrics()); + } + + static void reattach() { + metrics = UgiMetrics.create(); + } + + void addGetGroups(long latency) { + getGroups.add(latency); + if (getGroupsQuantiles != null) { + for (MutableQuantiles q : getGroupsQuantiles) { + q.add(latency); + } + } + } + + MutableGaugeInt getRenewalFailures() { + return renewalFailures; + } + } + + /** + * A login module that looks at the Kerberos, Unix, or Windows principal and + * adds the corresponding UserName. + */ + @InterfaceAudience.Private + public static class HadoopLoginModule implements LoginModule { + private Subject subject; + + @Override + public boolean abort() throws LoginException { + return true; + } + + private T getCanonicalUser(Class cls) { + for(T user: subject.getPrincipals(cls)) { + return user; + } + return null; + } + + @Override + public boolean commit() throws LoginException { + LOG.debug("hadoop login commit"); + // if we already have a user, we are done. + if (!subject.getPrincipals(User.class).isEmpty()) { + LOG.debug("Using existing subject: {}", subject.getPrincipals()); + return true; + } + Principal user = getCanonicalUser(KerberosPrincipal.class); + if (user != null) { + LOG.debug("Using kerberos user: {}", user); + } + //If we don't have a kerberos user and security is disabled, check + //if user is specified in the environment or properties + if (!isSecurityEnabled() && (user == null)) { + String envUser = System.getenv(HADOOP_USER_NAME); + if (envUser == null) { + envUser = System.getProperty(HADOOP_USER_NAME); + } + user = envUser == null ? null : new User(envUser); + } + // use the OS user + if (user == null) { + user = getCanonicalUser(OS_PRINCIPAL_CLASS); + LOG.debug("Using local user: {}", user); + } + // if we found the user, add our principal + if (user != null) { + LOG.debug("Using user: \"{}\" with name: {}", user, user.getName()); + + User userEntry = null; + try { + // LoginContext will be attached later unless it's an external + // subject. + AuthenticationMethod authMethod = (user instanceof KerberosPrincipal) + ? AuthenticationMethod.KERBEROS : AuthenticationMethod.SIMPLE; + userEntry = new User(user.getName(), authMethod, null); + } catch (Exception e) { + throw (LoginException)(new LoginException(e.toString()).initCause(e)); + } + LOG.debug("User entry: \"{}\"", userEntry); + + subject.getPrincipals().add(userEntry); + return true; + } + throw new LoginException("Failed to find user in name " + subject); + } + + @Override + public void initialize(Subject subject, CallbackHandler callbackHandler, + Map sharedState, Map options) { + this.subject = subject; + } + + @Override + public boolean login() throws LoginException { + LOG.debug("Hadoop login"); + return true; + } + + @Override + public boolean logout() throws LoginException { + LOG.debug("Hadoop logout"); + return true; + } + } + + /** + * Reattach the class's metrics to a new metric system. + */ + public static void reattachMetrics() { + UgiMetrics.reattach(); + } + + /** Metrics to track UGI activity */ + static UgiMetrics metrics = UgiMetrics.create(); + /** The auth method to use */ + private static AuthenticationMethod authenticationMethod; + /** Server-side groups fetching service */ + private static Groups groups; + /** Min time (in seconds) before relogin for Kerberos */ + private static long kerberosMinSecondsBeforeRelogin; + /** Boolean flag to enable auto-renewal for keytab based loging. */ + private static boolean kerberosKeyTabLoginRenewalEnabled; + /** A reference to Kerberos login auto renewal thread. */ + private static Optional kerberosLoginRenewalExecutor = + Optional.empty(); + /** The configuration to use */ + + private static Configuration conf; + + + /**Environment variable pointing to the token cache file*/ + public static final String HADOOP_TOKEN_FILE_LOCATION = + "HADOOP_TOKEN_FILE_LOCATION"; + /** Environment variable pointing to the base64 tokens. */ + public static final String HADOOP_TOKEN = "HADOOP_TOKEN"; + + public static boolean isInitialized() { + return conf != null; + } + + /** + * A method to initialize the fields that depend on a configuration. + * Must be called before useKerberos or groups is used. + */ + private static void ensureInitialized() { + if (!isInitialized()) { + synchronized(UserGroupInformation.class) { + if (!isInitialized()) { // someone might have beat us + initialize(new Configuration(), false); + } + } + } + } + + /** + * Initialize UGI and related classes. + * @param conf the configuration to use + */ + private static synchronized void initialize(Configuration conf, + boolean overrideNameRules) { + authenticationMethod = SecurityUtil.getAuthenticationMethod(conf); + if (overrideNameRules || !HadoopKerberosName.hasRulesBeenSet()) { + try { + HadoopKerberosName.setConfiguration(conf); + } catch (IOException ioe) { + throw new RuntimeException( + "Problem with Kerberos auth_to_local name configuration", ioe); + } + } + try { + kerberosMinSecondsBeforeRelogin = 1000L * conf.getLong( + HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN, + HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN_DEFAULT); + } + catch(NumberFormatException nfe) { + throw new IllegalArgumentException("Invalid attribute value for " + + HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN + " of " + + conf.get(HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN)); + } + + kerberosKeyTabLoginRenewalEnabled = conf.getBoolean( + HADOOP_KERBEROS_KEYTAB_LOGIN_AUTORENEWAL_ENABLED, + HADOOP_KERBEROS_KEYTAB_LOGIN_AUTORENEWAL_ENABLED_DEFAULT); + + // If we haven't set up testing groups, use the configuration to find it + if (!(groups instanceof TestingGroups)) { + groups = Groups.getUserToGroupsMappingService(conf); + } + UserGroupInformation.conf = conf; + + if (metrics.getGroupsQuantiles == null) { + int[] intervals = conf.getInts(HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS); + if (intervals != null && intervals.length > 0) { + final int length = intervals.length; + MutableQuantiles[] getGroupsQuantiles = new MutableQuantiles[length]; + for (int i = 0; i < length; i++) { + getGroupsQuantiles[i] = metrics.registry.newQuantiles( + "getGroups" + intervals[i] + "s", + "Get groups", "ops", "latency", intervals[i]); + } + metrics.getGroupsQuantiles = getGroupsQuantiles; + } + } + } + + /** + * Set the static configuration for UGI. + * In particular, set the security authentication mechanism and the + * group look up service. + * @param conf the configuration to use + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static void setConfiguration(Configuration conf) { + initialize(conf, true); + } + + @InterfaceAudience.Private + @VisibleForTesting + public static void reset() { + authenticationMethod = null; + conf = null; + groups = null; + kerberosMinSecondsBeforeRelogin = 0; + kerberosKeyTabLoginRenewalEnabled = false; + kerberosLoginRenewalExecutor = Optional.empty(); + setLoginUser(null); + HadoopKerberosName.setRules(null); + } + + /** + * Determine if UserGroupInformation is using Kerberos to determine + * user identities or is relying on simple authentication + * + * @return true if UGI is working in a secure environment + */ + public static boolean isSecurityEnabled() { + return !isAuthenticationMethodEnabled(AuthenticationMethod.SIMPLE); + } + + @InterfaceAudience.Private + @InterfaceStability.Evolving + private static boolean isAuthenticationMethodEnabled(AuthenticationMethod method) { + ensureInitialized(); + return (authenticationMethod == method); + } + + @InterfaceAudience.Private + @InterfaceStability.Evolving + @VisibleForTesting + static boolean isKerberosKeyTabLoginRenewalEnabled() { + ensureInitialized(); + return kerberosKeyTabLoginRenewalEnabled; + } + + @InterfaceAudience.Private + @InterfaceStability.Evolving + @VisibleForTesting + static Optional getKerberosLoginRenewalExecutor() { + ensureInitialized(); + return kerberosLoginRenewalExecutor; + } + + /** + * Information about the logged in user. + */ + private static final AtomicReference loginUserRef = + new AtomicReference<>(); + + private final Subject subject; + // All non-static fields must be read-only caches that come from the subject. + private final User user; + + private static String OS_LOGIN_MODULE_NAME; + private static Class OS_PRINCIPAL_CLASS; + + private static final boolean windows = + System.getProperty("os.name").startsWith("Windows"); + + /* Return the OS login module class name */ + /* For IBM JDK, use the common OS login module class name for all platforms */ + private static String getOSLoginModuleName() { + if (IBM_JAVA) { + return "com.ibm.security.auth.module.JAASLoginModule"; + } else { + return windows ? "com.sun.security.auth.module.NTLoginModule" + : "com.sun.security.auth.module.UnixLoginModule"; + } + } + + /* Return the OS principal class */ + /* For IBM JDK, use the common OS principal class for all platforms */ + @SuppressWarnings("unchecked") + private static Class getOsPrincipalClass() { + ClassLoader cl = ClassLoader.getSystemClassLoader(); + try { + String principalClass = null; + if (IBM_JAVA) { + principalClass = "com.ibm.security.auth.UsernamePrincipal"; + } else { + principalClass = windows ? "com.sun.security.auth.NTUserPrincipal" + : "com.sun.security.auth.UnixPrincipal"; + } + return (Class) cl.loadClass(principalClass); + } catch (ClassNotFoundException e) { + LOG.error("Unable to find JAAS classes:" + e.getMessage()); + } + return null; + } + static { + OS_LOGIN_MODULE_NAME = getOSLoginModuleName(); + OS_PRINCIPAL_CLASS = getOsPrincipalClass(); + } + + private static class RealUser implements Principal { + private final UserGroupInformation realUser; + + RealUser(UserGroupInformation realUser) { + this.realUser = realUser; + } + + @Override + public String getName() { + return realUser.getUserName(); + } + + public UserGroupInformation getRealUser() { + return realUser; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (o == null || getClass() != o.getClass()) { + return false; + } else { + return realUser.equals(((RealUser) o).realUser); + } + } + + @Override + public int hashCode() { + return realUser.hashCode(); + } + + @Override + public String toString() { + return realUser.toString(); + } + } + + private static HadoopLoginContext + newLoginContext(String appName, Subject subject, + HadoopConfiguration loginConf) + throws LoginException { + // Temporarily switch the thread's ContextClassLoader to match this + // class's classloader, so that we can properly load HadoopLoginModule + // from the JAAS libraries. + Thread t = Thread.currentThread(); + ClassLoader oldCCL = t.getContextClassLoader(); + t.setContextClassLoader(HadoopLoginModule.class.getClassLoader()); + try { + return new HadoopLoginContext(appName, subject, loginConf); + } finally { + t.setContextClassLoader(oldCCL); + } + } + + // return the LoginContext only if it's managed by the ugi. externally + // managed login contexts will be ignored. + private HadoopLoginContext getLogin() { + LoginContext login = user.getLogin(); + return (login instanceof HadoopLoginContext) + ? (HadoopLoginContext)login : null; + } + + private void setLogin(LoginContext login) { + user.setLogin(login); + } + + /** + * Set the last login time for logged in user + * @param loginTime the number of milliseconds since the beginning of time + */ + private void setLastLogin(long loginTime) { + user.setLastLogin(loginTime); + } + + /** + * Create a UserGroupInformation for the given subject. + * This does not change the subject or acquire new credentials. + * + * The creator of subject is responsible for renewing credentials. + * @param subject the user's subject + */ + UserGroupInformation(Subject subject) { + this.subject = subject; + // do not access ANY private credentials since they are mutable + // during a relogin. no principal locking necessary since + // relogin/logout does not remove User principal. + this.user = subject.getPrincipals(User.class).iterator().next(); + if (user == null || user.getName() == null) { + throw new IllegalStateException("Subject does not contain a valid User"); + } + } + + /** + * checks if logged in using kerberos + * @return true if the subject logged via keytab or has a Kerberos TGT + */ + public boolean hasKerberosCredentials() { + return user.getAuthenticationMethod() == AuthenticationMethod.KERBEROS; + } + + /** + * Return the current user, including any doAs in the current stack. + * @return the current user + * @throws IOException if login fails + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static UserGroupInformation getCurrentUser() throws IOException { + ensureInitialized(); + Subject subject = Subject.current(); + if (subject == null || subject.getPrincipals(User.class).isEmpty()) { + return getLoginUser(); + } else { + return new UserGroupInformation(subject); + } + } + + /** + * Find the most appropriate UserGroupInformation to use + * + * @param ticketCachePath The Kerberos ticket cache path, or NULL + * if none is specfied + * @param user The user name, or NULL if none is specified. + * + * @return The most appropriate UserGroupInformation + * @throws IOException raised on errors performing I/O. + */ + public static UserGroupInformation getBestUGI( + String ticketCachePath, String user) throws IOException { + if (ticketCachePath != null) { + return getUGIFromTicketCache(ticketCachePath, user); + } else if (user == null) { + return getCurrentUser(); + } else { + return createRemoteUser(user); + } + } + + /** + * Create a UserGroupInformation from a Kerberos ticket cache. + * + * @param user The principal name to load from the ticket + * cache + * @param ticketCache the path to the ticket cache file + * + * @throws IOException if the kerberos login fails + * @return UserGroupInformation. + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static UserGroupInformation getUGIFromTicketCache( + String ticketCache, String user) throws IOException { + if (!isAuthenticationMethodEnabled(AuthenticationMethod.KERBEROS)) { + return getBestUGI(null, user); + } + LoginParams params = new LoginParams(); + params.put(LoginParam.PRINCIPAL, user); + params.put(LoginParam.CCACHE, ticketCache); + return doSubjectLogin(null, params); + } + + /** + * Create a UserGroupInformation from a Subject with Kerberos principal. + * + * @param subject The KerberosPrincipal to use in UGI. + * The creator of subject is responsible for + * renewing credentials. + * + * @throws IOException raised on errors performing I/O. + * @throws KerberosAuthException if the kerberos login fails + * @return UserGroupInformation. + */ + public static UserGroupInformation getUGIFromSubject(Subject subject) + throws IOException { + if (subject == null) { + throw new KerberosAuthException(SUBJECT_MUST_NOT_BE_NULL); + } + + if (subject.getPrincipals(KerberosPrincipal.class).isEmpty()) { + throw new KerberosAuthException(SUBJECT_MUST_CONTAIN_PRINCIPAL); + } + + // null params indicate external subject login. no login context will + // be attached. + return doSubjectLogin(subject, null); + } + + /** + * Get the currently logged in user. If no explicit login has occurred, + * the user will automatically be logged in with either kerberos credentials + * if available, or as the local OS user, based on security settings. + * @return the logged in user + * @throws IOException if login fails + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static UserGroupInformation getLoginUser() throws IOException { + ensureInitialized(); + UserGroupInformation loginUser = loginUserRef.get(); + // a potential race condition exists only for the initial creation of + // the login user. there's no need to penalize all subsequent calls + // with sychronization overhead so optimistically create a login user + // and discard if we lose the race. + if (loginUser == null) { + UserGroupInformation newLoginUser = createLoginUser(null); + do { + // it's extremely unlikely that the login user will be non-null + // (lost CAS race), but be nulled before the subsequent get, but loop + // for correctness. + if (loginUserRef.compareAndSet(null, newLoginUser)) { + loginUser = newLoginUser; + // only spawn renewal if this login user is the winner. + loginUser.spawnAutoRenewalThreadForUserCreds(false); + } else { + loginUser = loginUserRef.get(); + } + } while (loginUser == null); + } + return loginUser; + } + + /** + * remove the login method that is followed by a space from the username + * e.g. "jack (auth:SIMPLE)" {@literal ->} "jack" + * + * @param userName input userName. + * @return userName without login method + */ + public static String trimLoginMethod(String userName) { + int spaceIndex = userName.indexOf(' '); + if (spaceIndex >= 0) { + userName = userName.substring(0, spaceIndex); + } + return userName; + } + + /** + * Log in a user using the given subject + * @param subject the subject to use when logging in a user, or null to + * create a new subject. + * + * If subject is not null, the creator of subject is responsible for renewing + * credentials. + * + * @throws IOException if login fails + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static void loginUserFromSubject(Subject subject) throws IOException { + setLoginUser(createLoginUser(subject)); + } + + private static + UserGroupInformation createLoginUser(Subject subject) throws IOException { + UserGroupInformation realUser = doSubjectLogin(subject, null); + UserGroupInformation loginUser = null; + try { + // If the HADOOP_PROXY_USER environment variable or property + // is specified, create a proxy user as the logged in user. + String proxyUser = System.getenv(HADOOP_PROXY_USER); + if (proxyUser == null) { + proxyUser = System.getProperty(HADOOP_PROXY_USER); + } + loginUser = proxyUser == null ? realUser : createProxyUser(proxyUser, realUser); + + // Load tokens from files + final Collection tokenFileLocations = new LinkedHashSet<>(); + tokenFileLocations.addAll(getTrimmedStringCollection( + System.getProperty(HADOOP_TOKEN_FILES))); + tokenFileLocations.addAll(getTrimmedStringCollection( + conf.get(HADOOP_TOKEN_FILES))); + tokenFileLocations.addAll(getTrimmedStringCollection( + System.getenv(HADOOP_TOKEN_FILE_LOCATION))); + for (String tokenFileLocation : tokenFileLocations) { + if (tokenFileLocation != null && tokenFileLocation.length() > 0) { + File tokenFile = new File(tokenFileLocation); + LOG.debug("Reading credentials from location {}", + tokenFile.getCanonicalPath()); + if (tokenFile.exists() && tokenFile.isFile()) { + Credentials cred = Credentials.readTokenStorageFile( + tokenFile, conf); + LOG.debug("Loaded {} tokens from {}", cred.numberOfTokens(), + tokenFile.getCanonicalPath()); + loginUser.addCredentials(cred); + } else { + LOG.info("Token file {} does not exist", + tokenFile.getCanonicalPath()); + } + } + } + + // Load tokens from base64 encoding + final Collection tokensBase64 = new LinkedHashSet<>(); + tokensBase64.addAll(getTrimmedStringCollection( + System.getProperty(HADOOP_TOKENS))); + tokensBase64.addAll(getTrimmedStringCollection( + conf.get(HADOOP_TOKENS))); + tokensBase64.addAll(getTrimmedStringCollection( + System.getenv(HADOOP_TOKEN))); + int numTokenBase64 = 0; + for (String tokenBase64 : tokensBase64) { + if (tokenBase64 != null && tokenBase64.length() > 0) { + try { + Token token = new Token<>(); + token.decodeFromUrlString(tokenBase64); + Credentials cred = new Credentials(); + cred.addToken(token.getService(), token); + loginUser.addCredentials(cred); + numTokenBase64++; + } catch (IOException ioe) { + LOG.error("Cannot add token {}: {}", + tokenBase64, ioe.getMessage()); + } + } + } + if (numTokenBase64 > 0) { + LOG.debug("Loaded {} base64 tokens", numTokenBase64); + } + } catch (IOException ioe) { + LOG.debug("Failure to load login credentials", ioe); + throw ioe; + } + LOG.debug("UGI loginUser: {}", loginUser); + return loginUser; + } + + @InterfaceAudience.Private + @InterfaceStability.Unstable + @VisibleForTesting + public static void setLoginUser(UserGroupInformation ugi) { + // if this is to become stable, should probably logout the currently + // logged in ugi if it's different + loginUserRef.set(ugi); + } + + private String getKeytab() { + HadoopLoginContext login = getLogin(); + return (login != null) + ? login.getConfiguration().getParameters().get(LoginParam.KEYTAB) + : null; + } + + /** + * Is the ugi managed by the UGI or an external subject? + * @return true if managed by UGI. + */ + private boolean isHadoopLogin() { + // checks if the private hadoop login context is managing the ugi. + return getLogin() != null; + } + + /** + * Is this user logged in from a keytab file managed by the UGI? + * @return true if the credentials are from a keytab file. + */ + public boolean isFromKeytab() { + // can't simply check if keytab is present since a relogin failure will + // have removed the keytab from priv creds. instead, check login params. + return hasKerberosCredentials() && isHadoopLogin() && getKeytab() != null; + } + + /** + * Is this user logged in from a ticket (but no keytab) managed by the UGI? + * @return true if the credentials are from a ticket cache. + */ + private boolean isFromTicket() { + return hasKerberosCredentials() && isHadoopLogin() && getKeytab() == null; + } + + /** + * Get the Kerberos TGT + * @return the user's TGT or null if none was found + */ + private KerberosTicket getTGT() { + Set tickets = subject + .getPrivateCredentials(KerberosTicket.class); + for (KerberosTicket ticket : tickets) { + if (SecurityUtil.isOriginalTGT(ticket)) { + return ticket; + } + } + return null; + } + + private long getRefreshTime(KerberosTicket tgt) { + long start = tgt.getStartTime().getTime(); + long end = tgt.getEndTime().getTime(); + return start + (long) ((end - start) * TICKET_RENEW_WINDOW); + } + + @InterfaceAudience.Private + @InterfaceStability.Unstable + public boolean shouldRelogin() { + return hasKerberosCredentials() && isHadoopLogin(); + } + + /** + * Spawn a thread to do periodic renewals of kerberos credentials. NEVER + * directly call this method. This method should only be used for ticket cache + * based kerberos credentials. + * + * @param force - used by tests to forcibly spawn thread + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + @VisibleForTesting + void spawnAutoRenewalThreadForUserCreds(boolean force) { + if (!force && (!shouldRelogin() || isFromKeytab())) { + return; + } + + //spawn thread only if we have kerb credentials + KerberosTicket tgt = getTGT(); + if (tgt == null) { + return; + } + String cmd = conf.get("hadoop.kerberos.kinit.command", "kinit"); + long nextRefresh = getRefreshTime(tgt); + executeAutoRenewalTask(getUserName(), + new TicketCacheRenewalRunnable(tgt, cmd, nextRefresh)); + } + + /** + * Spawn a thread to do periodic renewals of kerberos credentials from a + * keytab file. + */ + private void spawnAutoRenewalThreadForKeytab() { + if (!shouldRelogin() || isFromTicket()) { + return; + } + + // spawn thread only if we have kerb credentials + KerberosTicket tgt = getTGT(); + if (tgt == null) { + return; + } + long nextRefresh = getRefreshTime(tgt); + executeAutoRenewalTask(getUserName(), + new KeytabRenewalRunnable(tgt, nextRefresh)); + } + + /** + * Spawn a thread to do periodic renewals of kerberos credentials from a + * keytab file. NEVER directly call this method. + * + * @param userName Name of the user for which login needs to be renewed. + * @param task The reference of the login renewal task. + */ + private void executeAutoRenewalTask(final String userName, + AutoRenewalForUserCredsRunnable task) { + kerberosLoginRenewalExecutor = Optional.of( + Executors.newSingleThreadExecutor( + new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + Thread t = new Thread(r); + t.setDaemon(true); + t.setName("TGT Renewer for " + userName); + return t; + } + } + )); + kerberosLoginRenewalExecutor.get().submit(task); + } + + /** + * An abstract class which encapsulates the functionality required to + * auto renew Kerbeors TGT. The concrete implementations of this class + * are expected to provide implementation required to perform actual + * TGT renewal (see {@code TicketCacheRenewalRunnable} and + * {@code KeytabRenewalRunnable}). + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + @VisibleForTesting + abstract class AutoRenewalForUserCredsRunnable implements Runnable { + private KerberosTicket tgt; + private RetryPolicy rp; + private long nextRefresh; + private boolean runRenewalLoop = true; + + AutoRenewalForUserCredsRunnable(KerberosTicket tgt, long nextRefresh) { + this.tgt = tgt; + this.nextRefresh = nextRefresh; + this.rp = null; + } + + public void setRunRenewalLoop(boolean runRenewalLoop) { + this.runRenewalLoop = runRenewalLoop; + } + + /** + * This method is used to perform renewal of kerberos login ticket. + * The concrete implementations of this class should provide specific + * logic required to perform renewal as part of this method. + */ + protected abstract void relogin() throws IOException; + + @Override + public void run() { + do { + try { + long now = Time.now(); + LOG.debug("Current time is {}, next refresh is {}", now, nextRefresh); + if (now < nextRefresh) { + Thread.sleep(nextRefresh - now); + } + relogin(); + tgt = getTGT(); + if (tgt == null) { + LOG.warn("No TGT after renewal. Aborting renew thread for " + + getUserName()); + return; + } + nextRefresh = Math.max(getRefreshTime(tgt), + now + kerberosMinSecondsBeforeRelogin); + metrics.renewalFailures.set(0); + rp = null; + } catch (InterruptedException ie) { + LOG.warn("Terminating renewal thread"); + return; + } catch (IOException ie) { + metrics.renewalFailuresTotal.incr(); + final long now = Time.now(); + + if (tgt.isDestroyed()) { + LOG.error(String.format("TGT is destroyed. " + + "Aborting renew thread for %s.", getUserName()), ie); + return; + } + + long tgtEndTime; + // As described in HADOOP-15593 we need to handle the case when + // tgt.getEndTime() throws NPE because of JDK issue JDK-8147772 + // NPE is only possible if this issue is not fixed in the JDK + // currently used + try { + tgtEndTime = tgt.getEndTime().getTime(); + } catch (NullPointerException npe) { + LOG.error("NPE thrown while getting KerberosTicket endTime. " + + "Aborting renew thread for {}.", getUserName(), ie); + return; + } + + LOG.warn( + "Exception encountered while running the " + + "renewal command for {}. " + + "(TGT end time:{}, renewalFailures: {}, " + + "renewalFailuresTotal: {})", + getUserName(), tgtEndTime, metrics.renewalFailures.value(), + metrics.renewalFailuresTotal.value(), ie); + if (rp == null) { + // Use a dummy maxRetries to create the policy. The policy will + // only be used to get next retry time with exponential back-off. + // The final retry time will be later limited within the + // tgt endTime in getNextTgtRenewalTime. + rp = RetryPolicies.exponentialBackoffRetry(Long.SIZE - 2, + kerberosMinSecondsBeforeRelogin, TimeUnit.MILLISECONDS); + } + try { + nextRefresh = getNextTgtRenewalTime(tgtEndTime, now, rp); + } catch (Exception e) { + LOG.error("Exception when calculating next tgt renewal time", e); + return; + } + metrics.renewalFailures.incr(); + // retry until close enough to tgt endTime. + if (now > nextRefresh) { + LOG.error("TGT is expired. Aborting renew thread for {}.", + getUserName()); + return; + } + } + } while (runRenewalLoop); + } + } + + /** + * A concrete implementation of {@code AutoRenewalForUserCredsRunnable} class + * which performs TGT renewal using kinit command. + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + @VisibleForTesting + final class TicketCacheRenewalRunnable + extends AutoRenewalForUserCredsRunnable { + private String kinitCmd; + + TicketCacheRenewalRunnable(KerberosTicket tgt, String kinitCmd, + long nextRefresh) { + super(tgt, nextRefresh); + this.kinitCmd = kinitCmd; + } + + @Override + public void relogin() throws IOException { + String output = Shell.execCommand(kinitCmd, "-R"); + LOG.debug("Renewed ticket. kinit output: {}", output); + reloginFromTicketCache(); + } + } + + /** + * A concrete implementation of {@code AutoRenewalForUserCredsRunnable} class + * which performs TGT renewal using specified keytab. + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + @VisibleForTesting + final class KeytabRenewalRunnable extends AutoRenewalForUserCredsRunnable { + + KeytabRenewalRunnable(KerberosTicket tgt, long nextRefresh) { + super(tgt, nextRefresh); + } + + @Override + public void relogin() throws IOException { + reloginFromKeytab(); + } + } + + /** + * Get time for next login retry. This will allow the thread to retry with + * exponential back-off, until tgt endtime. + * Last retry is {@link #kerberosMinSecondsBeforeRelogin} before endtime. + * + * @param tgtEndTime EndTime of the tgt. + * @param now Current time. + * @param rp The retry policy. + * @return Time for next login retry. + */ + @VisibleForTesting + static long getNextTgtRenewalTime(final long tgtEndTime, final long now, + final RetryPolicy rp) throws Exception { + final long lastRetryTime = tgtEndTime - kerberosMinSecondsBeforeRelogin; + final RetryPolicy.RetryAction ra = rp.shouldRetry(null, + metrics.renewalFailures.value(), 0, false); + return Math.min(lastRetryTime, now + ra.delayMillis); + } + + /** + * Log a user in from a keytab file. Loads a user identity from a keytab + * file and logs them in. They become the currently logged-in user. + * @param user the principal name to load from the keytab + * @param path the path to the keytab file + * @throws IOException raised on errors performing I/O. + * @throws KerberosAuthException if it's a kerberos login exception. + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public + static void loginUserFromKeytab(String user, + String path + ) throws IOException { + if (!isSecurityEnabled()) + return; + + UserGroupInformation u = loginUserFromKeytabAndReturnUGI(user, path); + if (isKerberosKeyTabLoginRenewalEnabled()) { + u.spawnAutoRenewalThreadForKeytab(); + } + + setLoginUser(u); + + LOG.info( + "Login successful for user {} using keytab file {}. Keytab auto" + + " renewal enabled : {}", + user, new File(path).getName(), isKerberosKeyTabLoginRenewalEnabled()); + } + + /** + * Log the current user out who previously logged in using keytab. + * This method assumes that the user logged in by calling + * {@link #loginUserFromKeytab(String, String)}. + * + * @throws IOException raised on errors performing I/O. + * @throws KerberosAuthException if a failure occurred in logout, + * or if the user did not log in by invoking loginUserFromKeyTab() before. + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public void logoutUserFromKeytab() throws IOException { + if (!hasKerberosCredentials()) { + return; + } + + // Shutdown the background task performing login renewal. + if (getKerberosLoginRenewalExecutor().isPresent()) { + getKerberosLoginRenewalExecutor().get().shutdownNow(); + } + + HadoopLoginContext login = getLogin(); + String keytabFile = getKeytab(); + if (login == null || keytabFile == null) { + throw new KerberosAuthException(MUST_FIRST_LOGIN_FROM_KEYTAB); + } + + try { + LOG.debug("Initiating logout for {}", getUserName()); + // hadoop login context internally locks credentials. + login.logout(); + } catch (LoginException le) { + KerberosAuthException kae = new KerberosAuthException(LOGOUT_FAILURE, le); + kae.setUser(user.toString()); + kae.setKeytabFile(keytabFile); + throw kae; + } + + LOG.info("Logout successful for user " + getUserName() + + " using keytab file " + keytabFile); + } + + /** + * Re-login a user from keytab if TGT is expired or is close to expiry. + * + * @throws IOException raised on errors performing I/O. + * @throws KerberosAuthException if it's a kerberos login exception. + */ + public void checkTGTAndReloginFromKeytab() throws IOException { + reloginFromKeytab(true); + } + + // if the first kerberos ticket is not TGT, then remove and destroy it since + // the kerberos library of jdk always use the first kerberos ticket as TGT. + // See HADOOP-13433 for more details. + @VisibleForTesting + void fixKerberosTicketOrder() { + Set creds = getSubject().getPrivateCredentials(); + synchronized (creds) { + for (Iterator iter = creds.iterator(); iter.hasNext();) { + Object cred = iter.next(); + if (cred instanceof KerberosTicket) { + KerberosTicket ticket = (KerberosTicket) cred; + if (ticket.isDestroyed() || ticket.getServer() == null) { + LOG.warn("Ticket is already destroyed, remove it."); + iter.remove(); + } else if (!ticket.getServer().getName().startsWith("krbtgt")) { + LOG.warn( + "The first kerberos ticket is not TGT" + + "(the server principal is {}), remove and destroy it.", + ticket.getServer()); + iter.remove(); + try { + ticket.destroy(); + } catch (DestroyFailedException e) { + LOG.warn("destroy ticket failed", e); + } + } else { + return; + } + } + } + } + LOG.warn("Warning, no kerberos ticket found while attempting to renew ticket"); + } + + /** + * Re-Login a user in from a keytab file. Loads a user identity from a keytab + * file and logs them in. They become the currently logged-in user. This + * method assumes that {@link #loginUserFromKeytab(String, String)} had + * happened already. + * The Subject field of this UserGroupInformation object is updated to have + * the new credentials. + * @throws IOException raised on errors performing I/O. + * @throws KerberosAuthException on a failure + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public void reloginFromKeytab() throws IOException { + reloginFromKeytab(false); + } + + /** + * Force re-Login a user in from a keytab file irrespective of the last login + * time. Loads a user identity from a keytab file and logs them in. They + * become the currently logged-in user. This method assumes that + * {@link #loginUserFromKeytab(String, String)} had happened already. The + * Subject field of this UserGroupInformation object is updated to have the + * new credentials. + * + * @throws IOException raised on errors performing I/O. + * @throws KerberosAuthException on a failure + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public void forceReloginFromKeytab() throws IOException { + reloginFromKeytab(false, true); + } + + private void reloginFromKeytab(boolean checkTGT) throws IOException { + reloginFromKeytab(checkTGT, false); + } + + private void reloginFromKeytab(boolean checkTGT, boolean ignoreLastLoginTime) + throws IOException { + if (!shouldRelogin() || !isFromKeytab()) { + return; + } + HadoopLoginContext login = getLogin(); + if (login == null) { + throw new KerberosAuthException(MUST_FIRST_LOGIN_FROM_KEYTAB); + } + if (checkTGT) { + KerberosTicket tgt = getTGT(); + if (tgt != null && !shouldRenewImmediatelyForTests && + Time.now() < getRefreshTime(tgt)) { + return; + } + } + relogin(login, ignoreLastLoginTime); + } + + /** + * Re-Login a user in from the ticket cache. This + * method assumes that login had happened already. + * The Subject field of this UserGroupInformation object is updated to have + * the new credentials. + * @throws IOException raised on errors performing I/O. + * @throws KerberosAuthException on a failure + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public void reloginFromTicketCache() throws IOException { + if (!shouldRelogin() || !isFromTicket()) { + return; + } + HadoopLoginContext login = getLogin(); + if (login == null) { + throw new KerberosAuthException(MUST_FIRST_LOGIN); + } + relogin(login, false); + } + + private void relogin(HadoopLoginContext login, boolean ignoreLastLoginTime) + throws IOException { + // ensure the relogin is atomic to avoid leaving credentials in an + // inconsistent state. prevents other ugi instances, SASL, and SPNEGO + // from accessing or altering credentials during the relogin. + synchronized(login.getSubjectLock()) { + // another racing thread may have beat us to the relogin. + if (login == getLogin()) { + unprotectedRelogin(login, ignoreLastLoginTime); + } + } + } + + private void unprotectedRelogin(HadoopLoginContext login, + boolean ignoreLastLoginTime) throws IOException { + assert Thread.holdsLock(login.getSubjectLock()); + long now = Time.now(); + if (!hasSufficientTimeElapsed(now) && !ignoreLastLoginTime) { + return; + } + // register most recent relogin attempt + user.setLastLogin(now); + try { + LOG.debug("Initiating logout for {}", getUserName()); + //clear up the kerberos state. But the tokens are not cleared! As per + //the Java kerberos login module code, only the kerberos credentials + //are cleared + login.logout(); + //login and also update the subject field of this instance to + //have the new credentials (pass it to the LoginContext constructor) + login = newLoginContext( + login.getAppName(), login.getSubject(), login.getConfiguration()); + LOG.debug("Initiating re-login for {}", getUserName()); + login.login(); + // this should be unnecessary. originally added due to improper locking + // of the subject during relogin. + fixKerberosTicketOrder(); + setLogin(login); + } catch (LoginException le) { + KerberosAuthException kae = new KerberosAuthException(LOGIN_FAILURE, le); + kae.setUser(getUserName()); + throw kae; + } + } + + /** + * Log a user in from a keytab file. Loads a user identity from a keytab + * file and login them in. This new user does not affect the currently + * logged-in user. + * @param user the principal name to load from the keytab + * @param path the path to the keytab file + * @throws IOException if the keytab file can't be read + * @return UserGroupInformation. + */ + public + static UserGroupInformation loginUserFromKeytabAndReturnUGI(String user, + String path + ) throws IOException { + if (!isSecurityEnabled()) + return UserGroupInformation.getCurrentUser(); + + LoginParams params = new LoginParams(); + params.put(LoginParam.PRINCIPAL, user); + params.put(LoginParam.KEYTAB, path); + return doSubjectLogin(null, params); + } + + private boolean hasSufficientTimeElapsed(long now) { + if (!shouldRenewImmediatelyForTests && + now - user.getLastLogin() < kerberosMinSecondsBeforeRelogin ) { + LOG.warn("Not attempting to re-login since the last re-login was " + + "attempted less than " + (kerberosMinSecondsBeforeRelogin/1000) + + " seconds before. Last Login=" + user.getLastLogin()); + return false; + } + return true; + } + + /** + * Did the login happen via keytab. + * @return true or false + * @throws IOException raised on errors performing I/O. + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static boolean isLoginKeytabBased() throws IOException { + return getLoginUser().isFromKeytab(); + } + + /** + * Did the login happen via ticket cache. + * @return true or false + * @throws IOException raised on errors performing I/O. + */ + public static boolean isLoginTicketBased() throws IOException { + return getLoginUser().isFromTicket(); + } + + /** + * Create a user from a login name. It is intended to be used for remote + * users in RPC, since it won't have any credentials. + * @param user the full user principal name, must not be empty or null + * @return the UserGroupInformation for the remote user. + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static UserGroupInformation createRemoteUser(String user) { + return createRemoteUser(user, AuthMethod.SIMPLE); + } + + /** + * Create a user from a login name. It is intended to be used for remote + * users in RPC, since it won't have any credentials. + * + * @param user the full user principal name, must not be empty or null + * @param authMethod input authMethod. + * @return the UserGroupInformation for the remote user. + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static UserGroupInformation createRemoteUser(String user, AuthMethod authMethod) { + if (user == null || user.isEmpty()) { + throw new IllegalArgumentException("Null user"); + } + Subject subject = new Subject(); + subject.getPrincipals().add(new User(user)); + UserGroupInformation result = new UserGroupInformation(subject); + result.setAuthenticationMethod(authMethod); + return result; + } + + /** + * existing types of authentications' methods + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public enum AuthenticationMethod { + // currently we support only one auth per method, but eventually a + // subtype is needed to differentiate, ex. if digest is token or ldap + SIMPLE(AuthMethod.SIMPLE, + HadoopConfiguration.SIMPLE_CONFIG_NAME), + KERBEROS(AuthMethod.KERBEROS, + HadoopConfiguration.KERBEROS_CONFIG_NAME), + TOKEN(AuthMethod.TOKEN), + CERTIFICATE(null), + KERBEROS_SSL(null), + PROXY(null); + + private final AuthMethod authMethod; + private final String loginAppName; + + private AuthenticationMethod(AuthMethod authMethod) { + this(authMethod, null); + } + private AuthenticationMethod(AuthMethod authMethod, String loginAppName) { + this.authMethod = authMethod; + this.loginAppName = loginAppName; + } + + public AuthMethod getAuthMethod() { + return authMethod; + } + + String getLoginAppName() { + if (loginAppName == null) { + throw new UnsupportedOperationException( + this + " login authentication is not supported"); + } + return loginAppName; + } + + public static AuthenticationMethod valueOf(AuthMethod authMethod) { + for (AuthenticationMethod value : values()) { + if (value.getAuthMethod() == authMethod) { + return value; + } + } + throw new IllegalArgumentException( + "no authentication method for " + authMethod); + } + }; + + /** + * Create a proxy user using username of the effective user and the ugi of the + * real user. + * + * @param user input user. + * @param realUser input realUser. + * @return proxyUser ugi + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static UserGroupInformation createProxyUser(String user, + UserGroupInformation realUser) { + if (user == null || user.isEmpty()) { + throw new IllegalArgumentException("Null user"); + } + if (realUser == null) { + throw new IllegalArgumentException("Null real user"); + } + Subject subject = new Subject(); + Set principals = subject.getPrincipals(); + principals.add(new User(user, AuthenticationMethod.PROXY, null)); + principals.add(new RealUser(realUser)); + return new UserGroupInformation(subject); + } + + /** + * get RealUser (vs. EffectiveUser) + * @return realUser running over proxy user + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public UserGroupInformation getRealUser() { + for (RealUser p: subject.getPrincipals(RealUser.class)) { + return p.getRealUser(); + } + return null; + } + + /** + * If this is a proxy user, get the real user. Otherwise, return + * this user. + * @param user the user to check + * @return the real user or self + */ + public static UserGroupInformation getRealUserOrSelf(UserGroupInformation user) { + if (user == null) { + return null; + } + UserGroupInformation real = user.getRealUser(); + return real != null ? real : user; + } + + /** + * This class is used for storing the groups for testing. It stores a local + * map that has the translation of usernames to groups. + */ + private static class TestingGroups extends Groups { + private final Map> userToGroupsMapping = + new HashMap>(); + private Groups underlyingImplementation; + + private TestingGroups(Groups underlyingImplementation) { + super(new org.apache.hadoop.conf.Configuration()); + this.underlyingImplementation = underlyingImplementation; + } + + @Override + public List getGroups(String user) throws IOException { + List result = userToGroupsMapping.get(user); + + if (result == null) { + result = underlyingImplementation.getGroups(user); + } + + return result; + } + + private void setUserGroups(String user, String[] groups) { + userToGroupsMapping.put(user, Arrays.asList(groups)); + } + } + + /** + * Create a UGI for testing HDFS and MapReduce + * @param user the full user principal name + * @param userGroups the names of the groups that the user belongs to + * @return a fake user for running unit tests + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static UserGroupInformation createUserForTesting(String user, + String[] userGroups) { + ensureInitialized(); + UserGroupInformation ugi = createRemoteUser(user); + // make sure that the testing object is setup + if (!(groups instanceof TestingGroups)) { + groups = new TestingGroups(groups); + } + // add the user groups + ((TestingGroups) groups).setUserGroups(ugi.getShortUserName(), userGroups); + return ugi; + } + + + /** + * Create a proxy user UGI for testing HDFS and MapReduce + * + * @param user + * the full user principal name for effective user + * @param realUser + * UGI of the real user + * @param userGroups + * the names of the groups that the user belongs to + * @return a fake user for running unit tests + */ + public static UserGroupInformation createProxyUserForTesting(String user, + UserGroupInformation realUser, String[] userGroups) { + ensureInitialized(); + UserGroupInformation ugi = createProxyUser(user, realUser); + // make sure that the testing object is setup + if (!(groups instanceof TestingGroups)) { + groups = new TestingGroups(groups); + } + // add the user groups + ((TestingGroups) groups).setUserGroups(ugi.getShortUserName(), userGroups); + return ugi; + } + + /** + * Get the user's login name. + * @return the user's name up to the first '/' or '@'. + */ + public String getShortUserName() { + return user.getShortName(); + } + + public String getPrimaryGroupName() throws IOException { + List groups = getGroups(); + if (groups.isEmpty()) { + throw new IOException("There is no primary group for UGI " + this); + } + return groups.get(0); + } + + /** + * Get the user's full principal name. + * @return the user's full principal name. + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public String getUserName() { + return user.getName(); + } + + /** + * Add a TokenIdentifier to this UGI. The TokenIdentifier has typically been + * authenticated by the RPC layer as belonging to the user represented by this + * UGI. + * + * @param tokenId + * tokenIdentifier to be added + * @return true on successful add of new tokenIdentifier + */ + public synchronized boolean addTokenIdentifier(TokenIdentifier tokenId) { + return subject.getPublicCredentials().add(tokenId); + } + + /** + * Get the set of TokenIdentifiers belonging to this UGI + * + * @return the set of TokenIdentifiers belonging to this UGI + */ + public synchronized Set getTokenIdentifiers() { + return subject.getPublicCredentials(TokenIdentifier.class); + } + + /** + * Add a token to this UGI + * + * @param token Token to be added + * @return true on successful add of new token + */ + public boolean addToken(Token token) { + return (token != null) ? addToken(token.getService(), token) : false; + } + + /** + * Add a named token to this UGI + * + * @param alias Name of the token + * @param token Token to be added + * @return true on successful add of new token + */ + public boolean addToken(Text alias, Token token) { + synchronized (subject) { + getCredentialsInternal().addToken(alias, token); + return true; + } + } + + /** + * Obtain the collection of tokens associated with this user. + * + * @return an unmodifiable collection of tokens associated with user + */ + public Collection> getTokens() { + synchronized (subject) { + return Collections.unmodifiableCollection( + new ArrayList>(getCredentialsInternal().getAllTokens())); + } + } + + /** + * Obtain the tokens in credentials form associated with this user. + * + * @return Credentials of tokens associated with this user + */ + public Credentials getCredentials() { + synchronized (subject) { + Credentials creds = new Credentials(getCredentialsInternal()); + Iterator> iter = creds.getAllTokens().iterator(); + while (iter.hasNext()) { + if (iter.next().isPrivate()) { + iter.remove(); + } + } + return creds; + } + } + + /** + * Add the given Credentials to this user. + * @param credentials of tokens and secrets + */ + public void addCredentials(Credentials credentials) { + synchronized (subject) { + getCredentialsInternal().addAll(credentials); + } + } + + private synchronized Credentials getCredentialsInternal() { + final Credentials credentials; + final Set credentialsSet = + subject.getPrivateCredentials(Credentials.class); + if (!credentialsSet.isEmpty()){ + credentials = credentialsSet.iterator().next(); + } else { + credentials = new Credentials(); + subject.getPrivateCredentials().add(credentials); + } + return credentials; + } + + /** + * Get the group names for this user. {@link #getGroups()} is less + * expensive alternative when checking for a contained element. + * @return the list of users with the primary group first. If the command + * fails, it returns an empty list. + */ + public String[] getGroupNames() { + List groups = getGroups(); + return groups.toArray(new String[groups.size()]); + } + + /** + * Get the group names for this user. + * @return the list of users with the primary group first. If the command + * fails, it returns an empty list. + */ + public List getGroups() { + ensureInitialized(); + try { + return groups.getGroups(getShortUserName()); + } catch (IOException ie) { + LOG.debug("Failed to get groups for user {}", getShortUserName(), ie); + return Collections.emptyList(); + } + } + + /** + * Return the username. + */ + @Override + public String toString() { + StringBuilder sb = new StringBuilder(getUserName()); + sb.append(" (auth:"+getAuthenticationMethod()+")"); + if (getRealUser() != null) { + sb.append(" via ").append(getRealUser().toString()); + } + return sb.toString(); + } + + /** + * Sets the authentication method in the subject. + * + * @param authMethod input authMethod. + */ + public synchronized + void setAuthenticationMethod(AuthenticationMethod authMethod) { + user.setAuthenticationMethod(authMethod); + } + + /** + * Sets the authentication method in the subject. + * + * @param authMethod input authMethod. + */ + public void setAuthenticationMethod(AuthMethod authMethod) { + user.setAuthenticationMethod(AuthenticationMethod.valueOf(authMethod)); + } + + /** + * Get the authentication method from the subject + * + * @return AuthenticationMethod in the subject, null if not present. + */ + public synchronized AuthenticationMethod getAuthenticationMethod() { + return user.getAuthenticationMethod(); + } + + /** + * Get the authentication method from the real user's subject. If there + * is no real user, return the given user's authentication method. + * + * @return AuthenticationMethod in the subject, null if not present. + */ + public synchronized AuthenticationMethod getRealAuthenticationMethod() { + UserGroupInformation ugi = getRealUser(); + if (ugi == null) { + ugi = this; + } + return ugi.getAuthenticationMethod(); + } + + /** + * Returns the authentication method of a ugi. If the authentication method is + * PROXY, returns the authentication method of the real user. + * + * @param ugi input ugi. + * @return AuthenticationMethod + */ + public static AuthenticationMethod getRealAuthenticationMethod( + UserGroupInformation ugi) { + AuthenticationMethod authMethod = ugi.getAuthenticationMethod(); + if (authMethod == AuthenticationMethod.PROXY) { + authMethod = ugi.getRealUser().getAuthenticationMethod(); + } + return authMethod; + } + + /** + * Compare the subjects to see if they are equal to each other. + */ + @Override + public boolean equals(Object o) { + if (o == this) { + return true; + } else if (o == null || getClass() != o.getClass()) { + return false; + } else { + return subject == ((UserGroupInformation) o).subject; + } + } + + /** + * Return the hash of the subject. + */ + @Override + public int hashCode() { + return System.identityHashCode(subject); + } + + /** + * Get the underlying subject from this ugi. + * @return the subject that represents this user. + */ + protected Subject getSubject() { + return subject; + } + + /** + * Run the given action as the user. + * @param the return type of the run method + * @param action the method to execute + * @return the value from the run method + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public T doAs(PrivilegedAction action) { + if (LOG.isDebugEnabled()) { + LOG.debug("PrivilegedAction [as: {}][action: {}]", this, action, + new Exception()); + } + return Subject.doAs(subject, action); + } + + /** + * Run the given action as the user, potentially throwing an exception. + * @param the return type of the run method + * @param action the method to execute + * @return the value from the run method + * @throws IOException if the action throws an IOException + * @throws Error if the action throws an Error + * @throws RuntimeException if the action throws a RuntimeException + * @throws InterruptedException if the action throws an InterruptedException + * @throws UndeclaredThrowableException if the action throws something else + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public T doAs(PrivilegedExceptionAction action + ) throws IOException, InterruptedException { + try { + if (LOG.isDebugEnabled()) { + LOG.debug("PrivilegedAction [as: {}][action: {}]", this, action, + new Exception()); + } + return Subject.doAs(subject, action); + } catch (PrivilegedActionException pae) { + Throwable cause = pae.getCause(); + LOG.debug("PrivilegedActionException as: {}", this, cause); + if (cause == null) { + throw new RuntimeException("PrivilegedActionException with no " + + "underlying cause. UGI [" + this + "]" +": " + pae, pae); + } else if (cause instanceof IOException) { + throw (IOException) cause; + } else if (cause instanceof Error) { + throw (Error) cause; + } else if (cause instanceof RuntimeException) { + throw (RuntimeException) cause; + } else if (cause instanceof InterruptedException) { + throw (InterruptedException) cause; + } else { + throw new UndeclaredThrowableException(cause); + } + } + } + + /** + * Log current UGI and token information into specified log. + * + * @param log input log. + * @param caption input caption. + * @param ugi - UGI. + * @throws IOException raised on errors performing I/O. + */ + @InterfaceAudience.LimitedPrivate({"HDFS", "KMS"}) + @InterfaceStability.Unstable + public static void logUserInfo(Logger log, String caption, + UserGroupInformation ugi) throws IOException { + if (log.isDebugEnabled()) { + log.debug(caption + " UGI: " + ugi); + for (Token token : ugi.getTokens()) { + log.debug("+token:" + token); + } + } + } + + /** + * Log all (current, real, login) UGI and token info into specified log. + * + * @param log input log. + * @param ugi - UGI + * @throws IOException raised on errors performing I/O. + */ + @InterfaceAudience.LimitedPrivate({"HDFS", "KMS"}) + @InterfaceStability.Unstable + public static void logAllUserInfo(Logger log, UserGroupInformation ugi) throws + IOException { + if (log.isDebugEnabled()) { + logUserInfo(log, "Current", ugi.getCurrentUser()); + if (ugi.getRealUser() != null) { + logUserInfo(log, "Real", ugi.getRealUser()); + } + logUserInfo(log, "Login", ugi.getLoginUser()); + } + } + + /** + * Log all (current, real, login) UGI and token info into UGI debug log. + * @param ugi - UGI + * @throws IOException raised on errors performing I/O. + */ + public static void logAllUserInfo(UserGroupInformation ugi) throws + IOException { + logAllUserInfo(LOG, ugi); + } + + private void print() throws IOException { + System.out.println("User: " + getUserName()); + System.out.print("Group Ids: "); + System.out.println(); + String[] groups = getGroupNames(); + System.out.print("Groups: "); + for(int i=0; i < groups.length; i++) { + System.out.print(groups[i] + " "); + } + System.out.println(); + } + + /** + * Login a subject with the given parameters. If the subject is null, + * the login context used to create the subject will be attached. + * @param subject to login, null for new subject. + * @param params for login, null for externally managed ugi. + * @return UserGroupInformation for subject + * @throws IOException + */ + private static UserGroupInformation doSubjectLogin( + Subject subject, LoginParams params) throws IOException { + ensureInitialized(); + // initial default login. + if (subject == null && params == null) { + params = LoginParams.getDefaults(); + } + HadoopConfiguration loginConf = new HadoopConfiguration(params); + try { + HadoopLoginContext login = newLoginContext( + authenticationMethod.getLoginAppName(), subject, loginConf); + login.login(); + UserGroupInformation ugi = new UserGroupInformation(login.getSubject()); + // attach login context for relogin unless this was a pre-existing + // subject. + if (subject == null) { + params.put(LoginParam.PRINCIPAL, ugi.getUserName()); + ugi.setLogin(login); + ugi.setLastLogin(Time.now()); + } + return ugi; + } catch (LoginException le) { + KerberosAuthException kae = + new KerberosAuthException(FAILURE_TO_LOGIN, le); + if (params != null) { + kae.setPrincipal(params.get(LoginParam.PRINCIPAL)); + kae.setKeytabFile(params.get(LoginParam.KEYTAB)); + kae.setTicketCacheFile(params.get(LoginParam.CCACHE)); + } + throw kae; + } + } + + // parameters associated with kerberos logins. may be extended to support + // additional authentication methods. + enum LoginParam { + PRINCIPAL, + KEYTAB, + CCACHE, + } + + // explicitly private to prevent external tampering. + private static class LoginParams extends EnumMap + implements Parameters { + LoginParams() { + super(LoginParam.class); + } + + // do not add null values, nor allow existing values to be overriden. + @Override + public String put(LoginParam param, String val) { + boolean add = val != null && !containsKey(param); + return add ? super.put(param, val) : null; + } + + static LoginParams getDefaults() { + LoginParams params = new LoginParams(); + params.put(LoginParam.PRINCIPAL, System.getenv("KRB5PRINCIPAL")); + params.put(LoginParam.KEYTAB, System.getenv("KRB5KEYTAB")); + params.put(LoginParam.CCACHE, System.getenv("KRB5CCNAME")); + return params; + } + } + + // wrapper to allow access to fields necessary to recreate the same login + // context for relogin. explicitly private to prevent external tampering. + private static class HadoopLoginContext extends LoginContext { + private final String appName; + private final HadoopConfiguration conf; + private AtomicBoolean isLoggedIn = new AtomicBoolean(); + + HadoopLoginContext(String appName, Subject subject, + HadoopConfiguration conf) throws LoginException { + super(appName, subject, null, conf); + this.appName = appName; + this.conf = conf; + } + + String getAppName() { + return appName; + } + + HadoopConfiguration getConfiguration() { + return conf; + } + + // the locking model for logins cannot rely on ugi instance synchronization + // since a subject will be referenced by multiple ugi instances. + Object getSubjectLock() { + Subject subject = getSubject(); + // if subject is null, the login context will create the subject + // so just lock on this context. + return (subject == null) ? this : subject.getPrivateCredentials(); + } + + @Override + public void login() throws LoginException { + synchronized(getSubjectLock()) { + MutableRate metric = metrics.loginFailure; + long start = Time.monotonicNow(); + try { + super.login(); + isLoggedIn.set(true); + metric = metrics.loginSuccess; + } finally { + metric.add(Time.monotonicNow() - start); + } + } + } + + @Override + public void logout() throws LoginException { + synchronized(getSubjectLock()) { + if (isLoggedIn.compareAndSet(true, false)) { + super.logout(); + } + } + } + } + + /** + * A JAAS configuration that defines the login modules that we want + * to use for login. + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + private static class HadoopConfiguration + extends javax.security.auth.login.Configuration { + static final String KRB5_LOGIN_MODULE = + KerberosUtil.getKrb5LoginModuleName(); + static final String SIMPLE_CONFIG_NAME = "hadoop-simple"; + static final String KERBEROS_CONFIG_NAME = "hadoop-kerberos"; + + private static final Map BASIC_JAAS_OPTIONS = + new HashMap(); + static { + if ("true".equalsIgnoreCase(System.getenv("HADOOP_JAAS_DEBUG"))) { + BASIC_JAAS_OPTIONS.put("debug", "true"); + } + } + + static final AppConfigurationEntry OS_SPECIFIC_LOGIN = + new AppConfigurationEntry( + OS_LOGIN_MODULE_NAME, + LoginModuleControlFlag.REQUIRED, + BASIC_JAAS_OPTIONS); + + static final AppConfigurationEntry HADOOP_LOGIN = + new AppConfigurationEntry( + HadoopLoginModule.class.getName(), + LoginModuleControlFlag.REQUIRED, + BASIC_JAAS_OPTIONS); + + private final LoginParams params; + + HadoopConfiguration(LoginParams params) { + this.params = params; + } + + @Override + public LoginParams getParameters() { + return params; + } + + @Override + public AppConfigurationEntry[] getAppConfigurationEntry(String appName) { + ArrayList entries = new ArrayList<>(); + // login of external subject passes no params. technically only + // existing credentials should be used but other components expect + // the login to succeed with local user fallback if no principal. + if (params == null || appName.equals(SIMPLE_CONFIG_NAME)) { + entries.add(OS_SPECIFIC_LOGIN); + } else if (appName.equals(KERBEROS_CONFIG_NAME)) { + // existing semantics are the initial default login allows local user + // fallback. this is not allowed when a principal explicitly + // specified or during a relogin. + if (!params.containsKey(LoginParam.PRINCIPAL)) { + entries.add(OS_SPECIFIC_LOGIN); + } + entries.add(getKerberosEntry()); + } + entries.add(HADOOP_LOGIN); + return entries.toArray(new AppConfigurationEntry[0]); + } + + private AppConfigurationEntry getKerberosEntry() { + final Map options = new HashMap<>(BASIC_JAAS_OPTIONS); + LoginModuleControlFlag controlFlag = LoginModuleControlFlag.OPTIONAL; + // kerberos login is mandatory if principal is specified. principal + // will not be set for initial default login, but will always be set + // for relogins. + final String principal = params.get(LoginParam.PRINCIPAL); + if (principal != null) { + options.put("principal", principal); + controlFlag = LoginModuleControlFlag.REQUIRED; + } + + // use keytab if given else fallback to ticket cache. + if (IBM_JAVA) { + if (params.containsKey(LoginParam.KEYTAB)) { + final String keytab = params.get(LoginParam.KEYTAB); + if (keytab != null) { + options.put("useKeytab", prependFileAuthority(keytab)); + } else { + options.put("useDefaultKeytab", "true"); + } + options.put("credsType", "both"); + } else { + String ticketCache = params.get(LoginParam.CCACHE); + if (ticketCache != null) { + options.put("useCcache", prependFileAuthority(ticketCache)); + } else { + options.put("useDefaultCcache", "true"); + } + options.put("renewTGT", "true"); + } + } else { + if (params.containsKey(LoginParam.KEYTAB)) { + options.put("useKeyTab", "true"); + final String keytab = params.get(LoginParam.KEYTAB); + if (keytab != null) { + options.put("keyTab", keytab); + } + options.put("storeKey", "true"); + } else { + options.put("useTicketCache", "true"); + String ticketCache = params.get(LoginParam.CCACHE); + if (ticketCache != null) { + options.put("ticketCache", ticketCache); + } + options.put("renewTGT", "true"); + } + options.put("doNotPrompt", "true"); + } + options.put("refreshKrb5Config", "true"); + + return new AppConfigurationEntry( + KRB5_LOGIN_MODULE, controlFlag, options); + } + + private static String prependFileAuthority(String keytabPath) { + return keytabPath.startsWith("file://") + ? keytabPath + : "file://" + keytabPath; + } + } + + /** + * A test method to print out the current user's UGI. + * @param args if there are two arguments, read the user from the keytab + * and print it out. + * @throws Exception if any error occurs. + */ + public static void main(String [] args) throws Exception { + System.out.println("Getting UGI for current user"); + UserGroupInformation ugi = getCurrentUser(); + ugi.print(); + System.out.println("UGI: " + ugi); + System.out.println("Auth method " + ugi.user.getAuthenticationMethod()); + System.out.println("Keytab " + ugi.isFromKeytab()); + System.out.println("============================================================"); + + if (args.length == 2) { + System.out.println("Getting UGI from keytab...."); + loginUserFromKeytab(args[0], args[1]); + getCurrentUser().print(); + System.out.println("Keytab: " + ugi); + UserGroupInformation loginUgi = getLoginUser(); + System.out.println("Auth method " + loginUgi.getAuthenticationMethod()); + System.out.println("Keytab " + loginUgi.isFromKeytab()); + } + } +} diff --git a/pom.xml b/pom.xml index 861d406c5..893a31daa 100644 --- a/pom.xml +++ b/pom.xml @@ -1518,5 +1518,45 @@ + + java24-dependencies + + 24 + + + 24 + 24 + 24 + 24 + 24 + + + + patch/addax-kerberos-jdk24-patch + + + + + com.wgzhao.addax + addax-kerberos-jdk24-patch + ${project.version} + provided + + + + + + com.fasterxml.woodstox + woodstox-core + ${woodstox.version} + + + com.google.protobuf + protobuf-java + ${protobuf.version} + + + + From da3000f5b4dcfac2f7d5c014011297bc3c91e49e Mon Sep 17 00:00:00 2001 From: awol2005ex Date: Wed, 13 Aug 2025 13:11:05 +0800 Subject: [PATCH 40/40] [bugfixed] bug for kerberos error when jdk24 --- .../wgzhao/addax/rdbms/reader/CommonRdbmsReader.java | 7 +------ .../java/com/wgzhao/addax/rdbms/util/DataBaseType.java | 10 +--------- plugin/reader/hivereader/pom.xml | 2 -- 3 files changed, 2 insertions(+), 17 deletions(-) diff --git a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/CommonRdbmsReader.java b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/CommonRdbmsReader.java index 9f9447072..a0ec74d85 100644 --- a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/CommonRdbmsReader.java +++ b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/reader/CommonRdbmsReader.java @@ -280,12 +280,7 @@ protected Column createColumn(ResultSet rs, ResultSetMetaData metaData, int i) return new DateColumn(rs.getDate(i)); case Types.TIMESTAMP: - if(!"org.apache.hive.jdbc.HiveDriver".equals(this.dataBaseType.getDriverClassName())){ - return new TimestampColumn(rs.getTimestamp(i, Calendar.getInstance())); - }else{ - //hive not support method(Timestamp getTimestamp(int columnIndex, Calendar cal)) - return new TimestampColumn(rs.getTimestamp(i)); - } + return new TimestampColumn(rs.getTimestamp(i, Calendar.getInstance())); case Types.BINARY: case Types.VARBINARY: diff --git a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java index f236d5673..61f1cd6b4 100644 --- a/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java +++ b/lib/addax-rdbms/src/main/java/com/wgzhao/addax/rdbms/util/DataBaseType.java @@ -48,15 +48,7 @@ public enum DataBaseType Sybase("sybase", "com.sybase.jdbc4.jdbc.SybDriver"), Databend("databend", "com.databend.jdbc.DatabendDriver"), Access("access","net.ucanaccess.jdbc.UcanaccessDriver"), - HANA("hana", "com.sap.db.jdbc.Driver"), - VERTICA("vertica", "com.vertica.jdbc.Driver"), - DM("dm","dm.jdbc.driver.DmDriver"), - OSCAR("oscar","com.oscar.Driver"), - KINGBASE8("kingbase8","com.kingbase8.Driver"), - HIGHGO("highgo","com.highgo.jdbc.Driver"), - OCEANBASE("oceanbase","com.alipay.oceanbase.jdbc.Driver"), - GOLDENDB("goldendb","com.goldendb.jdbc.Driver"), - GBASEDBT("gbasedbt-sqli","com.gbasedbt.jdbc.Driver"); + HANA("hana", "com.sap.db.jdbc.Driver"); private static final Pattern jdbcUrlPattern = Pattern.compile("jdbc:\\w+:(?:thin:url=|//|thin:@|)([\\w\\d.,]+).*"); diff --git a/plugin/reader/hivereader/pom.xml b/plugin/reader/hivereader/pom.xml index 662207f92..18a6915a1 100644 --- a/plugin/reader/hivereader/pom.xml +++ b/plugin/reader/hivereader/pom.xml @@ -35,8 +35,6 @@ 2.1.1 - 3.2.4 - 4.2.0