diff --git a/pom.xml b/pom.xml index 47ffaa4a..d170ef50 100644 --- a/pom.xml +++ b/pom.xml @@ -29,6 +29,7 @@ -SNAPSHOT + 3.3.6 1.8 2.3.0 1.8 @@ -104,18 +105,36 @@ spark-streaming_2.12 3.4.0 provided + + + org.apache.hadoop + * + + org.apache.spark spark-sql-kafka-0-10_2.12 3.4.0 provided + + + org.apache.hadoop + * + + org.apache.spark spark-sql_2.12 3.4.0 provided + + + org.apache.hadoop + * + + com.amazonaws @@ -130,6 +149,41 @@ config 1.4.0 + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + + + org.apache.hadoop + hadoop-minicluster + ${hadoop.version} + test + + + com.google.code.gson + gson + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + + + com.google.code.gson + gson + + + + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version} + test + org.mockito @@ -215,6 +269,17 @@ 2.1.0 compile + + + org.xerial.snappy + snappy-java + 1.1.10.5 + + + org.apache.avro + avro + 1.11.3 + ${project.basedir}/target @@ -286,6 +351,9 @@ README.adoc README.md + + src/main/avro/KafkaRecord.avsc + src/main/java/com/teragrep/pth_06/avro/SyslogRecord.java @@ -297,6 +365,23 @@ + + org.apache.avro + avro-maven-plugin + 1.11.3 + + + + schema + + generate-sources + + ${project.basedir}/src/main/avro/ + ${project.basedir}/src/main/java/ + + + + org.jooq jooq-codegen-maven @@ -350,6 +435,7 @@ src/main/java/com/teragrep/pth_06/jooq/generated/**/*.* + src/main/java/com/teragrep/pth_06/avro/SyslogRecord.java @@ -434,6 +520,224 @@ true + + org.apache.maven.plugins + maven-checkstyle-plugin + 3.5.0 + + + + scan-errors + + check + + process-classes + + error + true + true + false + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + scan-warnings + + check + + process-classes + + warning + true + false + false + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/src/main/avro/KafkaRecord.avsc b/src/main/avro/KafkaRecord.avsc new file mode 100644 index 00000000..78406df7 --- /dev/null +++ b/src/main/avro/KafkaRecord.avsc @@ -0,0 +1,15 @@ +{"namespace": "com.teragrep.pth_06.avro", + "type": "record", + "name": "SyslogRecord", + "fields": [ + {"name": "timestamp", "type": "long"}, + {"name": "directory", "type": "string"}, + {"name": "stream", "type": "string"}, + {"name": "host", "type": "string"}, + {"name": "input", "type": "string"}, + {"name": "partition", "type": "string"}, + {"name": "offset", "type": "long"}, + {"name": "origin", "type": "string"}, + {"name": "payload", "type": "string"} + ] +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index d8025bc8..6dba2ae0 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -46,12 +46,15 @@ package com.teragrep.pth_06; import com.google.common.annotations.VisibleForTesting; +import com.google.gson.JsonArray; import com.teragrep.pth_06.config.Config; import com.teragrep.pth_06.planner.*; import com.teragrep.pth_06.planner.offset.DatasourceOffset; +import com.teragrep.pth_06.planner.offset.HdfsOffset; import com.teragrep.pth_06.planner.offset.KafkaOffset; import com.teragrep.pth_06.scheduler.*; import com.teragrep.pth_06.task.ArchiveMicroBatchInputPartition; +import com.teragrep.pth_06.task.HdfsMicroBatchInputPartition; import com.teragrep.pth_06.task.TeragrepPartitionReaderFactory; import com.teragrep.pth_06.task.KafkaMicroBatchInputPartition; import org.apache.spark.sql.connector.read.InputPartition; @@ -87,6 +90,8 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { private final Config config; private final ArchiveQuery aq; private final KafkaQuery kq; + private final HdfsQuery hq; + private final JsonArray hdfsOffsets; /** * Constructor for ArchiveMicroStreamReader @@ -98,6 +103,15 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { this.config = config; + if (config.isHdfsEnabled) { + this.hq = new HdfsQueryProcessor(config); + hdfsOffsets = hq.hdfsOffsetMapToJSON(); + } + else { + this.hq = new HdfsQueryProcessor(); + hdfsOffsets = new JsonArray(); + } + if (config.isArchiveEnabled) { this.aq = new ArchiveQueryProcessor(config); } @@ -106,7 +120,13 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { } if (config.isKafkaEnabled) { - this.kq = new KafkaQueryProcessor(config); + if (config.isHdfsEnabled) { + this.kq = new KafkaQueryProcessor(config); + this.kq.seekToHdfsOffsets(hdfsOffsets); + } + else { + this.kq = new KafkaQueryProcessor(config); + } } else { this.kq = null; @@ -119,10 +139,18 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { * Used for testing. */ @VisibleForTesting - ArchiveMicroStreamReader(ArchiveQuery aq, KafkaQuery kq, Config config) { + ArchiveMicroStreamReader(HdfsQuery hq, ArchiveQuery aq, KafkaQuery kq, Config config) { this.config = config; - this.aq = aq; - this.kq = kq; + this.aq = aq; // Uses its own hardcoded query string defined in MockTeragrepDatasource. + this.kq = kq; // Skips using query string (and thus topic filtering) altogether. + this.hq = hq; // Uses the query string from config for topic filtering. + if (!this.hq.isStub() && this.kq != null) { + hdfsOffsets = this.hq.hdfsOffsetMapToJSON(); + this.kq.seekToHdfsOffsets(hdfsOffsets); + } + else { + hdfsOffsets = new JsonArray(); + } LOGGER.debug("@VisibleForTesting MicroBatchReader> initialized"); } @@ -137,25 +165,37 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { public Offset initialOffset() { // archive only: subtract 3600s (1 hour) from earliest to return first row (start exclusive) DatasourceOffset rv; - if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled) { - // only archive - rv = new DatasourceOffset(new LongOffset(this.aq.getInitialOffset() - 3600L)); + HdfsOffset hdfsOffset = new HdfsOffset(); // stub + LongOffset longOffset = null; // Refactor null usage + KafkaOffset kafkaOffset = new KafkaOffset(); // stub + + if (this.config.isHdfsEnabled) { + hdfsOffset = new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()); + hdfsOffset.serialize(); } - else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled) { - // only kafka - rv = new DatasourceOffset(new KafkaOffset(this.kq.getBeginningOffsets(null))); + if (this.config.isArchiveEnabled) { + longOffset = new LongOffset(this.aq.getInitialOffset() - 3600L); } - else if (this.config.isArchiveEnabled) { - // both - rv = new DatasourceOffset( - new LongOffset(this.aq.getInitialOffset() - 3600L), - new KafkaOffset(this.kq.getBeginningOffsets(null)) - ); + if (this.config.isKafkaEnabled) { + if (this.config.isHdfsEnabled) { + if (hdfsOffsets.size() > 0) { + kafkaOffset = new KafkaOffset(this.kq.getConsumerPositions(hdfsOffsets)); + kafkaOffset.serialize(); + } + else { + kafkaOffset = new KafkaOffset(this.kq.getBeginningOffsets(null)); + kafkaOffset.serialize(); + } + } + else { + kafkaOffset = new KafkaOffset(this.kq.getBeginningOffsets(null)); + kafkaOffset.serialize(); + } } - else { - // neither - throw new IllegalStateException("no datasources enabled, can't get initial offset"); + if (hdfsOffset.isStub() && longOffset == null && kafkaOffset.isStub()) { + throw new IllegalStateException("no datasources enabled, can't get latest offset"); } + rv = new DatasourceOffset(hdfsOffset, longOffset, kafkaOffset); LOGGER.debug("offset[initial]= {}", rv); return rv; } @@ -172,6 +212,9 @@ public void commit(Offset offset) { if (this.config.isArchiveEnabled) { this.aq.commit(((DatasourceOffset) offset).getArchiveOffset().offset()); } + if (this.config.isHdfsEnabled) { + this.hq.commit(((DatasourceOffset) offset).getHdfsOffset()); + } } /** {@inheritDoc} */ @@ -188,26 +231,25 @@ public void stop() { @Override public Offset latestOffset() { DatasourceOffset rv; - if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled) { - // only archive - rv = new DatasourceOffset(new LongOffset(this.aq.incrementAndGetLatestOffset())); + HdfsOffset hdfsOffset = new HdfsOffset(); + LongOffset longOffset = null; + KafkaOffset kafkaOffset = new KafkaOffset(); + + if (this.config.isHdfsEnabled) { + hdfsOffset = new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap()); + hdfsOffset.serialize(); } - else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled) { - // only kafka - rv = new DatasourceOffset(new KafkaOffset(this.kq.getInitialEndOffsets())); + if (this.config.isArchiveEnabled) { + longOffset = new LongOffset(this.aq.incrementAndGetLatestOffset()); } - else if (this.config.isArchiveEnabled) { - // both - rv = new DatasourceOffset( - new LongOffset(this.aq.incrementAndGetLatestOffset()), - new KafkaOffset(this.kq.getInitialEndOffsets()) - ); + if (this.config.isKafkaEnabled) { + kafkaOffset = new KafkaOffset(this.kq.getInitialEndOffsets()); + kafkaOffset.serialize(); } - else { - // neither + if (hdfsOffset.isStub() && longOffset == null && kafkaOffset.isStub()) { throw new IllegalStateException("no datasources enabled, can't get latest offset"); } - + rv = new DatasourceOffset(hdfsOffset, longOffset, kafkaOffset); LOGGER.debug("offset[latest]= {}", rv); return rv; } @@ -223,18 +265,34 @@ else if (this.config.isArchiveEnabled) { public InputPartition[] planInputPartitions(Offset start, Offset end) { List inputPartitions = new LinkedList<>(); - Batch currentBatch = new Batch(config, aq, kq).processRange(start, end); + Batch currentBatch = new Batch(config, hq, aq, kq).processRange(start, end); for (LinkedList taskObjectList : currentBatch) { - // archive tasks LinkedList archiveTaskList = new LinkedList<>(); + // HDFS tasks + LinkedList hdfsTaskList = new LinkedList<>(); for (BatchSlice batchSlice : taskObjectList) { if (batchSlice.type.equals(BatchSlice.Type.ARCHIVE)) { archiveTaskList.add(batchSlice.archiveS3ObjectMetadata); } + if (batchSlice.type.equals(BatchSlice.Type.HDFS)) { + hdfsTaskList.add(batchSlice.hdfsFileMetadata); + } + if (batchSlice.type.equals(BatchSlice.Type.KAFKA)) { + inputPartitions + .add( + new KafkaMicroBatchInputPartition( + config.kafkaConfig.executorOpts, + batchSlice.kafkaTopicPartitionOffsetMetadata.topicPartition, + batchSlice.kafkaTopicPartitionOffsetMetadata.startOffset, + batchSlice.kafkaTopicPartitionOffsetMetadata.endOffset, + config.kafkaConfig.executorConfig, + config.kafkaConfig.skipNonRFC5424Records + ) + ); + } } - if (!archiveTaskList.isEmpty()) { inputPartitions .add( @@ -251,22 +309,8 @@ public InputPartition[] planInputPartitions(Offset start, Offset end) { ) ); } - - // kafka tasks - for (BatchSlice batchSlice : taskObjectList) { - if (batchSlice.type.equals(BatchSlice.Type.KAFKA)) { - inputPartitions - .add( - new KafkaMicroBatchInputPartition( - config.kafkaConfig.executorOpts, - batchSlice.kafkaTopicPartitionOffsetMetadata.topicPartition, - batchSlice.kafkaTopicPartitionOffsetMetadata.startOffset, - batchSlice.kafkaTopicPartitionOffsetMetadata.endOffset, - config.kafkaConfig.executorConfig, - config.kafkaConfig.skipNonRFC5424Records - ) - ); - } + if (!hdfsTaskList.isEmpty()) { + inputPartitions.add(new HdfsMicroBatchInputPartition(config.hdfsConfig, hdfsTaskList)); } } diff --git a/src/main/java/com/teragrep/pth_06/FileSystemFactory.java b/src/main/java/com/teragrep/pth_06/FileSystemFactory.java new file mode 100644 index 00000000..826925ca --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/FileSystemFactory.java @@ -0,0 +1,56 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06; + +import org.apache.hadoop.fs.FileSystem; + +import java.io.IOException; + +public interface FileSystemFactory { + + public abstract FileSystem fileSystem(boolean initializeUGI) throws IOException; + +} diff --git a/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java b/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java new file mode 100644 index 00000000..3351d92b --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java @@ -0,0 +1,181 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06; + +import com.teragrep.pth_06.config.HdfsConfig; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.security.UserGroupInformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URI; + +public final class FileSystemFactoryImpl implements FileSystemFactory { + + final Logger LOGGER = LoggerFactory.getLogger(FileSystemFactoryImpl.class); + + private final String kerberosAuthentication; + private final String hdfsUri; + private final String kerberosRealm; + private final String kerberosKdc; + private final String kerberosAuthorization; + private final String kerberosAutorenewal; + private final String useHdfsHostname; + private final String kerberosPrincipalPattern; + private final String hdfsTransferProtection; + private final String hdfsCipherSuites; + private final String kerberosKeytabUser; + private final String kerberosKeytabPath; + + public FileSystemFactoryImpl(HdfsConfig config) { + this( + config.kerberosAuthentication, + config.hdfsUri, + config.kerberosRealm, + config.kerberosKdc, + config.kerberosAuthorization, + config.kerberosAutorenewal, + config.useHdfsHostname, + config.kerberosPrincipalPattern, + config.hdfsTransferProtection, + config.hdfsCipherSuites, + config.kerberosKeytabUser, + config.kerberosKeytabPath + ); + } + + public FileSystemFactoryImpl( + String kerberosAuthentication, + String hdfsUri, + String kerberosRealm, + String kerberosKdc, + String kerberosAuthorization, + String kerberosAutorenewal, + String UseHdfsHostname, + String kerberosPrincipalPattern, + String hdfsTransferProtection, + String hdfsCipherSuites, + String kerberosKeytabUser, + String kerberosKeytabPath + ) { + this.kerberosAuthentication = kerberosAuthentication; + this.hdfsUri = hdfsUri; + this.kerberosRealm = kerberosRealm; + this.kerberosKdc = kerberosKdc; + this.kerberosAuthorization = kerberosAuthorization; + this.kerberosAutorenewal = kerberosAutorenewal; + this.useHdfsHostname = UseHdfsHostname; + this.kerberosPrincipalPattern = kerberosPrincipalPattern; + this.hdfsTransferProtection = hdfsTransferProtection; + this.hdfsCipherSuites = hdfsCipherSuites; + this.kerberosKeytabUser = kerberosKeytabUser; + this.kerberosKeytabPath = kerberosKeytabPath; + } + + @Override + public FileSystem fileSystem(boolean initializeUGI) throws IOException { + FileSystem fs; + if ("kerberos".equals(kerberosAuthentication)) { + // Code for initializing the FileSystem with kerberos. + LOGGER.debug("Kerberos authentication is enabled, using secure authentication for FileSystem."); + + // set kerberos host and realm + System.setProperty("java.security.krb5.realm", kerberosRealm); + System.setProperty("java.security.krb5.kdc", kerberosKdc); + + Configuration conf = new Configuration(); + + // enable kerberus + conf.set("hadoop.security.authentication", kerberosAuthentication); + conf.set("hadoop.security.authorization", kerberosAuthorization); + conf.set("hadoop.kerberos.keytab.login.autorenewal.enabled", kerberosAutorenewal); + + conf.set("fs.defaultFS", hdfsUri); // Set FileSystem URI + conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? + conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? + + // hack for running locally with fake DNS records, set this to true if overriding the host name in /etc/hosts + conf.set("dfs.client.use.datanode.hostname", useHdfsHostname); + + // server principal, the kerberos principle that the namenode is using + conf.set("dfs.namenode.kerberos.principal.pattern", kerberosPrincipalPattern); + + conf.set("dfs.data.transfer.protection", hdfsTransferProtection); + conf.set("dfs.encrypt.data.transfer.cipher.suites", hdfsCipherSuites); + + if (initializeUGI) { + LOGGER.debug("Initializing UserGroupInformation for Kerberos"); + UserGroupInformation.setConfiguration(conf); + UserGroupInformation.loginUserFromKeytab(kerberosKeytabUser, kerberosKeytabPath); + } + + // filesystem for HDFS access is set here + fs = FileSystem.get(conf); + } + else { + // Code for initializing the FileSystem without kerberos. + LOGGER.warn("Kerberos authentication is not enabled, using insecure authentication for FileSystem."); + + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsUri); + // Because of Maven + conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", LocalFileSystem.class.getName()); + // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + // filesystem for HDFS access is set here + fs = FileSystem.get(URI.create(hdfsUri), conf); + } + return fs; + } +} diff --git a/src/main/java/com/teragrep/pth_06/HdfsFileMetadata.java b/src/main/java/com/teragrep/pth_06/HdfsFileMetadata.java new file mode 100644 index 00000000..b99982f3 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/HdfsFileMetadata.java @@ -0,0 +1,72 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06; + +import org.apache.kafka.common.TopicPartition; + +import java.io.Serializable; + +// Class for holding serializable metadata of HDFS files containing kafka records. +public class HdfsFileMetadata implements Serializable { + + public final TopicPartition topicPartition; // Represents the Kafka topic partition which records the file contains. + public final long endOffset; // Represents the offset of the record that was last added to the file. + public final String hdfsFilePath; // Represents the file path where the file resides in HDFS. + public final long hdfsFileSize; // Represents the size of the file in HDFS. Used for scheduling the batch slice. + + public HdfsFileMetadata(TopicPartition topicPartition, long offset, String filePath, long fileSize) { + this.topicPartition = topicPartition; + this.endOffset = offset; + this.hdfsFilePath = filePath; + this.hdfsFileSize = fileSize; + } + + @Override + public String toString() { + return "HdfsFileMetadata{" + "topicPartition=" + topicPartition + ", endOffset=" + endOffset + ", hdfsFilePath=" + + hdfsFilePath + ", hdfsFileSize=" + hdfsFileSize + '}'; + } +} diff --git a/src/main/java/com/teragrep/pth_06/TeragrepDatasource.java b/src/main/java/com/teragrep/pth_06/TeragrepDatasource.java index 9066b5d9..2bfab1d5 100644 --- a/src/main/java/com/teragrep/pth_06/TeragrepDatasource.java +++ b/src/main/java/com/teragrep/pth_06/TeragrepDatasource.java @@ -99,7 +99,7 @@ public final class TeragrepDatasource implements DataSourceRegister, TableProvid }); @Override - public Table getTable(StructType schema, Transform[] partitioning, Map properties) { + public Table getTable(StructType schemaArg, Transform[] partitioning, Map properties) { return this; } diff --git a/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java b/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java index 618eaeed..b81d1f17 100644 --- a/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java +++ b/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java @@ -47,6 +47,8 @@ import org.jooq.DSLContext; +import java.util.Objects; + public final class ConditionConfig { private final DSLContext ctx; @@ -107,14 +109,22 @@ public boolean streamQuery() { @Override public boolean equals(Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final ConditionConfig cast = (ConditionConfig) object; return this.bloomEnabled == cast.bloomEnabled && this.streamQuery == cast.streamQuery && this.withoutFilters == cast.withoutFilters && this.ctx == cast.ctx; } + + @Override + public int hashCode() { + return Objects.hash(ctx, streamQuery, bloomEnabled, withoutFilters); + } } diff --git a/src/main/java/com/teragrep/pth_06/config/Config.java b/src/main/java/com/teragrep/pth_06/config/Config.java index 5da4d739..d2ef1851 100644 --- a/src/main/java/com/teragrep/pth_06/config/Config.java +++ b/src/main/java/com/teragrep/pth_06/config/Config.java @@ -59,12 +59,14 @@ public final class Config { public final String query; public final ArchiveConfig archiveConfig; public final KafkaConfig kafkaConfig; + public final HdfsConfig hdfsConfig; public final BatchConfig batchConfig; public final AuditConfig auditConfig; public final boolean isArchiveEnabled; public final boolean isKafkaEnabled; + public final boolean isHdfsEnabled; public final boolean isMetadataQuery; @@ -93,8 +95,16 @@ public Config(Map opts) { kafkaConfig = new KafkaConfig(); } + isHdfsEnabled = opts.getOrDefault("hdfs.enabled", "false").equalsIgnoreCase("true"); + if (isHdfsEnabled) { + hdfsConfig = new HdfsConfig(opts); + } + else { + hdfsConfig = new HdfsConfig(); + } + // check that at least one datasource is enabled - if (!isArchiveEnabled && !isKafkaEnabled) { + if (!isArchiveEnabled && !isKafkaEnabled && !isHdfsEnabled) { throw new IllegalStateException("No datasources enabled"); } diff --git a/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java new file mode 100644 index 00000000..4b7bc402 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java @@ -0,0 +1,137 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.config; + +import java.time.Instant; +import java.util.Map; + +public final class HdfsConfig { + + // HDFS + public final long includeFileEpochAndAfter; // Represents the cutoff epoch in milliseconds which dictates which files should not be fetched from HDFS based on their timestamps. + public final long includeRecordEpochAndAfter; // Represents the cutoff epoch in microseconds which dictates which record should not be fetched from HDFS based on the record timestamp. + public final String hdfsPath; // Represents the working directory path in HDFS filesystem. + public final String hdfsUri; // Represents the address of the HDFS server. + public final String useHdfsHostname; + public final String hdfsTransferProtection; + public final String hdfsCipherSuites; + public final boolean useMockHdfsDatabase; // Represents the configuration parameter if mock database should be used or not. Used only for testing. + + // Kerberos + public final String kerberosAuthentication; + public final String kerberosAuthorization; + public final String kerberosPrincipalPattern; + public final String kerberosKdc; + public final String kerberosRealm; + public final String kerberosKeytabUser; + public final String kerberosKeytabPath; + public final String kerberosAutorenewal; + + public final boolean isStub; + + public HdfsConfig(Map opts) { + includeFileEpochAndAfter = Long + .parseLong( + opts + .getOrDefault( + "hdfs.includeFileEpochAndAfter", + String.valueOf(Instant.now().toEpochMilli() - 72 * 3600000) + ) + ); // Default is -72h from now + includeRecordEpochAndAfter = Long + .parseLong( + opts + .getOrDefault( + "hdfs.includeRecordEpochAndAfter", String.valueOf(Long.MIN_VALUE / (1000 * 1000)) + ) + ); + hdfsPath = getOrThrow(opts, "hdfs.hdfsPath"); + hdfsUri = getOrThrow(opts, "hdfs.hdfsUri"); + useHdfsHostname = getOrThrow(opts, "hdfs.useHostName"); + hdfsTransferProtection = getOrThrow(opts, "hdfs.transferProtection"); + hdfsCipherSuites = getOrThrow(opts, "hdfs.cipherSuites"); + String useMockHdfsDatabaseString = opts.getOrDefault("hdfs.useMockHdfsDatabase", "false"); + useMockHdfsDatabase = "true".equals(useMockHdfsDatabaseString); + kerberosAuthentication = getOrThrow(opts, "hdfs.krbAuthentication"); + kerberosAuthorization = getOrThrow(opts, "hdfs.krbAuthorization"); + kerberosPrincipalPattern = getOrThrow(opts, "hdfs.krbPrincipalPattern"); + kerberosKdc = getOrThrow(opts, "hdfs.krbKdc"); + kerberosRealm = getOrThrow(opts, "hdfs.krbRealm"); + kerberosKeytabUser = getOrThrow(opts, "hdfs.krbKeytabUser"); + kerberosKeytabPath = getOrThrow(opts, "hdfs.krbKeytabPath"); + kerberosAutorenewal = opts.getOrDefault("hdfs.useKerberosAutorenewal", "false"); + isStub = false; + } + + public HdfsConfig() { + includeFileEpochAndAfter = 0L; + includeRecordEpochAndAfter = 0L; + hdfsPath = ""; + hdfsUri = ""; + useHdfsHostname = ""; + hdfsTransferProtection = ""; + hdfsCipherSuites = ""; + useMockHdfsDatabase = false; + kerberosAuthentication = ""; + kerberosAuthorization = ""; + kerberosPrincipalPattern = ""; + kerberosKdc = ""; + kerberosRealm = ""; + kerberosKeytabUser = ""; + kerberosKeytabPath = ""; + kerberosAutorenewal = "false"; + isStub = true; + } + + private String getOrThrow(Map opts, String key) { + String value = opts.get(key); + if (value == null) { + throw new IllegalArgumentException("Configuration item '" + key + "' was null!"); + } + return value; + } + +} diff --git a/src/main/java/com/teragrep/pth_06/planner/ArchiveQuery.java b/src/main/java/com/teragrep/pth_06/planner/ArchiveQuery.java index ea8240d1..45c0ceb1 100644 --- a/src/main/java/com/teragrep/pth_06/planner/ArchiveQuery.java +++ b/src/main/java/com/teragrep/pth_06/planner/ArchiveQuery.java @@ -59,14 +59,14 @@ */ public interface ArchiveQuery { - Result> processBetweenUnixEpochHours( + public abstract Result> processBetweenUnixEpochHours( long startHour, long endHour ); - void commit(long offset); + public abstract void commit(long offset); - Long getInitialOffset(); + public abstract Long getInitialOffset(); - Long incrementAndGetLatestOffset(); + public abstract Long incrementAndGetLatestOffset(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/ArchiveQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/ArchiveQueryProcessor.java index 86ded566..0950851a 100644 --- a/src/main/java/com/teragrep/pth_06/planner/ArchiveQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/ArchiveQueryProcessor.java @@ -51,7 +51,10 @@ import org.jooq.types.ULong; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.xml.sax.SAXException; +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; import java.sql.Date; import java.sql.SQLException; import java.time.Instant; @@ -102,7 +105,7 @@ public ArchiveQueryProcessor(Config config) { rollingDay = Instant.ofEpochSecond(this.earliestEpoch).atZone(ZoneId.systemDefault()).toLocalDate(); } - catch (Exception ex) { + catch (ParserConfigurationException | SAXException | IOException ex) { throw new RuntimeException( "ArchiveQueryProcessor problems when construction Query conditions query:" + config.query + " exception:" + ex diff --git a/src/main/java/com/teragrep/pth_06/planner/BatchSizeLimit.java b/src/main/java/com/teragrep/pth_06/planner/BatchSizeLimit.java index 95df9682..f262225b 100644 --- a/src/main/java/com/teragrep/pth_06/planner/BatchSizeLimit.java +++ b/src/main/java/com/teragrep/pth_06/planner/BatchSizeLimit.java @@ -52,16 +52,16 @@ final class BatchSizeLimit { /** Maximum weight of the batch (length of executor queue * amount of executors) */ - final long maxWeight; + private final long maxWeight; /** Maximum count of total objects, in case the weights of the objects are small */ - final long maxObjectCount; + private final long maxObjectCount; /** accumulated sum of weights */ - float accumulatedWeight; + private float accumulatedWeight; /** accumulated sum of object count */ - long accumulatedObjectCount; + private long accumulatedObjectCount; /** * Initialize the BatchSizeLimit with the given maximum size and maximum object count. diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java new file mode 100644 index 00000000..136f0877 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java @@ -0,0 +1,117 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.planner; + +import com.teragrep.pth_06.FileSystemFactoryImpl; +import com.teragrep.pth_06.HdfsFileMetadata; +import com.teragrep.pth_06.config.Config; +import org.apache.hadoop.fs.*; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.LinkedList; + +// Searches the HDFS database for files contained in directories based on topicsRegexString. Creates a list of objects where each object contains the metadata of a single HDFS-file. +public class HdfsDBClient { + + private final Logger LOGGER = LoggerFactory.getLogger(HdfsDBClient.class); + private final String path; + private final FileSystem fs; + private final TopicFilter topicFilter; + private final long ignoreBeforeEpoch; + + public HdfsDBClient(Config config, String topicsRegexString) throws IOException { + this(config, new TopicFilter(topicsRegexString), new FileSystemFactoryImpl(config.hdfsConfig).fileSystem(true)); + } + + public HdfsDBClient(Config config, TopicFilter topicFilter, FileSystem fs) { + this.topicFilter = topicFilter; + this.ignoreBeforeEpoch = config.hdfsConfig.includeFileEpochAndAfter; // Defines the minimum time window / cutoff epoch for making sure that the files which metadata is fetched by the planner are not mistakenly deleted by cfe_39 pruning before tasker can process the records inside. + path = config.hdfsConfig.hdfsPath; + this.fs = fs; + } + + // this queries and pulls the distinct file metadata values to the partitionList according to the given query conditions (condition only applies to topic names in planner side). + public LinkedList pullToPartitionList() throws IOException { + LinkedList rv = new LinkedList<>(); + // path holds the fileSystem path to the directory that holds a collection of other directories, each different directory representing a different topic. + FileStatus[] directoryStatuses = fs.listStatus(new Path(path), topicFilter); + /*If the path only holds one directory, fileStatuses will only hold one FileStatus object which returns this value when fileStatus.getPath() is called: + hdfs://localhost:34509/opt/teragrep/cfe_39/srv/testConsumerTopic*/ + + // Get the directory statuses. Each directory represents a Kafka topic. + if (directoryStatuses.length > 0) { + LOGGER.debug("Found {} matching directories", directoryStatuses.length); + for (FileStatus r : directoryStatuses) { + // Get the file statuses that are inside the directories. + FileStatus[] fileStatuses = fs.listStatus(r.getPath()); + for (FileStatus r2 : fileStatuses) { + String topic = r2.getPath().getParent().getName(); + String[] split = r2.getPath().getName().split("\\."); // The file name can be split to partition parameter and offset parameter. First value is partition and second is offset. + String partition = split[0]; + String offset = split[1]; + HdfsFileMetadata temp = new HdfsFileMetadata( + new TopicPartition(topic, Integer.parseInt(partition)), + Integer.parseInt(offset), + r2.getPath().toString(), + r2.getLen() + ); + // Add the HdfsFileMetadata object to the rv only if the file's modification timestamp is above ignoreBeforeEpoch. Timestamps are in milliseconds. + if (r2.getModificationTime() >= ignoreBeforeEpoch) { + rv.add(temp); + } + } + } + } + else { + LOGGER.info("No matching directories found"); + } + return rv; + } + +} diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java new file mode 100644 index 00000000..32700760 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java @@ -0,0 +1,72 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.planner; + +import com.google.gson.JsonArray; +import com.teragrep.pth_06.HdfsFileMetadata; +import com.teragrep.pth_06.planner.offset.HdfsOffset; + +import java.util.LinkedList; + +public interface HdfsQuery { + + public abstract LinkedList processBetweenHdfsFileMetadata( + HdfsOffset startOffset, + HdfsOffset endOffset + ); + + public abstract void commit(HdfsOffset offset); + + public abstract JsonArray hdfsOffsetMapToJSON(); + + public abstract HdfsOffset getBeginningOffsets(); + + public abstract HdfsOffset getInitialEndOffsets(); // Delete after testing incrementAndGetLatestOffset() functionality thoroughly. + + public abstract HdfsOffset incrementAndGetLatestOffset(); // replaces getInitialEndOffsets() + + public abstract boolean isStub(); +} diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java new file mode 100644 index 00000000..d194fbb0 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java @@ -0,0 +1,276 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.planner; + +import com.google.gson.JsonArray; +import com.google.gson.JsonObject; +import com.teragrep.pth_06.HdfsFileMetadata; +import com.teragrep.pth_06.config.Config; +import com.teragrep.pth_06.planner.offset.HdfsOffset; +import com.teragrep.pth_06.planner.walker.HdfsConditionWalker; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.xml.sax.SAXException; + +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; + +// Class for processing hdfs queries. +public class HdfsQueryProcessor implements HdfsQuery { + + private final Logger LOGGER = LoggerFactory.getLogger(HdfsQueryProcessor.class); + private LinkedList topicPartitionList; + private final HdfsDBClient hdfsDBClient; + private final Map hdfsOffsetMap; + private final Map latestHdfsOffsetMap; + private final long quantumLength; + private final long numPartitions; + private final long totalObjectCountLimit; + private final boolean stub; + + public HdfsQueryProcessor() { + totalObjectCountLimit = 0; + hdfsDBClient = null; // refactor null to stub + hdfsOffsetMap = new HashMap<>(); + latestHdfsOffsetMap = new HashMap<>(); + quantumLength = 0; + numPartitions = 0; + stub = true; + } + + public HdfsQueryProcessor(Config config) { + // get configs from config object + this.quantumLength = config.batchConfig.quantumLength; + this.numPartitions = config.batchConfig.numPartitions; + this.totalObjectCountLimit = config.batchConfig.totalObjectCountLimit; + // Filter only topics using regex pattern + String topicsRegexString = "^.*$"; // all topics if none given + if (config.query != null) { + try { + HdfsConditionWalker parser = new HdfsConditionWalker(); + topicsRegexString = parser.fromString(config.query); + } + catch (ParserConfigurationException | SAXException | IOException e) { + throw new RuntimeException( + "HdfsQueryProcessor problems when construction Query conditions query:" + config.query + + " exception:" + e + ); + } + } + // Implement hdfs db client that fetches the metadata for the files that are stored in hdfs based on topic name (aka. directory containing the files for a specific topic in HDFS). + // Remember to implement Kerberized HDFS access for prod. Tests are done using normal access on mini cluster. + try { + this.hdfsDBClient = new HdfsDBClient( + config, + topicsRegexString // topicsRegexString only searches for the given topic/topics (aka. directories). + ); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + + // Generate the new starting partition offsets for kafka. + hdfsOffsetMap = new HashMap<>(); // This parameter is used for generating the new start offsets for the KafkaOffsetPlanner. hdfsOffsetMapToJSON() is used to transfer the parameter using printer. + seekToResults(); // This method loads all the available metadata to TopicPartitionList from HDFS. + // Create a map that only contains the metadata with the highest offset for every topic partition. + for (HdfsFileMetadata r : topicPartitionList) { + long partitionStart = r.endOffset; + if (!hdfsOffsetMap.containsKey(r.topicPartition)) { + hdfsOffsetMap.put(r.topicPartition, partitionStart + 1); + } + else { + if (hdfsOffsetMap.get(r.topicPartition) < partitionStart) { + hdfsOffsetMap.replace(r.topicPartition, partitionStart + 1); + } + } + } + latestHdfsOffsetMap = new HashMap<>(); + stub = false; + LOGGER.debug("HdfsQueryProcessor.HdfsQueryProcessor>"); + } + + // pulls the metadata for available topic partition files in HDFS. + private void seekToResults() { + LOGGER.debug("HdfsQueryProcessor.seekToResults>"); + try { + topicPartitionList = hdfsDBClient.pullToPartitionList(); // queries the list of topic partitions based on the topic name condition filtering. + LOGGER.debug("HdfsQueryProcessor.seekToResults>"); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + // Returns all the available HDFS file metadata between the given topic partition offsets. + @Override + public LinkedList processBetweenHdfsFileMetadata(HdfsOffset startOffset, HdfsOffset endOffset) { + LinkedList rv = new LinkedList<>(); + Map endOffsetMap = endOffset.getOffsetMap(); + Map startOffsetMap = startOffset.getOffsetMap(); + for (HdfsFileMetadata r : topicPartitionList) { + if ( + (endOffsetMap.get(r.topicPartition) >= r.endOffset) + & (startOffsetMap.get(r.topicPartition) <= r.endOffset) + ) { + rv.add(new HdfsFileMetadata(r.topicPartition, r.endOffset, r.hdfsFilePath, r.hdfsFileSize)); + } + } + return rv; + } + + // Removes the committed topic partition offsets from topicPartitionList. + @Override + public void commit(HdfsOffset offset) { + Map offsetMap = offset.getOffsetMap(); + LinkedList newTopicPartitionList = new LinkedList<>(); + // Generate new topicPartitionList where the metadata with offset values lower than the offset values given as input parameter are filtered out. + for (HdfsFileMetadata r : topicPartitionList) { + if (offsetMap.get(r.topicPartition) < r.endOffset) { + newTopicPartitionList + .add(new HdfsFileMetadata(r.topicPartition, r.endOffset, r.hdfsFilePath, r.hdfsFileSize)); + } + } + topicPartitionList = newTopicPartitionList; + } + + // Prints json array containing the starting offsets for kafka to use. + @Override + public JsonArray hdfsOffsetMapToJSON() { + JsonArray ja = new JsonArray(); + for (Map.Entry entry : hdfsOffsetMap.entrySet()) { + String topic = entry.getKey().topic(); + String partition = String.valueOf(entry.getKey().partition()); + String offset = String.valueOf(entry.getValue()); + JsonObject jo = new JsonObject(); + jo.addProperty("topic", topic); + jo.addProperty("partition", partition); + jo.addProperty("offset", offset); + ja.add(jo); + } + return ja; + } + + // returns the starting offsets for all available (aka. filtered) topic partitions. + @Override + public HdfsOffset getBeginningOffsets() { + Map startOffset = new HashMap<>(); + // Go through the topicPartitionList to generate start offsets. + for (HdfsFileMetadata r : topicPartitionList) { + // Generate startOffset + // When going through the result, store the topic partition with the lowest offset to the startOffset object. + long partitionOffset = r.endOffset; + if (!startOffset.containsKey(r.topicPartition)) { + startOffset.put(r.topicPartition, partitionOffset); + } + else { + if (startOffset.get(r.topicPartition) > partitionOffset) { + startOffset.replace(r.topicPartition, partitionOffset); + } + } + } + HdfsOffset hdfsOffset = new HdfsOffset(startOffset); + hdfsOffset.serialize(); + return hdfsOffset; + } + + // returns the end offsets for all available (aka. filtered) topic partitions. + @Override + public HdfsOffset getInitialEndOffsets() { + Map endOffset = new HashMap<>(); + // Go through the topicPartitionList to generate end offsets. + for (HdfsFileMetadata r : topicPartitionList) { + long partitionOffset = r.endOffset; + // Generate endOffset + // When going through the result, store the topic partition with the highest offset to the endOffset object. + if (!endOffset.containsKey(r.topicPartition)) { + endOffset.put(r.topicPartition, partitionOffset); + } + else { + if (endOffset.get(r.topicPartition) < partitionOffset) { + endOffset.replace(r.topicPartition, partitionOffset); + } + } + } + HdfsOffset hdfsOffset = new HdfsOffset(endOffset); + hdfsOffset.serialize(); + return hdfsOffset; + } + + // Increments the latest offset values and returns that incremented offsets. Works by pulling data from the topicPartitionList until weight limit is reached. + @Override + public HdfsOffset incrementAndGetLatestOffset() { + if (this.latestHdfsOffsetMap.isEmpty()) { + HdfsOffset beginningOffsets = getBeginningOffsets(); + this.latestHdfsOffsetMap.putAll(beginningOffsets.getOffsetMap()); + } + // Initialize the batchSizeLimit object to split the data into appropriate sized batches + BatchSizeLimit batchSizeLimit = new BatchSizeLimit(quantumLength * numPartitions, totalObjectCountLimit); + // Keep loading more offsets from topicPartitionList until the limit is reached + Iterator iterator = topicPartitionList.iterator(); + while (!batchSizeLimit.isOverLimit() && iterator.hasNext()) { + HdfsFileMetadata r = iterator.next(); + // When going through the result, store the topic partition with the highest offset to the latestHdfsOffsetMap. + if (latestHdfsOffsetMap.get(r.topicPartition) < r.endOffset) { + latestHdfsOffsetMap.replace(r.topicPartition, r.endOffset); + batchSizeLimit.add(r.hdfsFileSize); + } + } + HdfsOffset hdfsOffset = new HdfsOffset(latestHdfsOffsetMap); + hdfsOffset.serialize(); + return hdfsOffset; + } + + @Override + public boolean isStub() { + return stub; + } +} diff --git a/src/main/java/com/teragrep/pth_06/planner/KafkaQuery.java b/src/main/java/com/teragrep/pth_06/planner/KafkaQuery.java index fafca70b..2be65cdb 100644 --- a/src/main/java/com/teragrep/pth_06/planner/KafkaQuery.java +++ b/src/main/java/com/teragrep/pth_06/planner/KafkaQuery.java @@ -45,6 +45,7 @@ */ package com.teragrep.pth_06.planner; +import com.google.gson.JsonArray; import com.teragrep.pth_06.planner.offset.KafkaOffset; import org.apache.kafka.common.TopicPartition; @@ -58,11 +59,15 @@ */ public interface KafkaQuery { - Map getInitialEndOffsets(); + public abstract Map getInitialEndOffsets(); - Map getEndOffsets(KafkaOffset startOffset); + public abstract Map getEndOffsets(KafkaOffset startOffset); - Map getBeginningOffsets(KafkaOffset endOffset); + public abstract Map getBeginningOffsets(KafkaOffset endOffset); - void commit(KafkaOffset offset); + public abstract void commit(KafkaOffset offset); + + public abstract void seekToHdfsOffsets(JsonArray hdfsStartOffsets); + + public abstract Map getConsumerPositions(JsonArray startOffsets); } diff --git a/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java index 7c123c8c..8cc825e2 100644 --- a/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java @@ -46,6 +46,9 @@ package com.teragrep.pth_06.planner; import com.google.common.annotations.VisibleForTesting; +import com.google.gson.JsonArray; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; import com.teragrep.pth_06.config.Config; import com.teragrep.pth_06.planner.offset.KafkaOffset; import com.teragrep.pth_06.planner.walker.KafkaWalker; @@ -56,7 +59,10 @@ import org.apache.kafka.common.errors.TopicAuthorizationException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.xml.sax.SAXException; +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; import java.time.Duration; import java.util.*; import java.util.regex.Matcher; @@ -91,7 +97,7 @@ public KafkaQueryProcessor(Config config) { KafkaWalker parser = new KafkaWalker(); topicsRegexString = parser.fromString(config.query); } - catch (Exception ex) { + catch (ParserConfigurationException | SAXException | IOException ex) { ex.printStackTrace(); throw new RuntimeException( "KafkaQueryProcessor problems when construction Query conditions query:" + config.query @@ -261,4 +267,40 @@ public Map getBeginningOffsets(KafkaOffset endOffset) { public void commit(KafkaOffset offset) { // no-op } + + @Override + public void seekToHdfsOffsets(JsonArray startOffsets) { + // startOffsets holds the new starting offsets for Kafka in JsonArray format: + // [{"topic":"testConsumerTopic", "partition":"7", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"8", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"5", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"6", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"3", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"4", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"1", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"2", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"0", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"9", "offset":"13"}] + // Deserialize JsonArray back to Map: + Map hdfsStartOffsets = new HashMap<>(); + for (JsonElement pa : startOffsets) { + JsonObject offsetObject = pa.getAsJsonObject(); + TopicPartition topicPartition = new TopicPartition( + offsetObject.get("topic").getAsString(), + offsetObject.get("partition").getAsInt() + ); + Long offset = offsetObject.get("offset").getAsLong(); + hdfsStartOffsets.put(topicPartition, offset); + } + + // Seek the consumer to the new starting offsets defined by startOffsets for all the topic partitions. + for (TopicPartition topicPartition : topicPartitionSet) { + if (hdfsStartOffsets.containsKey(topicPartition)) { + kafkaConsumer.seek(topicPartition, hdfsStartOffsets.get(topicPartition)); + } + } + + } + + @Override + public Map getConsumerPositions(JsonArray startOffsets) { + // Get the consumer position for all available topic partitions. + Map topicPartitionStartOffsetMap = new HashMap<>(); + for (TopicPartition topicPartition : topicPartitionSet) { + long position = kafkaConsumer.position(topicPartition, Duration.ofSeconds(60)); + topicPartitionStartOffsetMap.put(topicPartition, position); + } + return topicPartitionStartOffsetMap; + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/MockKafkaConsumerFactory.java b/src/main/java/com/teragrep/pth_06/planner/MockKafkaConsumerFactory.java index 106308dd..9e08abac 100644 --- a/src/main/java/com/teragrep/pth_06/planner/MockKafkaConsumerFactory.java +++ b/src/main/java/com/teragrep/pth_06/planner/MockKafkaConsumerFactory.java @@ -57,6 +57,7 @@ import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.HashMap; +import java.util.Map; /** *

Mock Kafka Consumer Factory

Mocked Kafka Consumer factory used for testing. @@ -65,9 +66,9 @@ * @author Mikko Kortelainen */ @VisibleForTesting -public class MockKafkaConsumerFactory { +public final class MockKafkaConsumerFactory { - final static private Logger LOGGER = LoggerFactory.getLogger(MockKafkaConsumerFactory.class); + private static final Logger LOGGER = LoggerFactory.getLogger(MockKafkaConsumerFactory.class); private MockKafkaConsumerFactory() { @@ -251,13 +252,13 @@ public static Consumer getConsumer() { consumer = new MockConsumer(OffsetResetStrategy.EARLIEST); consumer.assign(Collections.singletonList(new TopicPartition("testConsumerTopic", 0))); - HashMap beginningOffsets = new HashMap<>(); + Map beginningOffsets = new HashMap<>(); beginningOffsets.put(new TopicPartition("testConsumerTopic", 0), 0L); // start is inclusive consumer.updateBeginningOffsets(beginningOffsets); generateEvents(consumer); - HashMap endOffsets = new HashMap<>(); + Map endOffsets = new HashMap<>(); endOffsets.put(new TopicPartition("testConsumerTopic", 0), 14L); // end is exclusive consumer.updateEndOffsets(endOffsets); LOGGER.debug(endOffsets.toString()); diff --git a/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java b/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java index ccae7ea6..d6c46adb 100644 --- a/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java @@ -45,6 +45,7 @@ */ package com.teragrep.pth_06.planner; +import java.io.IOException; import java.sql.*; import java.time.Instant; import java.util.Set; @@ -61,6 +62,9 @@ import org.jooq.types.UShort; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.xml.sax.SAXException; + +import javax.xml.parsers.ParserConfigurationException; import static com.teragrep.pth_06.jooq.generated.streamdb.Streamdb.STREAMDB; import static com.teragrep.pth_06.jooq.generated.journaldb.Journaldb.JOURNALDB; @@ -133,7 +137,7 @@ public StreamDBClient(Config config) throws SQLException { streamdbCondition = walker.fromString(config.query, true); this.journaldbCondition = walker.fromString(config.query, false); } - catch (Exception e) { + catch (ParserConfigurationException | SAXException | IOException e) { throw new IllegalArgumentException(e); } @@ -336,7 +340,7 @@ public class NestedTopNQuery { logtimeFunction.as(logtime) }; - private Table getTableStatement(Condition journaldbCondition, Date day) { + private Table getTableStatement(Condition journaldbConditionArg, Date day) { SelectOnConditionStep selectOnConditionStep = select(resultFields) .from(GetArchivedObjectsFilterTable.FILTER_TABLE) @@ -359,7 +363,7 @@ private Table getTableStatement(Condition journaldbCondition, Date day) } return selectOnConditionStep - .where(JOURNALDB.LOGFILE.LOGDATE.eq(day).and(journaldbCondition)) + .where(JOURNALDB.LOGFILE.LOGDATE.eq(day).and(journaldbConditionArg)) .orderBy(logtimeForOrderBy, JOURNALDB.LOGFILE.ID.asc()) .asTable(innerTable); } diff --git a/src/main/java/com/teragrep/pth_06/planner/TopicFilter.java b/src/main/java/com/teragrep/pth_06/planner/TopicFilter.java new file mode 100644 index 00000000..7d727dcb --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/TopicFilter.java @@ -0,0 +1,63 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.planner; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; + +public class TopicFilter implements PathFilter { + + private final String topicsRegexString; + + public TopicFilter(String topicsRegexString) { + this.topicsRegexString = topicsRegexString; + } + + @Override + public boolean accept(Path path) { + return path.getName().matches(topicsRegexString); // Catches the directory names. + } +} diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTable.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTable.java index 03f7499a..ca622d46 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTable.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTable.java @@ -47,6 +47,6 @@ public interface CategoryTable { - void create(); + public abstract void create(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTableImpl.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTableImpl.java index d13c2678..63f908aa 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTableImpl.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTableImpl.java @@ -138,12 +138,15 @@ public void create() { */ @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final CategoryTableImpl cast = (CategoryTableImpl) object; return this.originTable.equals(cast.originTable) && this.ctx == cast.ctx && // equal only if same instance of DSLContext this.bloomTermId == cast.bloomTermId && this.tableFilters.equals(cast.tableFilters); diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/ConditionMatchBloomDBTables.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/ConditionMatchBloomDBTables.java index 3f32b31c..16ddb7e8 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/ConditionMatchBloomDBTables.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/ConditionMatchBloomDBTables.java @@ -111,12 +111,15 @@ public List> tables() { */ @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final ConditionMatchBloomDBTables cast = (ConditionMatchBloomDBTables) object; return this.condition.equals(cast.condition) && this.ctx == cast.ctx; } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/DatabaseTables.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/DatabaseTables.java index 2a4869f6..f0511676 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/DatabaseTables.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/DatabaseTables.java @@ -51,5 +51,5 @@ public interface DatabaseTables { - List> tables(); + public abstract List> tables(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/RegexExtractedValue.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/RegexExtractedValue.java index a723a0dc..a3fe4b0e 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/RegexExtractedValue.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/RegexExtractedValue.java @@ -75,10 +75,12 @@ public List tokens() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || getClass() != object.getClass()) + } + if (object == null || getClass() != object.getClass()) { return false; + } final RegexExtractedValue cast = (RegexExtractedValue) object; return value.equals(cast.value) && pattern.equals(cast.pattern); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/SearchTermBloomFilter.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/SearchTermBloomFilter.java index a30dc70f..c67bbd0c 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/SearchTermBloomFilter.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/SearchTermBloomFilter.java @@ -99,10 +99,12 @@ public byte[] bytes() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || getClass() != object.getClass()) + } + if (object == null || getClass() != object.getClass()) { return false; + } final SearchTermBloomFilter cast = (SearchTermBloomFilter) object; return expected.equals(cast.expected) && fpp.equals(cast.fpp) && stringTokens.equals(cast.stringTokens); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilterTypesFromMetadata.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilterTypesFromMetadata.java index a5efa749..4a3111aa 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilterTypesFromMetadata.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilterTypesFromMetadata.java @@ -120,10 +120,12 @@ public Result toResult() { */ @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || getClass() != object.getClass()) + } + if (object == null || getClass() != object.getClass()) { return false; + } final TableFilterTypesFromMetadata cast = (TableFilterTypesFromMetadata) object; return ctx == cast.ctx && Objects.equals(table, cast.table) && Objects .equals(expectedField, cast.expectedField) && Objects.equals(fppField, cast.fppField) && Objects diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilters.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilters.java index 54184424..2f4c52d3 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilters.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilters.java @@ -139,10 +139,12 @@ public Batch asBatch() { */ @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || getClass() != object.getClass()) + } + if (object == null || getClass() != object.getClass()) { return false; + } final TableFilters cast = (TableFilters) object; return bloomTermId == cast.bloomTermId && recordsInMetadata.equals(cast.recordsInMetadata) && ctx == cast.ctx && table.equals(cast.table) && thisTable.equals(cast.thisTable) && searchTerm.equals(cast.searchTerm); diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableRecords.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableRecords.java index 932070f8..91b870d8 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableRecords.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableRecords.java @@ -50,5 +50,5 @@ public interface TableRecords { - Result toResult(); + public abstract Result toResult(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/Tokenizable.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/Tokenizable.java index e209f536..c40ce443 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/Tokenizable.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/Tokenizable.java @@ -49,5 +49,5 @@ public interface Tokenizable { - List tokens(); + public abstract List tokens(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokenizedValue.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokenizedValue.java index fe32e624..6a07b959 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokenizedValue.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokenizedValue.java @@ -67,10 +67,12 @@ public List tokens() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || object.getClass() != this.getClass()) + } + if (object == null || object.getClass() != this.getClass()) { return false; + } final TokenizedValue cast = (TokenizedValue) object; return value.equals(cast.value); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokensAsStrings.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokensAsStrings.java index b0c9dce8..fa2d9985 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokensAsStrings.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokensAsStrings.java @@ -66,10 +66,12 @@ public List tokens() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || object.getClass() != this.getClass()) + } + if (object == null || object.getClass() != this.getClass()) { return false; + } final TokensAsStrings cast = (TokensAsStrings) object; return origin.equals(cast.origin); } diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java index 7e686bb9..52407dcd 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java @@ -61,18 +61,34 @@ */ public class DatasourceOffset extends Offset implements Serializable { - SerializedDatasourceOffset serializedDatasourceOffset; + private final SerializedDatasourceOffset serializedDatasourceOffset; + + public DatasourceOffset(HdfsOffset hdfsOffset, KafkaOffset kafkaOffset) { + this(hdfsOffset, null, kafkaOffset); + } + + public DatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset) { + this(hdfsOffset, archiveOffset, new KafkaOffset()); + } public DatasourceOffset(LongOffset archiveOffset, KafkaOffset kafkaOffset) { - this.serializedDatasourceOffset = new SerializedDatasourceOffset(archiveOffset, kafkaOffset); + this(new HdfsOffset(), archiveOffset, kafkaOffset); + } + + public DatasourceOffset(HdfsOffset hdfsOffset) { + this(hdfsOffset, null, new KafkaOffset()); } public DatasourceOffset(LongOffset archiveOffset) { - this.serializedDatasourceOffset = new SerializedDatasourceOffset(archiveOffset); + this(new HdfsOffset(), archiveOffset, new KafkaOffset()); } public DatasourceOffset(KafkaOffset kafkaOffset) { - this.serializedDatasourceOffset = new SerializedDatasourceOffset(kafkaOffset); + this(new HdfsOffset(), null, kafkaOffset); + } + + public DatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset, KafkaOffset kafkaOffset) { + this.serializedDatasourceOffset = new SerializedDatasourceOffset(hdfsOffset, archiveOffset, kafkaOffset); } public DatasourceOffset(String s) { @@ -80,6 +96,10 @@ public DatasourceOffset(String s) { this.serializedDatasourceOffset = gson.fromJson(s, SerializedDatasourceOffset.class); } + public HdfsOffset getHdfsOffset() { + return serializedDatasourceOffset.hdfsOffset; + } + public LongOffset getArchiveOffset() { return serializedDatasourceOffset.archiveOffset; } @@ -87,8 +107,8 @@ public LongOffset getArchiveOffset() { public KafkaOffset getKafkaOffset() { KafkaOffset kafkaOffset = serializedDatasourceOffset.kafkaOffset; - if (kafkaOffset == null || kafkaOffset.getOffsetMap() == null) { - return null; + if (kafkaOffset.isStub() || kafkaOffset.getOffsetMap().isEmpty()) { + throw new RuntimeException("kafkaOffset must not be empty"); } for (Map.Entry entry : kafkaOffset.getOffsetMap().entrySet()) { diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java new file mode 100644 index 00000000..52d1dd55 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java @@ -0,0 +1,126 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.planner.offset; + +import com.google.common.reflect.TypeToken; +import com.google.gson.Gson; +import org.apache.kafka.common.TopicPartition; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +// Class for representing a serializable offset of HDFS data source. +// S3 has epoch hours as offsets, kafka has native TopicPartition offsets and HDFS should have file-metadata (use same format as in Kafka, topicpartition + record offset, which can be extracted from the metadata). + +public class HdfsOffset extends org.apache.spark.sql.connector.read.streaming.Offset implements Serializable, Offset { + + private final Map serializedHdfsOffset; + private final boolean stub; + private final transient Map hdfsOffset; + + public HdfsOffset() { + this(new HashMap<>(), new HashMap<>(), true); + } + + public HdfsOffset(String s) { + this(new HashMap<>(), new Gson().fromJson(s, new TypeToken>() { + }.getType()), false); + } + + public HdfsOffset(Map serializedHdfsOffset) { + this(serializedHdfsOffset, new HashMap<>(), false); + } + + public HdfsOffset(Map hdfsOffset, Map serializedHdfsOffset, boolean stub) { + this.hdfsOffset = hdfsOffset; + this.serializedHdfsOffset = serializedHdfsOffset; + this.stub = stub; + } + + @Override + public Map getOffsetMap() { + Map rv = new HashMap<>(serializedHdfsOffset.size()); + + for (Map.Entry entry : serializedHdfsOffset.entrySet()) { + String topicAndPartition = entry.getKey(); + long offset = entry.getValue(); + + int splitterLocation = topicAndPartition.lastIndexOf('-'); + int partition = Integer.parseInt(topicAndPartition.substring(splitterLocation + 1)); + String topic = topicAndPartition.substring(0, splitterLocation); + rv.put(new TopicPartition(topic, partition), offset); + } + + return rv; + } + + @Override + public boolean isStub() { + return stub; + } + + @Override + public void serialize() { + if (!hdfsOffset.isEmpty()) { + for (Map.Entry entry : hdfsOffset.entrySet()) { + serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); + } + hdfsOffset.clear(); + } + } + + @Override + public String json() { + Gson gson = new Gson(); + return gson.toJson(serializedHdfsOffset); + } + + @Override + public String toString() { + return "HdfsOffset{" + "serializedHdfsOffset=" + serializedHdfsOffset + '}'; + } +} diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java index bb87d60f..390c1f91 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java @@ -48,10 +48,8 @@ import com.google.common.reflect.TypeToken; import com.google.gson.Gson; import org.apache.kafka.common.TopicPartition; -import org.apache.spark.sql.connector.read.streaming.Offset; import java.io.Serializable; -import java.lang.reflect.Type; import java.util.HashMap; import java.util.Map; @@ -61,26 +59,32 @@ * @since 08/06/2022 * @author Mikko Kortelainen */ -public class KafkaOffset extends Offset implements Serializable { - - private static final Type mapType = new TypeToken>() { - }.getType(); +public class KafkaOffset extends org.apache.spark.sql.connector.read.streaming.Offset implements Serializable, Offset { private final Map serializedKafkaOffset; + private final boolean stub; + private final transient Map kafkaOffset; - public KafkaOffset(Map offset) { - serializedKafkaOffset = new HashMap<>(offset.size()); - for (Map.Entry entry : offset.entrySet()) { - - serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } + public KafkaOffset() { + this(new HashMap<>(), new HashMap<>(), true); } public KafkaOffset(String s) { - Gson gson = new Gson(); - serializedKafkaOffset = gson.fromJson(s, mapType); + this(new HashMap<>(), new Gson().fromJson(s, new TypeToken>() { + }.getType()), false); + } + + public KafkaOffset(Map kafkaOffset) { + this(kafkaOffset, new HashMap<>(), false); + } + + public KafkaOffset(Map kafkaOffset, Map serializedKafkaOffset, boolean stub) { + this.kafkaOffset = kafkaOffset; + this.serializedKafkaOffset = serializedKafkaOffset; + this.stub = stub; } + @Override public Map getOffsetMap() { Map rv = new HashMap<>(serializedKafkaOffset.size()); @@ -97,6 +101,21 @@ public Map getOffsetMap() { return rv; } + @Override + public boolean isStub() { + return stub; + } + + @Override + public void serialize() { + if (!kafkaOffset.isEmpty()) { + for (Map.Entry entry : kafkaOffset.entrySet()) { + serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); + } + kafkaOffset.clear(); + } + } + @Override public String json() { Gson gson = new Gson(); diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/Offset.java b/src/main/java/com/teragrep/pth_06/planner/offset/Offset.java new file mode 100644 index 00000000..cb9683c5 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/offset/Offset.java @@ -0,0 +1,60 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.planner.offset; + +import org.apache.kafka.common.TopicPartition; + +import java.util.Map; + +public interface Offset { + + public abstract Map getOffsetMap(); + + public abstract boolean isStub(); + + public abstract void serialize(); + +} diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java index 8dd1e37f..2d681dbd 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java @@ -61,27 +61,19 @@ public class SerializedDatasourceOffset implements Serializable { private final Long version = 1L; + public final HdfsOffset hdfsOffset; public final LongOffset archiveOffset; public final KafkaOffset kafkaOffset; - public SerializedDatasourceOffset(LongOffset archiveOffset, KafkaOffset kafkaOffset) { + public SerializedDatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset, KafkaOffset kafkaOffset) { + this.hdfsOffset = hdfsOffset; this.archiveOffset = archiveOffset; this.kafkaOffset = kafkaOffset; } - public SerializedDatasourceOffset(LongOffset archiveOffset) { - this.kafkaOffset = null; - this.archiveOffset = archiveOffset; - } - - public SerializedDatasourceOffset(KafkaOffset kafkaOffset) { - this.kafkaOffset = kafkaOffset; - this.archiveOffset = null; - } - @Override public String toString() { - return "SerializedDatasourceOffset{" + "version=" + version + ", archiveOffset=" + archiveOffset - + ", kafkaOffset=" + kafkaOffset + '}'; + return "SerializedDatasourceOffset{" + "version=" + version + ", hdfsOffset" + hdfsOffset + ", archiveOffset=" + + archiveOffset + ", kafkaOffset=" + kafkaOffset + '}'; } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/ConditionWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/ConditionWalker.java index ad85fa15..9e086bc8 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/ConditionWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/ConditionWalker.java @@ -54,6 +54,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.w3c.dom.Element; +import org.xml.sax.SAXException; + +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; import java.util.HashSet; import java.util.Set; @@ -96,8 +100,9 @@ public ConditionWalker(DSLContext ctx, boolean bloomEnabled, boolean withoutFilt this.combinedMatchSet = new HashSet<>(); } - public Condition fromString(String inXml, boolean streamQuery) throws Exception { - this.streamQuery = streamQuery; + public Condition fromString(String inXml, boolean streamQueryInput) + throws IllegalStateException, ParserConfigurationException, IOException, SAXException { + this.streamQuery = streamQueryInput; return fromString(inXml); } @@ -107,25 +112,25 @@ public Set> conditionRequiredTables() { } @Override - public Condition emitLogicalOperation(String op, Object l, Object r) throws Exception { + public Condition emitLogicalOperation(String op, Object l, Object r) throws IllegalArgumentException { Condition rv; Condition left = (Condition) l; Condition right = (Condition) r; if (op == null) { - throw new Exception("Parse error, unbalanced elements. " + left.toString()); + throw new IllegalArgumentException("Parse error, unbalanced elements. " + left.toString()); } - if (op.equalsIgnoreCase("AND")) { + if ("AND".equalsIgnoreCase(op)) { rv = left.and(right); } - else if (op.equalsIgnoreCase("OR")) { + else if ("OR".equalsIgnoreCase(op)) { rv = left.or(right); } - else if (op.equalsIgnoreCase("NOT")) { + else if ("NOT".equalsIgnoreCase(op)) { rv = left.not(); } else { - throw new Exception( + throw new IllegalArgumentException( "Parse error, unssorted logical operation. op:" + op + " expression:" + left.toString() ); } @@ -133,21 +138,21 @@ else if (op.equalsIgnoreCase("NOT")) { } @Override - public Condition emitUnaryOperation(String op, Element current) throws Exception { + public Condition emitUnaryOperation(String op, Element current) throws IllegalArgumentException { Condition rv = emitElem(current); LOGGER.info("ConditionWalker.emitUnaryOperation incoming op:" + op + " element:" + current); if (op == null) { - throw new Exception("Parse error, op was null"); + throw new IllegalArgumentException("Parse error, op was null"); } if (rv != null) { - if (op.equalsIgnoreCase("NOT")) { + if ("NOT".equalsIgnoreCase(op)) { rv = rv.not(); } else { - throw new Exception( + throw new IllegalArgumentException( "Parse error, unsupported logical operation. op:" + op + " expression:" + rv.toString() ); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/EarliestWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/EarliestWalker.java index a20c5c66..ecca54ea 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/EarliestWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/EarliestWalker.java @@ -75,7 +75,7 @@ Long emitElem(Element current) { long earliestEpoch = globalEarliestEpoch; - if (tag.equalsIgnoreCase("earliest") || tag.equalsIgnoreCase("index_earliest")) { + if ("earliest".equalsIgnoreCase(tag) || "index_earliest".equalsIgnoreCase(tag)) { earliestEpoch = Long.parseLong(value); @@ -88,42 +88,46 @@ Long emitElem(Element current) { } @Override - public Long emitLogicalOperation(String op, Object l, Object r) throws Exception { + public Long emitLogicalOperation(String op, Object l, Object r) throws IllegalStateException { Long rv; Long left = (Long) l; Long right = (Long) r; if (op == null) { - throw new Exception("Parse error, unbalanced elements. " + left); + throw new IllegalStateException("Parse error, unbalanced elements. " + left); } - if (op.equalsIgnoreCase("AND") || op.equalsIgnoreCase("OR")) { + if ("AND".equalsIgnoreCase(op) || "OR".equalsIgnoreCase(op)) { rv = left < right ? left : right; } else { - throw new Exception("Parse error, unsorted logical operation. op: " + op + " expression: " + left); + throw new IllegalStateException( + "Parse error, unsorted logical operation. op: " + op + " expression: " + left + ); } return rv; } @Override - public Long emitUnaryOperation(String op, Element current) throws Exception { + public Long emitUnaryOperation(String op, Element current) throws IllegalStateException { Long rv = emitElem(current); LOGGER.info("EarliestWalker.emitUnaryOperation incoming op: " + op + " element: " + current); if (op == null) { - throw new Exception("Parse error op was null"); + throw new IllegalStateException("Parse error op was null"); } if (rv != null) { - if (op.equalsIgnoreCase("NOT")) { + if ("NOT".equalsIgnoreCase(op)) { // Unary operations ignored rv = null; } else { - throw new Exception("Parse error, unsupported logical operation: " + op + " expression: " + rv); + throw new IllegalStateException( + "Parse error, unsupported logical operation: " + op + " expression: " + rv + ); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java new file mode 100644 index 00000000..55efe709 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java @@ -0,0 +1,99 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.planner.walker; + +import com.teragrep.jue_01.GlobToRegEx; +import org.w3c.dom.Element; +import org.xml.sax.SAXException; + +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; + +// HDFS planner walker, only aims to filter out the topics as the planner only outputs the metadata for AVRO-files containing the records. The rest of the condition handling is done in the separate tasker walker. +public class HdfsConditionWalker extends XmlWalker { + + @Override + String emitElem(Element current) { + String tag = current.getTagName(); + String value = current.getAttribute("value"); + String operation = current.getAttribute("operation"); + + String queryCondition = null; + // only index equals supported + if ("index".equalsIgnoreCase(tag)) { + if ("EQUALS".equalsIgnoreCase(operation)) { + queryCondition = GlobToRegEx.regexify(value); + } + } + return queryCondition; + } + + public String fromString(String inXml) throws ParserConfigurationException, IOException, SAXException { + return super.fromString(inXml); + } + + @Override + String emitLogicalOperation(String op, Object l, Object r) throws IllegalStateException { + String left = (String) l; + String right = (String) r; + + String rv = null; + /* + index can not have two values at the same go therefore "AND".equals(op) + is not implemented + */ + if ("OR".equals(op)) { + rv = "(" + left + "|" + right + ")"; + } + return rv; + } + + @Override + String emitUnaryOperation(String op, Element current) throws IllegalStateException { + // NOT is a filter, not a topic matcher + return null; + } +} diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/KafkaWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/KafkaWalker.java index 8533f2bf..51997989 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/KafkaWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/KafkaWalker.java @@ -47,6 +47,10 @@ import com.teragrep.jue_01.GlobToRegEx; import org.w3c.dom.Element; +import org.xml.sax.SAXException; + +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; /** *

Kafka Walker

@@ -64,20 +68,20 @@ String emitElem(Element current) { String queryCondition = null; // only index equals supported - if (tag.equalsIgnoreCase("index")) { - if (operation.equalsIgnoreCase("EQUALS")) { + if ("index".equalsIgnoreCase(tag)) { + if ("EQUALS".equalsIgnoreCase(operation)) { queryCondition = GlobToRegEx.regexify(value); } } return queryCondition; } - public String fromString(String inXml) throws Exception { + public String fromString(String inXml) throws ParserConfigurationException, IOException, SAXException { return super.fromString(inXml); } @Override - public String emitLogicalOperation(String op, Object l, Object r) throws Exception { + public String emitLogicalOperation(String op, Object l, Object r) { String left = (String) l; String right = (String) r; diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/PlainWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/PlainWalker.java index 15c144d9..36769a70 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/PlainWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/PlainWalker.java @@ -81,12 +81,12 @@ public String traverse(TreeWalker walker, String indent) { private String indent = "\t"; @Override - public Element emitLogicalOperation(String op, Object l, Object r) throws Exception { + public Element emitLogicalOperation(String op, Object l, Object r) throws IllegalStateException { Element rv = null; String left = (String) l; String right = (String) r; if (op == null) { - throw new Exception("Parse error, unbalanced elements. " + l.toString()); + throw new IllegalStateException("Parse error, unbalanced elements. " + l.toString()); } System.out.println(indent + op); indent += "\t"; @@ -96,10 +96,10 @@ public Element emitLogicalOperation(String op, Object l, Object r) throws Except } @Override - public Element emitUnaryOperation(String op, Element current) throws Exception { + public Element emitUnaryOperation(String op, Element current) throws IllegalStateException { Element rv = null; if (op == null) { - throw new Exception("Parse error, unknown operation: " + op + " expression:" + current); + throw new IllegalStateException("Parse error, unknown operation: " + op + " expression:" + current); } System.out.println(indent + op); indent += "\t"; diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/Util.java b/src/main/java/com/teragrep/pth_06/planner/walker/Util.java index 55e6aedd..ab49836a 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/Util.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/Util.java @@ -45,6 +45,8 @@ */ package com.teragrep.pth_06.planner.walker; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.w3c.dom.Element; import javax.xml.transform.*; @@ -68,6 +70,8 @@ */ public class Util { + private static final Logger LOGGER = LoggerFactory.getLogger(Util.class); + public static String stripQuotes(String value) { Matcher m = Pattern.compile("^\"(.*)\"$").matcher(value); Matcher m1 = Pattern.compile("^'(.*)'$").matcher(value); @@ -106,8 +110,9 @@ public static long relativeTimeModifier(Timestamp timestamp, String value) { Instant rv = null; Matcher matcher = Pattern.compile("^-?\\d+").matcher(value); - if (!matcher.find()) + if (!matcher.find()) { throw new NumberFormatException("Unknown relative time modifier string [" + value + "]"); + } long v = Long.parseLong(matcher.group()); matcher.group().length(); @@ -213,8 +218,14 @@ public static String elementAsString(Element el) { str = buffer.toString(); } catch (TransformerConfigurationException tex) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(tex.getMessage(), tex); + } } catch (TransformerException ex) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(ex.getMessage()); + } } return str; } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java index dce9a1fc..9d6dc2bb 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java @@ -55,9 +55,12 @@ import org.w3c.dom.traversal.NodeFilter; import org.w3c.dom.traversal.TreeWalker; import org.xml.sax.InputSource; +import org.xml.sax.SAXException; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; import java.io.StringReader; import java.util.Arrays; import java.util.List; @@ -81,7 +84,7 @@ public XmlWalker() { } - public T fromString(String inXml) throws Exception { + public T fromString(String inXml) throws ParserConfigurationException, IOException, SAXException { T rv; DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance(); DocumentBuilder loader = factory.newDocumentBuilder(); @@ -102,9 +105,8 @@ public T fromString(String inXml) throws Exception { * @param walker * @param op operation String * @return Class which expr-part contains actual catalyst query tree - * @throws Exception */ - public T traverse(TreeWalker walker, String op) throws Exception { + public T traverse(TreeWalker walker, String op) { Node parend = walker.getCurrentNode(); Element current = ((Element) parend); T rv = null; @@ -122,7 +124,7 @@ public T traverse(TreeWalker walker, String op) throws Exception { NodeList children = parend.getChildNodes(); int count = children.getLength(); if (count < 1 || count > 2) { - throw new Exception("Error, wrong number of children:" + count + " op:" + op); + throw new IllegalStateException("Error, wrong number of children:" + count + " op:" + op); } // get left and right Node left = walker.firstChild(); @@ -150,7 +152,7 @@ public T traverse(TreeWalker walker, String op) throws Exception { } if (lft != null && rht != null) { if (op == null) { - throw new Exception("Parse error, unbalanced elements. " + lft.toString()); + throw new IllegalStateException("Parse error, unbalanced elements. " + lft.toString()); } rv = emitLogicalOperation(op, lft, rht); } @@ -162,11 +164,12 @@ else if (rht == null) { } break; } + default: } } else { // leaf - if (op != null && op.equals("NOT")) { + if (op != null && "NOT".equals(op)) { LOGGER.debug("Emit Unary operation op:" + op + " l:" + current); rv = emitUnaryOperation(op, current); } @@ -180,7 +183,7 @@ else if (rht == null) { // System.out.println("XmlWalker.traverse type:"+rv.getClass().getName() +" returns:"+((T)rv).toString()); return (T) rv; } - catch (Exception e) { + catch (IllegalStateException e) { LOGGER.error(e.toString()); } return null; @@ -199,7 +202,7 @@ else if (rht == null) { * * @return correct query according to implementation class */ - abstract T emitLogicalOperation(String op, Object left, Object right) throws Exception; + abstract T emitLogicalOperation(String op, Object left, Object right) throws IllegalStateException; /** * Abstract method which is called during traverse. Emits appropriate unary operation @@ -207,7 +210,7 @@ else if (rht == null) { * @param current DOM-element * @return correct query according to implementation class */ - abstract T emitUnaryOperation(String op, Element current) throws Exception; + abstract T emitUnaryOperation(String op, Element current) throws IllegalStateException; // escape special chars inside value /** @@ -218,15 +221,17 @@ else if (rht == null) { */ public static String escapeSpecialCharacters(String input) { final List specialCharacters = Arrays.asList("\\", "*", "+", "?", "%"); - if (input.equals("*")) { + if ("*".equals(input)) { return "%"; } else { return Arrays.stream(input.split("")).map((c) -> { - if (specialCharacters.contains(c)) + if (specialCharacters.contains(c)) { return "\\" + c; - else + } + else { return c; + } }).collect(Collectors.joining()); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/BloomQueryCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/BloomQueryCondition.java index a3a0a360..42d11fcb 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/BloomQueryCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/BloomQueryCondition.java @@ -51,7 +51,7 @@ public interface BloomQueryCondition { - boolean isBloomSearchCondition(); + public abstract boolean isBloomSearchCondition(); - Set> requiredTables(); + public abstract Set> requiredTables(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/CategoryTableCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/CategoryTableCondition.java index f3e10e6d..ae5ed243 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/CategoryTableCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/CategoryTableCondition.java @@ -102,10 +102,12 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || getClass() != object.getClass()) + } + if (object == null || getClass() != object.getClass()) { return false; + } final CategoryTableCondition cast = (CategoryTableCondition) object; return comparedTo.equals(cast.comparedTo) && bloomTermCondition.equals(cast.bloomTermCondition) && typeIdCondition.equals(cast.typeIdCondition) && categoryTable.equals(cast.categoryTable); diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java index 80ddd428..83a77cd6 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java @@ -49,6 +49,7 @@ import java.sql.Date; import java.time.Instant; +import java.util.Objects; import static com.teragrep.pth_06.jooq.generated.journaldb.Journaldb.JOURNALDB; @@ -87,13 +88,21 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final EarliestCondition cast = (EarliestCondition) object; return this.value.equals(cast.value); } + + @Override + public int hashCode() { + return Objects.hashCode(value); + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java index 4faad3f6..276e10db 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java @@ -55,6 +55,8 @@ import java.util.Set; +import java.util.Objects; + /** * Creates a query condition from provided dom element */ @@ -92,6 +94,8 @@ public Condition condition() { QueryCondition host = new HostCondition(value, operation, config.streamQuery()); condition = host.condition(); break; + default: + LOGGER.debug("Element tag was not index, sourcetype or host: <{}>", tag); } if (!config.streamQuery()) { // Handle also time qualifiers @@ -132,13 +136,21 @@ public Set> requiredTables() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final ElementCondition cast = (ElementCondition) object; return this.element.equals(cast.element) && this.config.equals(cast.config); } + + @Override + public int hashCode() { + return Objects.hash(element, config); + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java index d79d836a..eaa1cf59 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java @@ -48,6 +48,8 @@ import com.teragrep.pth_06.planner.StreamDBClient; import org.jooq.Condition; +import java.util.Objects; + import static com.teragrep.pth_06.jooq.generated.streamdb.Streamdb.STREAMDB; public final class HostCondition implements QueryCondition { @@ -70,7 +72,7 @@ public Condition condition() { else { condition = StreamDBClient.GetArchivedObjectsFilterTable.host.like(value.replace('*', '%').toLowerCase()); } - if (operation.equalsIgnoreCase("NOT_EQUALS")) { + if ("NOT_EQUALS".equalsIgnoreCase(operation)) { condition = condition.not(); } return condition; @@ -78,14 +80,22 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final HostCondition cast = (HostCondition) object; return this.streamQuery == cast.streamQuery && this.value.equals(cast.value) && this.operation.equals(cast.operation); } + + @Override + public int hashCode() { + return Objects.hash(value, operation, streamQuery); + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java index a49a1951..2cb3d0e0 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java @@ -48,6 +48,8 @@ import com.teragrep.pth_06.planner.StreamDBClient; import org.jooq.Condition; +import java.util.Objects; + import static com.teragrep.pth_06.jooq.generated.streamdb.Streamdb.STREAMDB; public final class IndexCondition implements QueryCondition { @@ -71,7 +73,7 @@ public Condition condition() { condition = StreamDBClient.GetArchivedObjectsFilterTable.directory .like(value.replace('*', '%').toLowerCase()); } - if (operation.equalsIgnoreCase("NOT_EQUALS")) { + if ("NOT_EQUALS".equalsIgnoreCase(operation)) { condition = condition.not(); } return condition; @@ -79,14 +81,22 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final IndexCondition cast = (IndexCondition) object; return this.streamQuery == cast.streamQuery && this.value.equals(cast.value) && this.operation.equals(cast.operation); } + + @Override + public int hashCode() { + return Objects.hash(value, operation, streamQuery); + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java index c8fb9505..8fa4efc5 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java @@ -54,6 +54,7 @@ import org.slf4j.LoggerFactory; import java.util.HashSet; +import java.util.Objects; import java.util.Set; import static com.teragrep.pth_06.jooq.generated.bloomdb.Bloomdb.BLOOMDB; @@ -140,13 +141,21 @@ public Set> requiredTables() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final IndexStatementCondition cast = (IndexStatementCondition) object; return this.value.equals(cast.value) && this.config.equals(cast.config); } + + @Override + public int hashCode() { + return Objects.hash(value, config); + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java index d11a7781..e26add38 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java @@ -49,6 +49,7 @@ import java.sql.Date; import java.time.Instant; +import java.util.Objects; import static com.teragrep.pth_06.jooq.generated.journaldb.Journaldb.JOURNALDB; @@ -84,13 +85,21 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final LatestCondition cast = (LatestCondition) object; return this.value.equals(cast.value); } + + @Override + public int hashCode() { + return Objects.hashCode(value); + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/QueryCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/QueryCondition.java index b9dec429..a71064a4 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/QueryCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/QueryCondition.java @@ -49,5 +49,5 @@ public interface QueryCondition { - Condition condition(); + public abstract Condition condition(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/RegexLikeCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/RegexLikeCondition.java index 9026227b..346c0717 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/RegexLikeCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/RegexLikeCondition.java @@ -89,10 +89,12 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || object.getClass() != this.getClass()) + } + if (object == null || object.getClass() != this.getClass()) { return false; + } final RegexLikeCondition cast = (RegexLikeCondition) object; return valueField.equals(cast.valueField) && comparedToField.equals(cast.comparedToField); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java index 2c3c9aee..6ee21fff 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java @@ -48,6 +48,8 @@ import com.teragrep.pth_06.planner.StreamDBClient; import org.jooq.Condition; +import java.util.Objects; + import static com.teragrep.pth_06.jooq.generated.streamdb.Streamdb.STREAMDB; public final class SourceTypeCondition implements QueryCondition { @@ -70,7 +72,7 @@ public Condition condition() { else { condition = StreamDBClient.GetArchivedObjectsFilterTable.stream.like(value.replace('*', '%').toLowerCase()); } - if (operation.equalsIgnoreCase("NOT_EQUALS")) { + if ("NOT_EQUALS".equalsIgnoreCase(operation)) { condition = condition.not(); } return condition; @@ -78,14 +80,22 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final SourceTypeCondition cast = (SourceTypeCondition) object; return this.streamQuery == cast.streamQuery && this.value.equals(cast.value) && this.operation.equals(cast.operation); } + + @Override + public int hashCode() { + return Objects.hash(value, operation, streamQuery); + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ValidElement.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ValidElement.java index 0ea44822..49456144 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ValidElement.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ValidElement.java @@ -47,6 +47,8 @@ import org.w3c.dom.Element; +import java.util.Objects; + public final class ValidElement { private final Element element; @@ -88,16 +90,24 @@ public String operation() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final ValidElement cast = (ValidElement) object; boolean equalName = this.element.getTagName().equals(cast.element.getTagName()); boolean equalOperation = this.element.getAttribute("operation").equals(cast.element.getAttribute("operation")); boolean equalValue = this.element.getAttribute("value").equals(cast.element.getAttribute("value")); return equalName && equalOperation && equalValue; } + + @Override + public int hashCode() { + return Objects.hash(element.getTagName(), element.getAttribute("operation"), element.getAttribute("value")); + } } diff --git a/src/main/java/com/teragrep/pth_06/scheduler/Batch.java b/src/main/java/com/teragrep/pth_06/scheduler/Batch.java index e3bf3eb4..daacf731 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/Batch.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/Batch.java @@ -47,6 +47,7 @@ import com.teragrep.pth_06.config.Config; import com.teragrep.pth_06.planner.ArchiveQuery; +import com.teragrep.pth_06.planner.HdfsQuery; import com.teragrep.pth_06.planner.KafkaQuery; import org.apache.spark.sql.connector.read.streaming.Offset; import org.slf4j.Logger; @@ -67,17 +68,18 @@ public final class Batch extends LinkedList> { private long numberOfBatches = 0; private final LinkedList runQueueArray; private final Config config; + private final HdfsQuery hdfsQuery; private final ArchiveQuery archiveQuery; private final KafkaQuery kafkaQuery; - public Batch(Config config, ArchiveQuery aq, KafkaQuery kq) { + public Batch(Config config, HdfsQuery hq, ArchiveQuery aq, KafkaQuery kq) { this.config = config; this.runQueueArray = new LinkedList<>(); for (int i = 0; i < config.batchConfig.numPartitions; i++) { this.runQueueArray.add(new BatchTaskQueue()); } - + this.hdfsQuery = hq; this.archiveQuery = aq; this.kafkaQuery = kq; } @@ -85,20 +87,30 @@ public Batch(Config config, ArchiveQuery aq, KafkaQuery kq) { public Batch processRange(Offset start, Offset end) { LOGGER.debug("processRange"); - BatchSliceCollection slice = null; + BatchSliceCollection slice = new StubBatchSliceCollection(); + + if (config.isHdfsEnabled) { + slice = new HdfsBatchSliceCollection(this.hdfsQuery).processRange(start, end); + } + if (config.isArchiveEnabled) { - slice = new ArchiveBatchSliceCollection(this.archiveQuery).processRange(start, end); + if (slice.isEmpty()) { + slice = new ArchiveBatchSliceCollection(this.archiveQuery).processRange(start, end); + } + else { + slice.addAll(new ArchiveBatchSliceCollection(this.archiveQuery).processRange(start, end)); + } } if (config.isKafkaEnabled) { - if (slice == null) { + if (slice.isEmpty()) { slice = new KafkaBatchSliceCollection(this.kafkaQuery).processRange(start, end); } else { slice.addAll(new KafkaBatchSliceCollection(this.kafkaQuery).processRange(start, end)); } } - if (slice != null && !slice.isEmpty()) { + if (!slice.isEmpty()) { this.addSlice(slice); } diff --git a/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java b/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java index 330eabb7..6dc9fdfa 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java @@ -46,6 +46,7 @@ package com.teragrep.pth_06.scheduler; import com.teragrep.pth_06.ArchiveS3ObjectMetadata; +import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.KafkaTopicPartitionOffsetMetadata; import java.io.Serializable; @@ -61,24 +62,35 @@ */ public final class BatchSlice implements Serializable { - public enum Type { - ARCHIVE, KAFKA + public static enum Type { + ARCHIVE, KAFKA, HDFS } public final Type type; public final ArchiveS3ObjectMetadata archiveS3ObjectMetadata; public final KafkaTopicPartitionOffsetMetadata kafkaTopicPartitionOffsetMetadata; + public final HdfsFileMetadata hdfsFileMetadata; public BatchSlice(ArchiveS3ObjectMetadata archiveS3ObjectMetadata) { this.type = Type.ARCHIVE; this.archiveS3ObjectMetadata = archiveS3ObjectMetadata; this.kafkaTopicPartitionOffsetMetadata = null; + this.hdfsFileMetadata = null; } public BatchSlice(KafkaTopicPartitionOffsetMetadata kafkaTopicPartitionOffsetMetadata) { this.type = Type.KAFKA; this.archiveS3ObjectMetadata = null; this.kafkaTopicPartitionOffsetMetadata = kafkaTopicPartitionOffsetMetadata; + this.hdfsFileMetadata = null; + } + + public BatchSlice(HdfsFileMetadata hdfsFileMetadata) { + this.type = Type.HDFS; + this.archiveS3ObjectMetadata = null; + this.kafkaTopicPartitionOffsetMetadata = null; + this.hdfsFileMetadata = hdfsFileMetadata; + } public long getSize() { @@ -95,6 +107,14 @@ public long getSize() { case KAFKA: // TODO estimate based on offset delta return 1024 * 1024 * 16; + case HDFS: + // The files in HDFS have a default size limiter set to them during creation. Use that limit (64000000) or hdfsFileMetadata.getFileSize(). + if (hdfsFileMetadata != null) { + return hdfsFileMetadata.hdfsFileSize; + } + else { + throw new RuntimeException("Expected HDFS file metadata, instead was null"); + } default: throw new IllegalStateException("unknown BatchSliceType " + type); } @@ -103,6 +123,7 @@ public long getSize() { @Override public String toString() { return "BatchSlice{" + "batchSliceType=" + type + ", archiveS3ObjectMetadata=" + archiveS3ObjectMetadata - + ", kafkaTopicPartitionOffsetMetadata=" + kafkaTopicPartitionOffsetMetadata + '}'; + + ", kafkaTopicPartitionOffsetMetadata=" + kafkaTopicPartitionOffsetMetadata + ", hdfsFileMetadata=" + + hdfsFileMetadata + '}'; } } diff --git a/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java b/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java new file mode 100644 index 00000000..ed334c8f --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java @@ -0,0 +1,80 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.scheduler; + +import com.teragrep.pth_06.HdfsFileMetadata; +import com.teragrep.pth_06.planner.HdfsQuery; +import com.teragrep.pth_06.planner.offset.DatasourceOffset; +import com.teragrep.pth_06.planner.offset.HdfsOffset; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.spark.sql.connector.read.streaming.Offset; + +import java.util.LinkedList; + +public final class HdfsBatchSliceCollection extends BatchSliceCollection { + + private final Logger LOGGER = LoggerFactory.getLogger(HdfsBatchSliceCollection.class); + private final HdfsQuery hq; + + public HdfsBatchSliceCollection(HdfsQuery hq) { + super(); + this.hq = hq; + } + + public HdfsBatchSliceCollection processRange(Offset start, Offset end) { + // HDFS: + LOGGER.debug("processRange(): args: start: " + start + " end: " + end); + HdfsOffset hdfsStartOffset = ((DatasourceOffset) start).getHdfsOffset(); + HdfsOffset hdfsEndOffset = ((DatasourceOffset) end).getHdfsOffset(); + LinkedList result = hq.processBetweenHdfsFileMetadata(hdfsStartOffset, hdfsEndOffset); + for (HdfsFileMetadata r : result) { + this.add(new BatchSlice(r)); + } + return this; + } + +} diff --git a/src/main/java/com/teragrep/pth_06/scheduler/StubBatchSliceCollection.java b/src/main/java/com/teragrep/pth_06/scheduler/StubBatchSliceCollection.java new file mode 100644 index 00000000..2afa3d53 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/scheduler/StubBatchSliceCollection.java @@ -0,0 +1,60 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.scheduler; + +import org.apache.spark.sql.connector.read.streaming.Offset; + +public final class StubBatchSliceCollection extends BatchSliceCollection { + + public StubBatchSliceCollection() { + super(); + } + + @Override + public BatchSliceCollection processRange(Offset start, Offset end) { + throw new UnsupportedOperationException("StubBatchSliceCollection should not be processed."); + } +} diff --git a/src/main/java/com/teragrep/pth_06/task/ArchiveMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/ArchiveMicroBatchInputPartitionReader.java index 1fc45a46..6920112d 100644 --- a/src/main/java/com/teragrep/pth_06/task/ArchiveMicroBatchInputPartitionReader.java +++ b/src/main/java/com/teragrep/pth_06/task/ArchiveMicroBatchInputPartitionReader.java @@ -186,8 +186,9 @@ public InternalRow get() { @Override public void close() throws IOException { - if (LOGGER.isDebugEnabled()) + if (LOGGER.isDebugEnabled()) { LOGGER.debug("ArchiveMicroBatchInputPartitionReader.close>"); + } if (rowConverter != null) { rowConverter.close(); } diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java new file mode 100644 index 00000000..487ed4ac --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java @@ -0,0 +1,97 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.task; + +import com.teragrep.pth_06.HdfsFileMetadata; +import com.teragrep.pth_06.config.HdfsConfig; +import org.apache.spark.sql.connector.read.InputPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedList; + +public class HdfsMicroBatchInputPartition implements InputPartition { + + public final Logger LOGGER = LoggerFactory.getLogger(HdfsMicroBatchInputPartition.class); + + public final LinkedList taskObjectList; + + public final long includeRecordEpochAndAfter; // Represents the cutoff epoch which dictates which files should not be fetched from HDFS based on their timestamps. + public final String hdfsPath; // Represents the working directory path in HDFS filesystem. + public final String hdfsUri; // Represents the address of the HDFS server. + public final String useHdfsHostname; + public final String hdfsTransferProtection; + public final String hdfsCipherSuites; + public final boolean useMockHdfsDatabase; // Represents the configuration parameter if mock database should be used or not. Used only for testing. + public final String kerberosAuthentication; + public final String kerberosAuthorization; + public final String kerberosPrincipalPattern; + public final String kerberosKdc; + public final String kerberosRealm; + public final String kerberosKeytabUser; + public final String kerberosKeytabPath; + public final String kerberosAutorenewal; + + public HdfsMicroBatchInputPartition(HdfsConfig hdfsConfig, LinkedList hdfsTaskList) { + this.taskObjectList = hdfsTaskList; + kerberosAutorenewal = hdfsConfig.kerberosAutorenewal; + hdfsUri = hdfsConfig.hdfsUri; + hdfsPath = hdfsConfig.hdfsPath; + includeRecordEpochAndAfter = hdfsConfig.includeRecordEpochAndAfter; + useHdfsHostname = hdfsConfig.useHdfsHostname; + hdfsTransferProtection = hdfsConfig.hdfsTransferProtection; + hdfsCipherSuites = hdfsConfig.hdfsCipherSuites; + useMockHdfsDatabase = hdfsConfig.useMockHdfsDatabase; + kerberosAuthentication = hdfsConfig.kerberosAuthentication; + kerberosAuthorization = hdfsConfig.kerberosAuthorization; + kerberosPrincipalPattern = hdfsConfig.kerberosPrincipalPattern; + kerberosKdc = hdfsConfig.kerberosKdc; + kerberosRealm = hdfsConfig.kerberosRealm; + kerberosKeytabUser = hdfsConfig.kerberosKeytabUser; + kerberosKeytabPath = hdfsConfig.kerberosKeytabPath; + } + +} diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java new file mode 100644 index 00000000..90f4792a --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java @@ -0,0 +1,121 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.task; + +import com.teragrep.pth_06.FileSystemFactoryImpl; +import com.teragrep.pth_06.HdfsFileMetadata; +import com.teragrep.pth_06.task.hdfs.HdfsRecordConversionImpl; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.LinkedList; + +public final class HdfsMicroBatchInputPartitionReader implements PartitionReader { + + final Logger LOGGER = LoggerFactory.getLogger(HdfsMicroBatchInputPartitionReader.class); + private final HdfsRecordConversionImpl hdfsRecordConversionImpl; + + public HdfsMicroBatchInputPartitionReader( + long cutoffEpoch, + String kerberosAuthentication, + String hdfsUri, + String hdfsPath, + String kerberosRealm, + String kerberosKdc, + String kerberosAuthorization, + String kerberosAutorenewal, + String UseHdfsHostname, + String kerberosPrincipalPattern, + String hdfsTransferProtection, + String hdfsCipherSuites, + String kerberosKeytabUser, + String kerberosKeytabPath, + LinkedList taskObjectList + ) throws IOException { + + FileSystem fs = new FileSystemFactoryImpl( + kerberosAuthentication, + hdfsUri, + kerberosRealm, + kerberosKdc, + kerberosAuthorization, + kerberosAutorenewal, + UseHdfsHostname, + kerberosPrincipalPattern, + hdfsTransferProtection, + hdfsCipherSuites, + kerberosKeytabUser, + kerberosKeytabPath + ).fileSystem(false); + + this.hdfsRecordConversionImpl = new HdfsRecordConversionImpl(fs, taskObjectList, cutoffEpoch); + } + + @Override + public boolean next() throws IOException { + // True if data is available, false if not. + return hdfsRecordConversionImpl.next(); + } + + @Override + public InternalRow get() { + // Fetches the available data. + InternalRow rv = hdfsRecordConversionImpl.row(); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("get(): <{}>", rv.getLong(7)); + } + return rv; + } + + @Override + public void close() throws IOException { + LOGGER.debug("HdfsMicroBatchInputPartitionReader.close"); + } +} diff --git a/src/main/java/com/teragrep/pth_06/task/TeragrepPartitionReaderFactory.java b/src/main/java/com/teragrep/pth_06/task/TeragrepPartitionReaderFactory.java index 589ccf79..c4773cd6 100644 --- a/src/main/java/com/teragrep/pth_06/task/TeragrepPartitionReaderFactory.java +++ b/src/main/java/com/teragrep/pth_06/task/TeragrepPartitionReaderFactory.java @@ -50,6 +50,8 @@ import org.apache.spark.sql.connector.read.PartitionReader; import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import java.io.IOException; + /** *

TeragrepPartitionReaderFactory

Used to create appropriate PartitionReaders based on the type of the * InputPartition provided. @@ -112,6 +114,31 @@ else if (inputPartition instanceof KafkaMicroBatchInputPartition) { kip.skipNonRFC5424Records ); } + else if (inputPartition instanceof HdfsMicroBatchInputPartition) { + HdfsMicroBatchInputPartition hip = (HdfsMicroBatchInputPartition) inputPartition; + try { + return new HdfsMicroBatchInputPartitionReader( + hip.includeRecordEpochAndAfter, + hip.kerberosAuthentication, + hip.hdfsUri, + hip.hdfsPath, + hip.kerberosRealm, + hip.kerberosKdc, + hip.kerberosAuthorization, + hip.kerberosAutorenewal, + hip.useHdfsHostname, + hip.kerberosPrincipalPattern, + hip.hdfsTransferProtection, + hip.hdfsCipherSuites, + hip.kerberosKeytabUser, + hip.kerberosKeytabPath, + hip.taskObjectList + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } // inputPartition is neither Archive nor Kafka type throw new RuntimeException( diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java new file mode 100644 index 00000000..07e31d77 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java @@ -0,0 +1,60 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.task.hdfs; + +import com.teragrep.pth_06.avro.SyslogRecord; + +import java.io.IOException; + +public interface AvroRead { + + public abstract boolean next(); + + public abstract SyslogRecord record(); + + public abstract void close() throws IOException; + +} diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java new file mode 100644 index 00000000..99d40b2f --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java @@ -0,0 +1,106 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.task.hdfs; + +import com.teragrep.pth_06.HdfsFileMetadata; +import com.teragrep.pth_06.avro.SyslogRecord; +import org.apache.avro.file.DataFileStream; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.LinkedList; + +// This class will allow reading the contents of the avro-files that are using SyslogRecord schema from hdfs. +public final class AvroReadImpl implements AvroRead { + + final Logger LOGGER = LoggerFactory.getLogger(AvroReadImpl.class); + + private final DataFileStream reader; + private final LinkedList syslogRecordBuffer; + + public AvroReadImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata) throws IOException { + this.reader = new DataFileStream<>( + fs.open(new Path(hdfsFileMetadata.hdfsFilePath)), + new SpecificDatumReader<>(SyslogRecord.class) + ); + syslogRecordBuffer = new LinkedList<>(); + } + + @Override + public boolean next() { + boolean hasnext = reader.hasNext(); + if (hasnext) { + syslogRecordBuffer.clear(); + syslogRecordBuffer.add(reader.next()); + return true; + } + else { + return false; + } + } + + @Override + public SyslogRecord record() { + if (syslogRecordBuffer.size() == 1) { + return syslogRecordBuffer.getFirst(); + } + else { + throw new IllegalStateException( + "Invalid amount of records in the buffer, expected 1 got " + syslogRecordBuffer.size() + ); + } + } + + @Override + public void close() throws IOException { + reader.close(); + } + +} diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java new file mode 100644 index 00000000..97914134 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java @@ -0,0 +1,58 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.task.hdfs; + +import org.apache.spark.sql.catalyst.InternalRow; + +import java.io.IOException; + +public interface HdfsRecordConversion { + + public abstract boolean next() throws IOException; + + public abstract InternalRow row(); + +} diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java new file mode 100644 index 00000000..62bfb059 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java @@ -0,0 +1,127 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.task.hdfs; + +import com.teragrep.pth_06.HdfsFileMetadata; +import com.teragrep.pth_06.avro.SyslogRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter; +import org.apache.spark.unsafe.types.UTF8String; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.LinkedList; + +// This class will read the records from avro-files fetched from HDFS with the help of AvroReadImpl and convert them to InternalRow used by pth_06. +public final class HdfsRecordConversionImpl implements HdfsRecordConversion { + + final Logger LOGGER = LoggerFactory.getLogger(HdfsRecordConversionImpl.class); + + private final LinkedList reads; + private final LinkedList taskObjectList; + private final long cutoffEpoch; + private final FileSystem fs; + + public HdfsRecordConversionImpl(FileSystem fs, LinkedList taskObjectList, long cutoffEpoch) { + this.fs = fs; + this.taskObjectList = taskObjectList; + this.reads = new LinkedList<>(); + this.cutoffEpoch = cutoffEpoch; + } + + @Override + public boolean next() throws IOException { + // Load the taskObjects from taskObjectList to reads list. + if (reads.isEmpty() && !taskObjectList.isEmpty()) { + open(); + } + + boolean hasNext = false; + while (!hasNext && !reads.isEmpty()) { + hasNext = reads.getFirst().next(); + if (!hasNext) { + reads.getFirst().close(); + reads.removeFirst(); + } + else { + // Time based inclusion WIP + SyslogRecord record = reads.getFirst().record(); + if (record.getTimestamp() < cutoffEpoch) { + hasNext = false; + } + } + } + return hasNext; + } + + @Override + public InternalRow row() { + SyslogRecord currentRecord = reads.getFirst().record(); + UnsafeRowWriter rowWriter = new UnsafeRowWriter(11); + rowWriter.reset(); + rowWriter.zeroOutNullBytes(); + rowWriter.write(0, currentRecord.getTimestamp()); + rowWriter.write(1, UTF8String.fromString(currentRecord.getPayload().toString())); + rowWriter.write(2, UTF8String.fromString(currentRecord.getDirectory().toString())); + rowWriter.write(3, UTF8String.fromString(currentRecord.getStream().toString())); + rowWriter.write(4, UTF8String.fromString(currentRecord.getHost().toString())); + rowWriter.write(5, UTF8String.fromString(currentRecord.getInput().toString())); + rowWriter.write(6, UTF8String.fromString(currentRecord.getPartition().toString())); + rowWriter.write(7, currentRecord.getOffset()); + rowWriter.write(8, UTF8String.fromString(currentRecord.getOrigin().toString())); + return rowWriter.getRow(); + } + + private void open() throws IOException { + for (HdfsFileMetadata taskObject : taskObjectList) { + reads.add(new AvroReadImpl(fs, taskObject)); + } + taskObjectList.clear(); + } + +} diff --git a/src/main/java/com/teragrep/pth_06/task/s3/Pth06S3Client.java b/src/main/java/com/teragrep/pth_06/task/s3/Pth06S3Client.java index 30e7c0e1..bf777805 100644 --- a/src/main/java/com/teragrep/pth_06/task/s3/Pth06S3Client.java +++ b/src/main/java/com/teragrep/pth_06/task/s3/Pth06S3Client.java @@ -92,11 +92,11 @@ public Pth06S3Client(String S3endPoint, String S3identity, String S3credential) /** * Defines which signing algorithm is used. Default is V2. * - * @param signerKind Signing algorithm kind. + * @param signerKindInput Signing algorithm kind. * @return Reference to this builder. */ - public Pth06S3Client withSigner(SignerKind signerKind) { - this.signerKind = signerKind; + public Pth06S3Client withSigner(SignerKind signerKindInput) { + this.signerKind = signerKindInput; return this; } @@ -106,8 +106,9 @@ public Pth06S3Client withSigner(SignerKind signerKind) { * @return AmazonS3Client using the builder's values. */ public AmazonS3 build() { - if (LOGGER.isDebugEnabled()) + if (LOGGER.isDebugEnabled()) { LOGGER.info("Building S3APIWrapper for endpoint {}.", S3endPoint); + } if (StringUtils.isBlank(this.S3endPoint)) { throw new IllegalStateException("S3 endpoint is required."); @@ -168,7 +169,7 @@ public AmazonS3 build() { /** * S3 signing algorithm kinds. */ - public enum SignerKind { + public static enum SignerKind { // These constants are from com.amazonaws.services.s3.AmazonS3Client. Constants are private so cannot be used // here directly. Values select the authorization signing algorithm. V2 has to be used with TOS, V4 with AWS. diff --git a/src/main/java/com/teragrep/pth_06/task/s3/RowConverter.java b/src/main/java/com/teragrep/pth_06/task/s3/RowConverter.java index aaa8b766..fd45fa3f 100644 --- a/src/main/java/com/teragrep/pth_06/task/s3/RowConverter.java +++ b/src/main/java/com/teragrep/pth_06/task/s3/RowConverter.java @@ -129,8 +129,9 @@ public RowConverter( this.stream = UTF8String.fromString(stream.toLowerCase()); this.host = UTF8String.fromString(host.toLowerCase()); - if (LOGGER.isDebugEnabled()) + if (LOGGER.isDebugEnabled()) { LOGGER.debug("RowConverter> created with partition:" + this.bucket + " path: " + this.path); + } this.rowWriter = new UnsafeRowWriter(11); @@ -138,8 +139,9 @@ public RowConverter( this.skipNonRFC5424Files = skipNonRFC5424Files; - if (LOGGER.isDebugEnabled()) + if (LOGGER.isDebugEnabled()) { LOGGER.info("Initialized s3client:" + s3client); + } // initial status this.currentOffset = 0L; @@ -246,12 +248,13 @@ static long rfc3339ToEpoch(ZonedDateTime zonedDateTime) { public InternalRow get() { //System.out.println("RowConverter.get>"); - if (LOGGER.isDebugEnabled()) + if (LOGGER.isDebugEnabled()) { LOGGER .debug( "RowConverter.get> Partition (" + this.id + "):" + bucket + "/" + path + " Get(" + currentOffset + ")" ); + } if (LOGGER.isDebugEnabled()) { LOGGER.debug("Parsersyslog event:" + rfc5424Frame.toString()); @@ -285,8 +288,9 @@ public InternalRow get() { rowWriter.write(7, currentOffset); rowWriter.write(8, UTF8String.fromBytes(origin)); - if (LOGGER.isDebugEnabled()) + if (LOGGER.isDebugEnabled()) { LOGGER.debug("Get Event, row=written"); + } auditPlugin .audit( diff --git a/src/test/java/com/teragrep/pth_06/FileSystemFactoryTest.java b/src/test/java/com/teragrep/pth_06/FileSystemFactoryTest.java new file mode 100644 index 00000000..9e6a2555 --- /dev/null +++ b/src/test/java/com/teragrep/pth_06/FileSystemFactoryTest.java @@ -0,0 +1,118 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06; + +import com.teragrep.pth_06.planner.MockHDFS; +import org.apache.hadoop.fs.FileSystem; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class FileSystemFactoryTest { + + private final String hdfsPath = "hdfs:///opt/teragrep/cfe_39/srv/"; + private String hdfsUri; // Can only be defined after starting the mock hdfs. + private final MockHDFS mockHDFS = new MockHDFS(hdfsPath); + + @BeforeEach + public void setUp() throws IOException, InterruptedException { + hdfsUri = mockHDFS.startMiniCluster(true); + } + + @AfterEach + public void teardown() { + mockHDFS.teardownMiniCluster(); + } + + @Test + public void createInsecureFileSystemTest() { + assertDoesNotThrow(() -> { + FileSystemFactoryImpl fileSystemFactory = new FileSystemFactoryImpl( + "", + hdfsUri, + "", + "", + "", + "", + "", + "", + "", + "", + "", + "" + ); + FileSystem fs = fileSystemFactory.fileSystem(true); + fs.close(); + }); + } + + @Test + public void createFileSystemFailureTest() { + FileSystemFactoryImpl fileSystemFactory = new FileSystemFactoryImpl( + "", + "wrongUri", + "", + "", + "", + "", + "", + "", + "", + "", + "", + "" + ); + IllegalArgumentException e = Assertions + .assertThrows(IllegalArgumentException.class, () -> fileSystemFactory.fileSystem(true)); + Assertions.assertEquals("java.net.UnknownHostException: wrongUri", e.getMessage()); + } + +} diff --git a/src/test/java/com/teragrep/pth_06/InstantationKafkaDisabledTest.java b/src/test/java/com/teragrep/pth_06/InstantationKafkaDisabledTest.java new file mode 100644 index 00000000..13ab627d --- /dev/null +++ b/src/test/java/com/teragrep/pth_06/InstantationKafkaDisabledTest.java @@ -0,0 +1,458 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06; + +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.Bucket; +import com.cloudbees.syslog.Facility; +import com.cloudbees.syslog.SDElement; +import com.cloudbees.syslog.Severity; +import com.cloudbees.syslog.SyslogMessage; +import com.teragrep.pth_06.planner.MockDBData; +import com.teragrep.pth_06.planner.MockHDFS; +import com.teragrep.pth_06.planner.MockKafkaConsumerFactory; +import com.teragrep.pth_06.task.s3.MockS3; +import com.teragrep.pth_06.task.s3.Pth06S3Client; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.streaming.StreamingQuery; +import org.apache.spark.sql.streaming.StreamingQueryException; +import org.apache.spark.sql.streaming.Trigger; +import org.jooq.Record11; +import org.jooq.Result; +import org.jooq.types.ULong; +import org.junit.jupiter.api.*; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.sql.Date; +import java.util.Iterator; +import java.util.Map; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.TimeoutException; +import java.util.zip.GZIPOutputStream; + +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class InstantationKafkaDisabledTest { + + private SparkSession spark = null; + + private final String s3endpoint = "http://127.0.0.1:48080"; + private final String s3identity = "s3identity"; + private final String s3credential = "s3credential"; + + private final MockS3 mockS3 = new MockS3(s3endpoint, s3identity, s3credential); + + private final MockDBData mockDBData = new MockDBData(); + + private long expectedRows = 0L; + + private final String hdfsPath = "hdfs:///opt/teragrep/cfe_39/srv/"; + private String hdfsUri; // Can only be defined after starting the mock hdfs. + private final MockHDFS mockHDFS = new MockHDFS(hdfsPath); + + @BeforeAll + public void prepareEnv() throws Exception { + //Logger.getRootLogger().setLevel(Level.ERROR); + //Logger.getLogger("org.apache.spark").setLevel(Level.WARN); + //Logger.getLogger("org.spark-project").setLevel(Level.WARN); + + mockS3.start(); + + // Start mock hdfs with a full set of records from kafka inserted inside. + hdfsUri = mockHDFS.startMiniCluster(true); + + spark = SparkSession + .builder() + .appName("Java Spark SQL basic example") + .master("local[2]") + .config("spark.driver.extraJavaOptions", "-Duser.timezone=EET") + .config("spark.executor.extraJavaOptions", "-Duser.timezone=EET") + .config("spark.sql.session.timeZone", "UTC") + .getOrCreate(); + + //spark.sparkContext().setLogLevel("ERROR"); + + expectedRows = preloadS3Data() + MockKafkaConsumerFactory.getNumRecords(); + /*Files 0.9 and 0.13 are stored to HDFS, 14 records out of 14 will be read from HDFS instead of Kafka.*/ + } + + @Test + public void fullScanDisabledKafkaTest() throws StreamingQueryException, TimeoutException { + // please notice that JAVA_HOME=/usr/lib/jvm/java-1.8.0 mvn clean test -Pdev is required + Dataset df = spark + .readStream() + .format("com.teragrep.pth_06.MockTeragrepDatasource") + .option("archive.enabled", "true") + .option("S3endPoint", s3endpoint) + .option("S3identity", s3identity) + .option("S3credential", s3credential) + .option("DBusername", "mock") + .option("DBpassword", "mock") + .option("DBurl", "mock") + .option("DBstreamdbname", "mock") + .option("DBjournaldbname", "mock") + .option("num_partitions", "1") + .option("queryXML", "") // Only affects HDFS execution in the current test configuration. + // audit information + .option("TeragrepAuditQuery", "index=f17") + .option("TeragrepAuditReason", "test run at fullScanTest()") + .option("TeragrepAuditUser", System.getProperty("user.name")) + // kafka options + .option("kafka.enabled", "false") + .option("kafka.bootstrap.servers", "") + .option("kafka.sasl.mechanism", "") + .option("kafka.security.protocol", "") + .option("kafka.sasl.jaas.config", "") + .option("kafka.useMockKafkaConsumer", "true") + .option("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + // HDFS options + .option("hdfs.enabled", "true") + .option("hdfs.hdfsPath", hdfsPath) + .option("hdfs.hdfsUri", hdfsUri) + .option("hdfs.UseHostName", "false") + .option("hdfs.transferProtection", "test") + .option("hdfs.cipherSuites", "test") + .option("hdfs.useMockHdfsDatabase", "true") + .option("hdfs.krbAuthentication", "false") + .option("hdfs.krbAuthorization", "test") + .option("hdfs.krbPrincipalPattern", "test") + .option("hdfs.krbKdc", "test") + .option("hdfs.krbRealm", "test") + .option("hdfs.krbKeytabUser", "test") + .option("hdfs.krbKeytabPath", "test") + .option("hdfs.useKerberosAutorenewal", "true") + .option("hdfs.includeRecordEpochAndAfter", 0L) // record timestamps are in 1650872090806000 granularity (epochMicros) + .load(); + + Dataset df2 = df.agg(functions.count("*")); + + StreamingQuery streamingQuery = df2 + .writeStream() + .outputMode("complete") + .format("memory") + .trigger(Trigger.ProcessingTime(0)) + .queryName("MockArchiveQuery") + .option("checkpointLocation", "/tmp/checkpoint/" + UUID.randomUUID()) + .option("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + .start(); + + StreamingQuery sq = df.writeStream().foreachBatch((ds, i) -> { + ds.show(false); + }).start(); + sq.processAllAvailable(); + sq.stop(); + sq.awaitTermination(); + + long rowCount = 0; + while (!streamingQuery.awaitTermination(1000)) { + + long resultSize = spark.sqlContext().sql("SELECT * FROM MockArchiveQuery").count(); + if (resultSize > 0) { + rowCount = spark.sqlContext().sql("SELECT * FROM MockArchiveQuery").first().getAs(0); + System.out.println(rowCount); + } + if ( + streamingQuery.lastProgress() == null + || streamingQuery.status().message().equals("Initializing sources") + ) { + // query has not started + } + else if (streamingQuery.lastProgress().sources().length != 0) { + if (isArchiveDone(streamingQuery)) { + streamingQuery.stop(); + } + } + } + Assertions.assertEquals(expectedRows, rowCount); + } + + @Test + public void fullScanDisabledHdfsAndKafkaTest() throws StreamingQueryException, TimeoutException { + // please notice that JAVA_HOME=/usr/lib/jvm/java-1.8.0 mvn clean test -Pdev is required + Dataset df = spark + .readStream() + .format("com.teragrep.pth_06.MockTeragrepDatasource") + .option("archive.enabled", "true") + .option("S3endPoint", s3endpoint) + .option("S3identity", s3identity) + .option("S3credential", s3credential) + .option("DBusername", "mock") + .option("DBpassword", "mock") + .option("DBurl", "mock") + .option("DBstreamdbname", "mock") + .option("DBjournaldbname", "mock") + .option("num_partitions", "1") + .option("queryXML", "") // Only affects HDFS execution in the current test configuration. + // audit information + .option("TeragrepAuditQuery", "index=f17") + .option("TeragrepAuditReason", "test run at fullScanTest()") + .option("TeragrepAuditUser", System.getProperty("user.name")) + // kafka options + .option("kafka.enabled", "false") + .option("kafka.bootstrap.servers", "") + .option("kafka.sasl.mechanism", "") + .option("kafka.security.protocol", "") + .option("kafka.sasl.jaas.config", "") + .option("kafka.useMockKafkaConsumer", "true") + .option("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + // HDFS options + .option("hdfs.enabled", "false") + .option("hdfs.hdfsPath", hdfsPath) + .option("hdfs.hdfsUri", hdfsUri) + .option("hdfs.UseHostName", "false") + .option("hdfs.transferProtection", "test") + .option("hdfs.cipherSuites", "test") + .option("hdfs.useMockHdfsDatabase", "true") + .option("hdfs.krbAuthentication", "false") + .option("hdfs.krbAuthorization", "test") + .option("hdfs.krbPrincipalPattern", "test") + .option("hdfs.krbKdc", "test") + .option("hdfs.krbRealm", "test") + .option("hdfs.krbKeytabUser", "test") + .option("hdfs.krbKeytabPath", "test") + .option("hdfs.useKerberosAutorenewal", "true") + .option("hdfs.includeRecordEpochAndAfter", 0L) // record timestamps are in 1650872090806000 granularity (epochMicros) + .load(); + + Dataset df2 = df.agg(functions.count("*")); + + StreamingQuery streamingQuery = df2 + .writeStream() + .outputMode("complete") + .format("memory") + .trigger(Trigger.ProcessingTime(0)) + .queryName("MockArchiveQuery") + .option("checkpointLocation", "/tmp/checkpoint/" + UUID.randomUUID()) + .option("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + .start(); + + StreamingQuery sq = df.writeStream().foreachBatch((ds, i) -> { + ds.show(false); + }).start(); + sq.processAllAvailable(); + sq.stop(); + sq.awaitTermination(); + + long rowCount = 0; + while (!streamingQuery.awaitTermination(1000)) { + + long resultSize = spark.sqlContext().sql("SELECT * FROM MockArchiveQuery").count(); + if (resultSize > 0) { + rowCount = spark.sqlContext().sql("SELECT * FROM MockArchiveQuery").first().getAs(0); + System.out.println(rowCount); + } + if ( + streamingQuery.lastProgress() == null + || streamingQuery.status().message().equals("Initializing sources") + ) { + // query has not started + } + else if (streamingQuery.lastProgress().sources().length != 0) { + if (isArchiveDone(streamingQuery)) { + streamingQuery.stop(); + } + } + } + Assertions.assertEquals(expectedRows - MockKafkaConsumerFactory.getNumRecords(), rowCount); + } + + private boolean isArchiveDone(StreamingQuery outQ) { + Boolean archiveDone = true; + for (int i = 0; i < outQ.lastProgress().sources().length; i++) { + String startOffset = outQ.lastProgress().sources()[i].startOffset(); + String endOffset = outQ.lastProgress().sources()[i].endOffset(); + String description = outQ.lastProgress().sources()[i].description(); + + if (description != null && !description.startsWith("com.teragrep.pth_06.ArchiveMicroStreamReader@")) { + // ignore others than archive + continue; + } + + if (startOffset != null) { + if (!startOffset.equalsIgnoreCase(endOffset)) { + archiveDone = false; + } + } + else { + archiveDone = false; + } + } + return archiveDone; + } + + @AfterAll + public void decommissionEnv() throws Exception { + mockS3.stop(); + mockHDFS.teardownMiniCluster(); + } + + private long preloadS3Data() throws IOException { + long rows = 0L; + AmazonS3 amazonS3 = new Pth06S3Client(s3endpoint, s3identity, s3credential).build(); + + TreeMap>> virtualDatabaseMap = mockDBData + .getVirtualDatabaseMap(); + + for ( + Map.Entry>> entry : virtualDatabaseMap + .entrySet() + ) { + Iterator> it = entry + .getValue() + .iterator(); + while (it.hasNext()) { + // id, directory, stream, host, logtag, logdate, bucket, path, logtime, filesize + Record11 record10 = it + .next(); + Long id = record10.get(0, ULong.class).longValue(); + String directory = record10.get(1, String.class); + String stream = record10.get(2, String.class); + String host = record10.get(3, String.class); + String logtag = record10.get(4, String.class); + Date logdate = record10.get(5, Date.class); + String bucket = record10.get(6, String.class); + String path = record10.get(7, String.class); + Long logtime = record10.get(8, ULong.class).longValue(); + Long filesize = record10.get(9, ULong.class).longValue(); + + String recordAsJson = record10.formatJSON(); + + // <46>1 2010-01-01T12:34:56.123456+02:00 hostname.domain.tld pstats - - + SyslogMessage syslog = new SyslogMessage(); + syslog = syslog + .withFacility(Facility.USER) + .withSeverity(Severity.WARNING) + .withTimestamp(logtime) + .withHostname(host) + .withAppName(logtag) + .withMsg(recordAsJson); + + // [event_id@48577 hostname="hostname.domain.tld" uuid="" unixtime="" id_source="source"] + + SDElement event_id_48577 = new SDElement("event_id@48577") + .addSDParam("hostname", host) + .addSDParam("uuid", UUID.randomUUID().toString()) + .addSDParam("source", "source") + .addSDParam("unixtime", Long.toString(System.currentTimeMillis())); + + syslog = syslog.withSDElement(event_id_48577); + + // [event_format@48577 original_format="rfc5424"] + + SDElement event_format_48577 = new SDElement("event_id@48577").addSDParam("original_format", "rfc5424"); + + syslog = syslog.withSDElement(event_format_48577); + + // [event_node_relay@48577 hostname="relay.domain.tld" source="hostname.domain.tld" source_module="imudp"] + + SDElement event_node_relay_48577 = new SDElement("event_node_relay@48577") + .addSDParam("hostname", "relay.domain.tld") + .addSDParam("source", host) + .addSDParam("source_module", "imudp"); + + syslog = syslog.withSDElement(event_node_relay_48577); + + // [event_version@48577 major="2" minor="2" hostname="relay.domain.tld" version_source="relay"] + + SDElement event_version_48577 = new SDElement("event_version@48577") + .addSDParam("major", "2") + .addSDParam("minor", "2") + .addSDParam("hostname", "relay.domain.tld") + .addSDParam("version_source", "relay"); + + syslog = syslog.withSDElement(event_version_48577); + + // [event_node_router@48577 source="relay.domain.tld" source_module="imrelp" hostname="router.domain.tld"] + + SDElement event_node_router_48577 = new SDElement("event_node_router@48577") + .addSDParam("source", "relay.domain.tld") + .addSDParam("source_module", "imrelp") + .addSDParam("hostname", "router.domain.tld"); + + syslog = syslog.withSDElement(event_node_router_48577); + + // [origin@48577 hostname="original.hostname.domain.tld"] + + SDElement origin_48577 = new SDElement("origin@48577") + .addSDParam("hostname", "original.hostname.domain.tld"); + syslog = syslog.withSDElement(origin_48577); + + // check if this bucket exists + boolean bucketExists = false; + for (Bucket existingBucket : amazonS3.listBuckets()) { + if (existingBucket.getName().equals(bucket)) { + bucketExists = true; + break; + } + } + if (!bucketExists) { + amazonS3.createBucket(bucket); + } + + // compress the message + String syslogMessage = syslog.toRfc5424SyslogMessage(); + + ByteArrayOutputStream outStream = new ByteArrayOutputStream(syslogMessage.length()); + GZIPOutputStream gzip = new GZIPOutputStream(outStream); + gzip.write(syslogMessage.getBytes()); + gzip.close(); + + ByteArrayInputStream inStream = new ByteArrayInputStream(outStream.toByteArray()); + + // upload as file + amazonS3.putObject(bucket, path, inStream, null); + rows++; + } + + } + return rows; + } +} diff --git a/src/test/java/com/teragrep/pth_06/InstantiationTest.java b/src/test/java/com/teragrep/pth_06/InstantiationTest.java index 971a910b..9765cd81 100644 --- a/src/test/java/com/teragrep/pth_06/InstantiationTest.java +++ b/src/test/java/com/teragrep/pth_06/InstantiationTest.java @@ -54,6 +54,7 @@ import com.google.gson.Gson; import com.google.gson.JsonObject; import com.teragrep.pth_06.planner.MockDBData; +import com.teragrep.pth_06.planner.MockHDFS; import com.teragrep.pth_06.planner.MockKafkaConsumerFactory; import com.teragrep.pth_06.task.s3.MockS3; import com.teragrep.pth_06.task.s3.Pth06S3Client; @@ -96,6 +97,10 @@ public class InstantiationTest { private long expectedRows = 0L; + private final String hdfsPath = "hdfs:///opt/teragrep/cfe_39/srv/"; + private String hdfsUri; // Can only be defined after starting the mock hdfs. + private final MockHDFS mockHDFS = new MockHDFS(hdfsPath); + @BeforeAll public void prepareEnv() throws Exception { //Logger.getRootLogger().setLevel(Level.ERROR); @@ -104,6 +109,9 @@ public void prepareEnv() throws Exception { mockS3.start(); + // Start mock hdfs with 10 out of 14 records from kafka inserted inside. + hdfsUri = mockHDFS.startMiniCluster(false); + spark = SparkSession .builder() .appName("Java Spark SQL basic example") @@ -116,6 +124,7 @@ public void prepareEnv() throws Exception { //spark.sparkContext().setLogLevel("ERROR"); expectedRows = preloadS3Data() + MockKafkaConsumerFactory.getNumRecords(); + /*File 0.9 is stored to HDFS, 10 records out of 14 will be read from HDFS instead of Kafka.*/ } @Test @@ -134,7 +143,7 @@ public void fullScanTest() throws StreamingQueryException, TimeoutException { .option("DBstreamdbname", "mock") .option("DBjournaldbname", "mock") .option("num_partitions", "1") - .option("queryXML", "") + .option("queryXML", "") // Only affects HDFS execution in the current test configuration. // audit information .option("TeragrepAuditQuery", "index=f17") .option("TeragrepAuditReason", "test run at fullScanTest()") @@ -147,6 +156,23 @@ public void fullScanTest() throws StreamingQueryException, TimeoutException { .option("kafka.sasl.jaas.config", "") .option("kafka.useMockKafkaConsumer", "true") .option("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + // HDFS options + .option("hdfs.enabled", "true") + .option("hdfs.hdfsPath", hdfsPath) + .option("hdfs.hdfsUri", hdfsUri) + .option("hdfs.useHostName", "false") + .option("hdfs.transferProtection", "test") + .option("hdfs.cipherSuites", "test") + .option("hdfs.useMockHdfsDatabase", "true") + .option("hdfs.krbAuthentication", "false") + .option("hdfs.krbAuthorization", "test") + .option("hdfs.krbPrincipalPattern", "test") + .option("hdfs.krbKdc", "test") + .option("hdfs.krbRealm", "test") + .option("hdfs.krbKeytabUser", "test") + .option("hdfs.krbKeytabPath", "test") + .option("hdfs.useKerberosAutorenewal", "true") + .option("hdfs.includeRecordEpochAndAfter", 0L) // record timestamps are in 1650872090806000 granularity (epochMicros) .load(); Dataset df2 = df.agg(functions.count("*")); @@ -272,6 +298,7 @@ private boolean isArchiveDone(StreamingQuery outQ) { @AfterAll public void decommissionEnv() throws Exception { mockS3.stop(); + mockHDFS.teardownMiniCluster(); } private long preloadS3Data() throws IOException { diff --git a/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java b/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java index ac38bc0a..8289bd1c 100644 --- a/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java +++ b/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java @@ -126,6 +126,14 @@ public StructType readSchema() { public MicroBatchStream toMicroBatchStream(String checkpointLocation) { Config config = new Config(options); + HdfsQuery hdfsQueryProcessor; + if (config.isHdfsEnabled) { + hdfsQueryProcessor = new HdfsQueryProcessor(config); + } + else { + hdfsQueryProcessor = new HdfsQueryProcessor(); + } + ArchiveQuery archiveQueryProcessor = new MockArchiveQueryProcessor( "" ); @@ -140,7 +148,12 @@ public MicroBatchStream toMicroBatchStream(String checkpointLocation) { kafkaQueryProcessor = null; } - return new ArchiveMicroStreamReader(archiveQueryProcessor, kafkaQueryProcessor, config); + return new ArchiveMicroStreamReader( + hdfsQueryProcessor, + archiveQueryProcessor, + kafkaQueryProcessor, + config + ); } }; } diff --git a/src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.13 b/src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.13 new file mode 100644 index 00000000..553f5957 Binary files /dev/null and b/src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.13 differ diff --git a/src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.9 b/src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.9 new file mode 100644 index 00000000..c98cda54 Binary files /dev/null and b/src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.9 differ diff --git a/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java b/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java index 0f329712..1d6625b8 100644 --- a/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java +++ b/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java @@ -46,6 +46,7 @@ package com.teragrep.pth_06.planner; import com.teragrep.pth_06.planner.offset.DatasourceOffset; +import com.teragrep.pth_06.planner.offset.HdfsOffset; import com.teragrep.pth_06.planner.offset.KafkaOffset; import org.apache.kafka.common.TopicPartition; import org.apache.spark.sql.execution.streaming.LongOffset; @@ -63,6 +64,7 @@ public void serdeTest() { Map topicPartitionLongMap = new HashMap<>(); topicPartitionLongMap.put(new TopicPartition("test", 0), 0L); KafkaOffset kafkaOffset = new KafkaOffset(topicPartitionLongMap); + kafkaOffset.serialize(); DatasourceOffset datasourceOffset = new DatasourceOffset(longOffset, kafkaOffset); String ser = datasourceOffset.json(); @@ -77,6 +79,7 @@ public void kafkaOffsetSerdeTest() { Map topicPartitionLongMap = new HashMap<>(); topicPartitionLongMap.put(new TopicPartition("test", 777), 9999L); KafkaOffset kafkaOffset = new KafkaOffset(topicPartitionLongMap); + kafkaOffset.serialize(); String ser = kafkaOffset.json(); KafkaOffset deser = new KafkaOffset(ser); @@ -90,4 +93,24 @@ public void kafkaOffsetSerdeTest() { Assertions.assertEquals(9999L, offset); } } + + @Test + public void HdfsOffsetSerdeTest() { + Map topicPartitionLongMap = new HashMap<>(); + topicPartitionLongMap.put(new TopicPartition("test", 777), 9999L); + HdfsOffset hdfsOffset = new HdfsOffset(topicPartitionLongMap); + hdfsOffset.serialize(); + + String ser = hdfsOffset.json(); + HdfsOffset deser = new HdfsOffset(ser); + + for (Map.Entry entry : deser.getOffsetMap().entrySet()) { + TopicPartition topicPartition = entry.getKey(); + long offset = entry.getValue(); + + Assertions.assertEquals("test", topicPartition.topic()); + Assertions.assertEquals(777, topicPartition.partition()); + Assertions.assertEquals(9999L, offset); + } + } } diff --git a/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java b/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java new file mode 100644 index 00000000..fff53799 --- /dev/null +++ b/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java @@ -0,0 +1,158 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.planner; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.nio.file.Files; +import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public final class MockHDFS { + + private static final Logger LOGGER = LoggerFactory.getLogger(MockHDFS.class); + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static String hdfsURI; + private static String hdfsPath; + + public MockHDFS(String hdfsPath) { + MockHDFS.hdfsPath = hdfsPath; // "hdfs:///opt/teragrep/cfe_39/srv/" + } + + // Start minicluster and initialize config. Returns the hdfsUri of the minicluster. + public String startMiniCluster(boolean insertAll) throws IOException, InterruptedException { + + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + // System.out.println("hdfsURI: " + hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + + insertMockFiles(insertAll); + + return hdfsURI; + } + + // Teardown the minicluster + public void teardownMiniCluster() { + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + public static void insertMockFiles(boolean insertAll) throws IOException { + String path = hdfsPath + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsURI); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + //Get the filesystem - HDFS + FileSystem fs = FileSystem.get(URI.create(hdfsURI), conf); + + //==== Create directory if not exists + Path workingDir = fs.getWorkingDirectory(); + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + } + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/pth_06/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + if (fs.exists(hdfswritepath)) { + throw new RuntimeException("File " + avroFile.getName() + " already exists"); + } + Path readPath = new Path(avroFile.getPath()); + /* This condition can be used for managing the test offsets between HDFS and Kafka. If 0.13 file is included in the HDFS, then Kafka won't have any new records available for reading. + If 0.13 is excluded, then records from 0 to 8 will be read from HDFS and records from 9 to 13 will be read from Kafka. + Implement a method to decide if all the files should be added to HDFS or not*/ + if (insertAll) { + fs.copyFromLocalFile(readPath, hdfswritepath); + } + else { + if (!avroFile.getName().equals("0.13")) { + fs.copyFromLocalFile(readPath, hdfswritepath); + } + } + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); + } + fs.close(); + } +} diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java new file mode 100644 index 00000000..ba53a1ac --- /dev/null +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java @@ -0,0 +1,134 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.task.hdfs; + +import com.teragrep.pth_06.HdfsFileMetadata; +import com.teragrep.pth_06.avro.SyslogRecord; +import com.teragrep.pth_06.planner.MockHDFS; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.net.URI; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class AvroReadImplTest { + + private final String hdfsPath = "hdfs:///opt/teragrep/cfe_39/srv/"; + private String hdfsUri; // Can only be defined after starting the mock hdfs. + private final MockHDFS mockHDFS = new MockHDFS(hdfsPath); + + @BeforeEach + public void setUp() throws IOException, InterruptedException { + hdfsUri = mockHDFS.startMiniCluster(true); + } + + @AfterEach + public void teardown() { + mockHDFS.teardownMiniCluster(); + } + + @Test + public void testReadAllRecords() { + assertDoesNotThrow(() -> { + + // build FileSystem object + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsUri); + // Because of Maven + conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", LocalFileSystem.class.getName()); + // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + // filesystem for HDFS access is set here + FileSystem fs = FileSystem.get(URI.create(hdfsUri), conf); + + // build hdfsFileMetadata object + HdfsFileMetadata testConsumerTopic09 = new HdfsFileMetadata( + new TopicPartition("testConsumerTopic", 0), + 9, + hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", + 0 + ); + AvroReadImpl avroReadImpl1 = new AvroReadImpl(fs, testConsumerTopic09); + long rowNum = 0L; + while (avroReadImpl1.next()) { + SyslogRecord syslogRecord = avroReadImpl1.record(); + Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 0 to 9. + rowNum++; + } + avroReadImpl1.close(); + Assertions.assertEquals(10, rowNum); // Asserts that expected number of records were consumed from the files. + + HdfsFileMetadata testConsumerTopic013 = new HdfsFileMetadata( + new TopicPartition("testConsumerTopic", 0), + 13, + hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", + 0 + ); + AvroReadImpl avroReadImpl2 = new AvroReadImpl(fs, testConsumerTopic013); + while (avroReadImpl2.next()) { + SyslogRecord syslogRecord = avroReadImpl2.record(); + Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 10 to 13. + rowNum++; + } + avroReadImpl2.close(); + Assertions.assertEquals(14, rowNum); // Asserts that expected number of records were consumed from the files. + }); + } + +} diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java new file mode 100644 index 00000000..e2afd265 --- /dev/null +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java @@ -0,0 +1,239 @@ +/* + * Teragrep Archive Datasource (pth_06) + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.pth_06.task.hdfs; + +import com.teragrep.pth_06.HdfsFileMetadata; +import com.teragrep.pth_06.planner.MockHDFS; +import com.teragrep.pth_06.task.HdfsMicroBatchInputPartitionReader; +import org.apache.kafka.common.TopicPartition; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.*; + +import java.io.IOException; +import java.util.LinkedList; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class HdfsMicroBatchInputPartitionReaderTest { + + private final String hdfsPath = "hdfs:///opt/teragrep/cfe_39/srv/"; + private String hdfsUri; // Can only be defined after starting the mock hdfs. + private final MockHDFS mockHDFS = new MockHDFS(hdfsPath); + + @BeforeEach + public void setUp() throws IOException, InterruptedException { + hdfsUri = mockHDFS.startMiniCluster(true); + } + + @AfterEach + public void teardown() { + mockHDFS.teardownMiniCluster(); + } + + @Test + public void testHdfsConsumer2Files() { + assertDoesNotThrow(() -> { + // create task object list + LinkedList taskObjectList = new LinkedList<>(); + // Add taskObjects to the taskObjectList according to what files are stored in minicluster during setup. + taskObjectList + .add(new HdfsFileMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); + taskObjectList + .add(new HdfsFileMetadata(new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0)); + + HdfsMicroBatchInputPartitionReader hdfsMicroBatchInputPartitionReader = new HdfsMicroBatchInputPartitionReader( + 0L, + "", + hdfsUri, + hdfsPath, + "", + "", + "", + "", + "", + "", + "", + "", + "", + "", + taskObjectList + ); + + // read through the files in HDFS + long rowNum = 0L; + while (hdfsMicroBatchInputPartitionReader.next()) { + InternalRow internalRow = hdfsMicroBatchInputPartitionReader.get(); + Assertions.assertEquals(rowNum, internalRow.getLong(7)); // Checks offsets of the consumed records which should range from 0 to 13. + rowNum++; + } + hdfsMicroBatchInputPartitionReader.close(); + Assertions.assertEquals(14, rowNum); // Asserts that expected number of records were consumed from the files. + }); + + } + + @Test + public void testHdfsConsumer1File() { + assertDoesNotThrow(() -> { + // create task object list + LinkedList taskObjectList = new LinkedList<>(); + // Add only the taskObject related to testConsumerTopic/0.9 file to the taskObjectList. + taskObjectList + .add(new HdfsFileMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); + + HdfsMicroBatchInputPartitionReader hdfsMicroBatchInputPartitionReader = new HdfsMicroBatchInputPartitionReader( + 0L, + "", + hdfsUri, + hdfsPath, + "", + "", + "", + "", + "", + "", + "", + "", + "", + "", + taskObjectList + ); + + // read through the files in HDFS + long rowNum = 0L; + while (hdfsMicroBatchInputPartitionReader.next()) { + InternalRow internalRow = hdfsMicroBatchInputPartitionReader.get(); + Assertions.assertEquals(rowNum, internalRow.getLong(7)); // Checks offsets of the consumed records which should range from 0 to 9. + rowNum++; + } + hdfsMicroBatchInputPartitionReader.close(); + Assertions.assertEquals(10, rowNum); // Asserts that expected number of records were consumed from the file. + }); + + } + + @Test + public void testHdfsConsumer1FileAlt() { + assertDoesNotThrow(() -> { + // create task object list + LinkedList taskObjectList = new LinkedList<>(); + // Add only the taskObject related to testConsumerTopic/0.13 file to the taskObjectList. + taskObjectList + .add(new HdfsFileMetadata(new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0)); + + HdfsMicroBatchInputPartitionReader hdfsMicroBatchInputPartitionReader = new HdfsMicroBatchInputPartitionReader( + 0L, + "", + hdfsUri, + hdfsPath, + "", + "", + "", + "", + "", + "", + "", + "", + "", + "", + taskObjectList + ); + + // read through the files in HDFS + long rowNum = 10L; + while (hdfsMicroBatchInputPartitionReader.next()) { + InternalRow internalRow = hdfsMicroBatchInputPartitionReader.get(); + Assertions.assertEquals(rowNum, internalRow.getLong(7)); // Checks offsets of the consumed records which should range from 10 to 13. + rowNum++; + } + hdfsMicroBatchInputPartitionReader.close(); + Assertions.assertEquals(14, rowNum); + }); + + } + + @Test + public void testCutoffEpoch() { + assertDoesNotThrow(() -> { + // create task object list + LinkedList taskObjectList = new LinkedList<>(); + // Add taskObjects to the taskObjectList according to what files are stored in minicluster during setup. + taskObjectList + .add(new HdfsFileMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); + taskObjectList + .add(new HdfsFileMetadata(new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0)); + + HdfsMicroBatchInputPartitionReader hdfsMicroBatchInputPartitionReader = new HdfsMicroBatchInputPartitionReader( + 1650872090804001L, // Offset 0 has timestamp of 1650872090804000L + "", + hdfsUri, + hdfsPath, + "", + "", + "", + "", + "", + "", + "", + "", + "", + "", + taskObjectList + ); + + // read through the files in HDFS + long rowNum = 1L; // Offset 0 has timestamp of 1650872090804000L, which means it is filtered out by the time based inclusion. + while (hdfsMicroBatchInputPartitionReader.next()) { + InternalRow internalRow = hdfsMicroBatchInputPartitionReader.get(); + Assertions.assertEquals(rowNum, internalRow.getLong(7)); // Checks offsets of the consumed records which should range from 1 to 13. + rowNum++; + } + hdfsMicroBatchInputPartitionReader.close(); + Assertions.assertEquals(14, rowNum); + }); + } + +}