From faf29558c073239469e57470fd3463c6b6a8c8ce Mon Sep 17 00:00:00 2001 From: lifengchao Date: Wed, 29 May 2024 17:09:27 +0800 Subject: [feature][connector-clickhouse] TSG-21409 clickhouse sink buffer优化, 解决可能oom问题 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../clickhouse/ClickHouseConnectorOptions.java | 7 + .../clickhouse/ClickHouseTableFactory.java | 8 +- .../connectors/clickhouse/buffer/BufferPool.java | 105 ++++++++++++ .../clickhouse/buffer/ReusedByteArrayWriter.java | 84 ++++++++++ .../buffer/ReusedColumnWriterBuffer.java | 64 +++++++ .../ClickHousePreparedBatchInsertStatement.java | 2 +- .../sink/AbstractBatchIntervalClickHouseSink.java | 185 ++++++++++++++++----- .../sink/EventBatchIntervalClickHouseSink.java | 15 +- .../clickhouse/util/BlockColumnsByteSizeInfo.java | 13 ++ .../clickhouse/util/ClickHouseUtils.java | 152 +++++++++++++---- 10 files changed, 554 insertions(+), 81 deletions(-) create mode 100644 groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/buffer/BufferPool.java create mode 100644 groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/buffer/ReusedByteArrayWriter.java create mode 100644 groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/buffer/ReusedColumnWriterBuffer.java create mode 100644 groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/util/BlockColumnsByteSizeInfo.java diff --git a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/ClickHouseConnectorOptions.java b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/ClickHouseConnectorOptions.java index 28ac94d..ee82603 100644 --- a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/ClickHouseConnectorOptions.java +++ b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/ClickHouseConnectorOptions.java @@ -2,6 +2,7 @@ package com.geedgenetworks.connectors.clickhouse; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.MemorySize; import java.time.Duration; @@ -26,6 +27,12 @@ public class ClickHouseConnectorOptions { .defaultValue(100000) .withDescription("The flush max size , over this number of records, will flush data."); + public static final ConfigOption BATCH_BYTE_SIZE = + ConfigOptions.key("batch.byte.size") + .memoryType() + .defaultValue(MemorySize.parse("200mb")) + .withDescription("The flush max buffer byte size , over this byte size, will flush data."); + public static final ConfigOption BATCH_INTERVAL = ConfigOptions.key("batch.interval") .durationType() diff --git a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/ClickHouseTableFactory.java b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/ClickHouseTableFactory.java index 96a73b9..441bc00 100644 --- a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/ClickHouseTableFactory.java +++ b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/ClickHouseTableFactory.java @@ -8,10 +8,12 @@ import com.geedgenetworks.core.factories.FactoryUtil.TableFactoryHelper; import com.geedgenetworks.core.factories.SinkTableFactory; import com.geedgenetworks.common.Event; import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.util.Preconditions; import java.util.HashSet; import java.util.Properties; @@ -39,10 +41,13 @@ public class ClickHouseTableFactory implements SinkTableFactory { String host = config.get(HOST); String table = config.get(TABLE); int batchSize = config.get(BATCH_SIZE); + MemorySize batchByteMemorySize = config.get(BATCH_BYTE_SIZE); + Preconditions.checkArgument(batchByteMemorySize.getMebiBytes() < 1000, "batch.byte.size can not bigger than 1000m"); + int batchByteSize = (int) batchByteMemorySize.getBytes(); long batchIntervalMs = config.get(BATCH_INTERVAL).toMillis(); Properties connInfo = getClickHouseConnInfo(context.getOptions()); - final SinkFunction sinkFunction = new EventBatchIntervalClickHouseSink(schema, batchSize, batchIntervalMs, host, table, connInfo); + final SinkFunction sinkFunction = new EventBatchIntervalClickHouseSink(schema, batchSize, batchByteSize, batchIntervalMs, host, table, connInfo); return new SinkProvider() { @Override @@ -71,6 +76,7 @@ public class ClickHouseTableFactory implements SinkTableFactory { public Set> optionalOptions() { final Set> options = new HashSet<>(); options.add(BATCH_SIZE); + options.add(BATCH_BYTE_SIZE); options.add(BATCH_INTERVAL); return options; } diff --git a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/buffer/BufferPool.java b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/buffer/BufferPool.java new file mode 100644 index 0000000..61f702d --- /dev/null +++ b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/buffer/BufferPool.java @@ -0,0 +1,105 @@ +package com.geedgenetworks.connectors.clickhouse.buffer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.Buffer; +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.concurrent.locks.ReentrantLock; + +public class BufferPool { + static final Logger LOG = LoggerFactory.getLogger(BufferPool.class); + private final ArrayDeque free; + private final ReentrantLock lock; + private final long minCacheSize; + private final long maxCacheSize; + private final long keepAliveTimeMillis; + private final long clearIntervalMs; + private long lastClearTs; + private long currentCacheSize; + private long lastLogTs; + + public BufferPool(long minCacheSize, long maxCacheSize, long keepAliveTimeMillis) { + this.free = new ArrayDeque<>(); + this.lock = new ReentrantLock(); + this.minCacheSize = minCacheSize; + this.maxCacheSize = maxCacheSize; + this.keepAliveTimeMillis = keepAliveTimeMillis; + this.clearIntervalMs = Math.max(keepAliveTimeMillis / 10, 60000); + this.lastClearTs = System.currentTimeMillis(); + this.currentCacheSize = 0; + this.lastLogTs = System.currentTimeMillis(); + } + + ByteBuffer allocate(int size) { + lock.lock(); + try { + if (!free.isEmpty()) { + ByteBuffer buffer = free.pollFirst().buffer; + currentCacheSize -= buffer.capacity(); + return buffer; + } + } finally { + lock.unlock(); + } + return ByteBuffer.allocate(size); + } + + void deallocate(ByteBuffer buffer) { + lock.lock(); + try { + if (currentCacheSize + buffer.capacity() <= maxCacheSize) { + ((Buffer) buffer).clear(); + free.addFirst(new ByteBufferWithTs(buffer, System.currentTimeMillis())); + currentCacheSize += buffer.capacity(); + } else { + // 直接回收掉 + } + } finally { + lock.unlock(); + } + + clearExpiredBuffer(); + } + + public long getCurrentCacheSize() { + return currentCacheSize; + } + + private void clearExpiredBuffer() { + long ts = System.currentTimeMillis(); + + if(ts - lastLogTs > 300000){ + LOG.warn("currentCacheSize: " + (getCurrentCacheSize() >>> 20) + "M"); + lastLogTs = ts; + } + + if (ts - lastClearTs < clearIntervalMs) { + return; + } + lastClearTs = ts; + + lock.lock(); + try { + ByteBufferWithTs ele = free.peekLast(); + while (ele != null && currentCacheSize - ele.buffer.capacity() >= minCacheSize && ts - ele.ts > keepAliveTimeMillis) { + free.pollLast(); + currentCacheSize -= ele.buffer.capacity(); + ele = free.peekLast(); + } + } finally { + lock.unlock(); + } + } + + static class ByteBufferWithTs { + ByteBuffer buffer; + long ts; + + public ByteBufferWithTs(ByteBuffer buffer, long ts) { + this.buffer = buffer; + this.ts = ts; + } + } +} diff --git a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/buffer/ReusedByteArrayWriter.java b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/buffer/ReusedByteArrayWriter.java new file mode 100644 index 0000000..31ab724 --- /dev/null +++ b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/buffer/ReusedByteArrayWriter.java @@ -0,0 +1,84 @@ +package com.geedgenetworks.connectors.clickhouse.buffer; + +import com.github.housepower.buffer.BuffedWriter; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.LinkedList; +import java.util.List; + +public class ReusedByteArrayWriter implements BuffedWriter { + private final int blockSize; + private final BufferPool bufferPool; + private ByteBuffer buffer; + + private final List byteBufferList = new LinkedList<>(); + + public ReusedByteArrayWriter(int blockSize, BufferPool bufferPool) { + this.blockSize = blockSize; + this.bufferPool = bufferPool; + reuseOrAllocateByteBuffer(); + } + + @Override + public void writeBinary(byte byt) throws IOException { + buffer.put(byt); + flushToTarget(false); + } + + @Override + public void writeBinary(byte[] bytes) throws IOException { + writeBinary(bytes, 0, bytes.length); + } + + @Override + public void writeBinary(byte[] bytes, int offset, int length) throws IOException { + + while (buffer.remaining() < length) { + int num = buffer.remaining(); + buffer.put(bytes, offset, num); + flushToTarget(true); + + offset += num; + length -= num; + } + + buffer.put(bytes, offset, length); + flushToTarget(false); + } + + @Override + public void flushToTarget(boolean force) throws IOException { + if (buffer.hasRemaining() && !force) { + return; + } + reuseOrAllocateByteBuffer(); + } + + public List getBufferList() { + return byteBufferList; + } + + public void reset() { + byteBufferList.forEach(b -> { + // upcast is necessary, see detail at: + // https://bitbucket.org/ijabz/jaudiotagger/issues/313/java-8-javalangnosuchmethoderror + // ((Buffer) b).clear(); + bufferPool.deallocate(b); + }); + byteBufferList.clear(); + + reuseOrAllocateByteBuffer(); + } + + private ByteBuffer reuseOrAllocateByteBuffer() { + ByteBuffer newBuffer = bufferPool.allocate(blockSize); + if (newBuffer == null) { + newBuffer = ByteBuffer.allocate(blockSize); + } + + buffer = newBuffer; + byteBufferList.add(buffer); + return buffer; + } +} diff --git a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/buffer/ReusedColumnWriterBuffer.java b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/buffer/ReusedColumnWriterBuffer.java new file mode 100644 index 0000000..0f84aac --- /dev/null +++ b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/buffer/ReusedColumnWriterBuffer.java @@ -0,0 +1,64 @@ +package com.geedgenetworks.connectors.clickhouse.buffer; + +import com.github.housepower.data.ColumnWriterBuffer; +import com.github.housepower.serde.BinarySerializer; +import com.github.housepower.settings.ClickHouseDefines; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.Buffer; +import java.nio.ByteBuffer; +import java.util.List; + +import static java.lang.Math.min; + +public class ReusedColumnWriterBuffer extends ColumnWriterBuffer { + private static Field columnWriterField; + private final ReusedByteArrayWriter reusedColumnWriter; + + public ReusedColumnWriterBuffer(BufferPool bufferPool) { + super(); + try { + columnWriterField.set(this, null); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + this.reusedColumnWriter = new ReusedByteArrayWriter(ClickHouseDefines.COLUMN_BUFFER_BYTES, bufferPool); + this.column = new BinarySerializer(reusedColumnWriter, false); + } + + static { + try { + columnWriterField = ColumnWriterBuffer.class.getDeclaredField("columnWriter"); + columnWriterField.setAccessible(true); + } catch (NoSuchFieldException e) { + e.printStackTrace(); + } + } + + @Override + public void writeTo(BinarySerializer serializer) throws IOException { + // add a temp buffer to reduce memory requirement in case of large remaining data in buffer + byte[] writeBuffer = new byte[4*1024]; + for (ByteBuffer buffer : reusedColumnWriter.getBufferList()) { + // upcast is necessary, see detail at: + // https://bitbucket.org/ijabz/jaudiotagger/issues/313/java-8-javalangnosuchmethoderror + ((Buffer) buffer).flip(); + while (buffer.hasRemaining()) { + int remaining = buffer.remaining(); + int thisLength = min(remaining, writeBuffer.length); + buffer.get(writeBuffer, 0, thisLength); + serializer.writeBytes(writeBuffer, 0, thisLength); + } + } + } + + @Override + public void reset() { + reusedColumnWriter.reset(); + } + + public List getBufferList() { + return reusedColumnWriter.getBufferList(); + } +} diff --git a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/jdbc/ClickHousePreparedBatchInsertStatement.java b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/jdbc/ClickHousePreparedBatchInsertStatement.java index 65cbb96..d632c94 100644 --- a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/jdbc/ClickHousePreparedBatchInsertStatement.java +++ b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/jdbc/ClickHousePreparedBatchInsertStatement.java @@ -159,7 +159,7 @@ public class ClickHousePreparedBatchInsertStatement implements SQLPreparedStatem // this.block.initWriteBuffer(); } // clean up block on close - this.block.cleanup(); + // this.block.cleanup(); this.isClosed = true; } diff --git a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/sink/AbstractBatchIntervalClickHouseSink.java b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/sink/AbstractBatchIntervalClickHouseSink.java index 72fba40..6761d42 100644 --- a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/sink/AbstractBatchIntervalClickHouseSink.java +++ b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/sink/AbstractBatchIntervalClickHouseSink.java @@ -1,10 +1,12 @@ package com.geedgenetworks.connectors.clickhouse.sink; import com.alibaba.fastjson2.JSON; +import com.geedgenetworks.connectors.clickhouse.buffer.BufferPool; import com.geedgenetworks.connectors.clickhouse.jdbc.BytesCharVarSeq; import com.geedgenetworks.connectors.clickhouse.jdbc.ClickHouseBatchInsertConnection; import com.geedgenetworks.connectors.clickhouse.jdbc.ClickHousePreparedBatchInsertStatement; import com.geedgenetworks.connectors.clickhouse.jdbc.DataTypeStringV2; +import com.geedgenetworks.connectors.clickhouse.util.BlockColumnsByteSizeInfo; import com.geedgenetworks.connectors.clickhouse.util.ClickHouseUtils; import com.geedgenetworks.core.metrics.InternalMetrics; import com.github.housepower.data.*; @@ -15,7 +17,7 @@ import com.github.housepower.jdbc.ClickHouseArray; import com.github.housepower.misc.BytesCharSeq; import com.github.housepower.misc.DateTimeUtil; import com.github.housepower.settings.SettingKey; -import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; @@ -42,6 +44,8 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; import java.util.stream.IntStream; +import static com.geedgenetworks.connectors.clickhouse.util.ClickHouseUtils.writeBytesSizeByLen; + public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFunction implements CheckpointedFunction { static final Logger LOG = LoggerFactory.getLogger(AbstractBatchIntervalClickHouseSink.class); @@ -52,15 +56,18 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun // 标准日期时间格式,精确到毫秒:yyyy-MM-dd HH:mm:ss.SSS public static final String NORM_DATETIME_MS_PATTERN = "yyyy-MM-dd HH:mm:ss.SSS"; public static final DateTimeFormatter NORM_DATETIME_MS_FORMATTER = DateTimeFormatter.ofPattern(NORM_DATETIME_MS_PATTERN); - private int batchSize; - private long batchIntervalMs; + private final int batchSize; + private final int batchByteSize; + private final long batchIntervalMs; private transient volatile boolean closed; private transient InternalMetrics internalMetrics; private transient Thread outThread; private transient ReentrantLock lock; private transient Block batch; + private int batchWriteByteSize; private transient BlockingQueue outBatchQueue; private transient BlockingQueue freeBatchQueue; + private transient BufferPool bufferPool; private transient Exception flushException; private transient long lastFlushTs; // flush ck 相关 @@ -72,13 +79,16 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun protected ZoneId tz; protected String[] columnNames; protected Object[] columnDefaultValues; + protected int[] columnDefaultSizes; protected IDataType[] columnTypes; protected ValueConverter[] columnConverters; + protected final SizeHelper writeSizeHelper; - public AbstractBatchIntervalClickHouseSink( - int batchSize, long batchIntervalMs, String host, String table, Properties connInfo) { + public AbstractBatchIntervalClickHouseSink(int batchSize, int batchByteSize, long batchIntervalMs, String host, String table, Properties connInfo) { this.batchSize = batchSize; + this.batchByteSize = batchByteSize; this.batchIntervalMs = batchIntervalMs; + this.writeSizeHelper = new SizeHelper(); this.urls = ClickHouseUtils.buildUrlsFromHost(host); this.table = table; this.connInfo = connInfo; @@ -142,23 +152,26 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun urlIndex = getRuntimeContext().getIndexOfThisSubtask() % urls.length; // 获取要插入的列信息 - Tuple2 columnsAndDefaultValues = ClickHouseUtils.getInsertColumnsAndDefaultValuesForTable( urls, urlIndex, connInfo, table); - columnNames = columnsAndDefaultValues.f0; - columnDefaultValues = columnsAndDefaultValues.f1; + Tuple3 columnsAndDefaultValuesAndDefaultSizes = ClickHouseUtils.getInsertColumnsAndDefaultValuesAndDefaultSizesForTable( urls, urlIndex, connInfo, table); + columnNames = columnsAndDefaultValuesAndDefaultSizes.f0; + columnDefaultValues = columnsAndDefaultValuesAndDefaultSizes.f1; + columnDefaultSizes = columnsAndDefaultValuesAndDefaultSizes.f2; insertSql = ClickHouseUtils.genePreparedInsertSql(this.table, this.columnNames); LOG.warn("insertColumnsCount:" + columnNames.length); LOG.warn("insertColumns:" + String.join(",", columnNames)); - LOG.warn("insertColumnDefaultValues:" + IntStream.range(0, columnNames.length).mapToObj(i -> columnNames[i] + ":" + columnDefaultValues[i]).collect(Collectors.joining(","))); + LOG.warn("insertColumnDefaultValuesAndSizes:" + IntStream.range(0, columnNames.length).mapToObj(i -> columnNames[i] + ":" + columnDefaultValues[i] + "(" + columnDefaultSizes[i] + ")").collect(Collectors.joining(","))); LOG.warn("insertSql:" + insertSql); // 获取时区用于解析DateTime类型 tz = ClickHouseUtils.chooseTimeZone(urls, urlIndex, connInfo); + bufferPool = new BufferPool(0, 1024L * 1024 * Math.max(columnNames.length * 2, 200), 1000 * 60 * 20); + // 初始化Block Block block = ClickHouseUtils.getInsertBlockForSql(urls, urlIndex, connInfo, insertSql); assert block.columnCnt() == columnNames.length; - block.initWriteBuffer(); + ClickHouseUtils.initBlockWriteBuffer(block, bufferPool); // block.initWriteBuffer(); freeBatchQueue.put(block); Field typeField = AbstractColumn.class.getDeclaredField("type"); @@ -190,7 +203,7 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun // 从block复制block block = ClickHouseUtils.newInsertBlockFrom(block); - block.initWriteBuffer(); + ClickHouseUtils.initBlockWriteBuffer(block, bufferPool); // block.initWriteBuffer(); freeBatchQueue.put(block); batch = freeBatchQueue.take(); } @@ -202,7 +215,7 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun void onInit(Configuration parameters) throws Exception {} void onClose() throws Exception {} - abstract boolean addBatch(Block batch, T data) throws Exception; + abstract int addBatch(Block batch, T data) throws Exception; @Override public final void invoke(T value, Context context) throws Exception { @@ -211,10 +224,12 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun lock.lock(); try { - if (addBatch(batch, value)) { + int writeSize = addBatch(batch, value); + if (writeSize > 0) { batch.appendRow(); + batchWriteByteSize += writeSize; } - if (batch.rowCnt() >= batchSize) { + if (batch.rowCnt() >= batchSize || batchWriteByteSize >= batchByteSize) { // LOG.warn("flush"); flush(); } @@ -235,6 +250,7 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun } outBatchQueue.put(batch); batch = freeBatchQueue.take(); + batchWriteByteSize = 0; } finally { lock.unlock(); } @@ -248,6 +264,10 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun if(recycle){ // 必须保证放入freeBatchQueue,否则发生异常没放入freeBatchQueue会导致block丢失,freeBatchQueue.take()申请block时会一直阻塞 freeBatchQueue.put(block); + int mebiBytes = (int) (bufferPool.getCurrentCacheSize() >>> 20); + if(mebiBytes >= 150){ + LOG.warn("bufferPoolCacheBufferSize: " + mebiBytes + "M"); + } } } } @@ -255,8 +275,11 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun private void doFlush(Block block) throws Exception { long start = System.currentTimeMillis(); int rowCnt = block.rowCnt(); - long blockByteSize = ClickHouseUtils.getBlockColumnsByteSize(block); - LOG.warn("flush " + rowCnt + ", totalSize:" + (blockByteSize >>> 20) + "M" + ", start:" + new Timestamp(start) + "," + (start - lastFlushTs)); + BlockColumnsByteSizeInfo blockByteSizeInfo = ClickHouseUtils.getBlockColumnsByteSizeInfo(block); + LOG.warn("flush " + rowCnt + ", totalSize:" + (blockByteSizeInfo.totalSize >>> 20) + "M, totalBufferSize: " + (blockByteSizeInfo.totalBufferSize >>> 20) + "M, start:" + new Timestamp(start) + "," + (start - lastFlushTs)); + if(!blockByteSizeInfo.bigColumnsInfo.isEmpty()){ + LOG.warn("bigColumnsInfo:" + blockByteSizeInfo.bigColumnsInfo); + } lastFlushTs = System.currentTimeMillis(); int retryCount = 0; @@ -278,7 +301,7 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun stmt.executeBatch(); internalMetrics.incrementOutEvents(rowCnt); - internalMetrics.incrementOutBytes(blockByteSize); + internalMetrics.incrementOutBytes(blockByteSizeInfo.totalSize); LOG.warn("flush " + rowCnt + " end:" + new Timestamp(System.currentTimeMillis()) + "," + (System.currentTimeMillis() - start)); return; @@ -398,15 +421,27 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun return this::convertInt8; } - if (type instanceof DataTypeUInt8 || type instanceof DataTypeInt16) { + if (type instanceof DataTypeUInt8) { + return this::convertUInt8; + } + + if (type instanceof DataTypeInt16) { return this::convertInt16; } - if (type instanceof DataTypeUInt16 || type instanceof DataTypeInt32) { + if (type instanceof DataTypeUInt16) { + return this::convertUInt16; + } + + if (type instanceof DataTypeInt32) { return this::convertInt32; } - if (type instanceof DataTypeUInt32 || type instanceof DataTypeInt64) { + if (type instanceof DataTypeUInt32) { + return this::convertUInt32; + } + + if (type instanceof DataTypeInt64) { return this::convertInt64; } @@ -437,11 +472,11 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun if (type instanceof DataTypeNullable) { IDataType nestedDataType = ((DataTypeNullable) type).getNestedDataType(); ValueConverter converter = this.makeConverter(nestedDataType); - return obj -> { + return (obj, sizeHelper) -> { if (obj == null) { return null; } - return converter.convert(obj); + return converter.convert(obj, sizeHelper); }; } @@ -449,8 +484,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun IDataType eleDataType = ((DataTypeArray) type).getElemDataType(); ValueConverter eleConverter = this.makeConverter(eleDataType); Object defaultValue = new ClickHouseArray(eleDataType, new Object[0]); - return obj -> { - return this.convertArray(obj, eleDataType, eleConverter, defaultValue); + return (obj, sizeHelper) -> { + return this.convertArray(obj, eleDataType, eleConverter, defaultValue, sizeHelper); }; } @@ -465,18 +500,20 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun final BytesCharVarSeq bytesCharVarSeq = new BytesCharVarSeq(bytes, 0); @Override - public Object convert(Object obj) throws ClickHouseSQLException { + public Object convert(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } if (obj instanceof byte[]) { byte[] bs = (byte[]) obj; + sizeHelper.size += writeBytesSizeByLen(bs.length); bytesCharVarSeq.setBytesAndLen(bs, bs.length); return bytesCharVarSeq; } String str; if (obj instanceof CharSequence) { if (((CharSequence) obj).length() == 0) { + sizeHelper.size += 1; return ClickHouseUtils.EMPTY_BYTES_CHAR_SEQ; } str = obj.toString(); @@ -490,6 +527,7 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun bs = new byte[length]; } int len = encodeUTF8(str, bs); + sizeHelper.size += writeBytesSizeByLen(len); bytesCharVarSeq.setBytesAndLen(bs, len); return bytesCharVarSeq; } @@ -501,16 +539,19 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun final byte[] bytes = new byte[MAX_STR_BYTES_LENGTH]; @Override - public Object convert(Object obj) throws ClickHouseSQLException { + public Object convert(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } if (obj instanceof byte[]) { - return new BytesCharSeq((byte[]) obj); + byte[] bs = (byte[]) obj; + sizeHelper.size += writeBytesSizeByLen(bs.length); + return new BytesCharSeq(bs); } String str; if (obj instanceof CharSequence) { if (((CharSequence) obj).length() == 0) { + sizeHelper.size += 1; return ClickHouseUtils.EMPTY_BYTES_CHAR_SEQ; } str = obj.toString(); @@ -524,12 +565,14 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun bs = new byte[length]; } int len = encodeUTF8(str, bs); + sizeHelper.size += writeBytesSizeByLen(len); return new BytesCharSeq(Arrays.copyOf(bytes, len)); } }; } - private Object convertDate(Object obj) throws ClickHouseSQLException { + private Object convertDate(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 2; if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } @@ -539,7 +582,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertDate32(Object obj) throws ClickHouseSQLException { + private Object convertDate32(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 4; if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } @@ -549,7 +593,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertDateTime(Object obj) throws ClickHouseSQLException { + private Object convertDateTime(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 4; if (obj instanceof Number) { long ts = ((Number) obj).longValue(); // 小于UINT32_MAX认为单位是s @@ -575,7 +620,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertDateTime64(Object obj) throws ClickHouseSQLException { + private Object convertDateTime64(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 8; if (obj instanceof Number) { long ts = ((Number) obj).longValue(); // 小于UINT32_MAX认为单位是s @@ -601,7 +647,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertInt8(Object obj) throws ClickHouseSQLException { + private Object convertInt8(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 1; if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } @@ -611,7 +658,19 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertInt16(Object obj) throws ClickHouseSQLException { + private Object convertUInt8(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 1; + if (obj == null) { + throw new ClickHouseSQLException(-1, "type doesn't support null value"); + } + if (obj instanceof Number) return ((Number) obj).shortValue(); + if (obj instanceof String) return (short) Integer.parseInt((String) obj); + + throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); + } + + private Object convertInt16(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 2; if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } @@ -621,7 +680,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertInt32(Object obj) throws ClickHouseSQLException { + private Object convertUInt16(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 2; if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } @@ -631,7 +691,30 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertInt64(Object obj) throws ClickHouseSQLException { + private Object convertInt32(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 4; + if (obj == null) { + throw new ClickHouseSQLException(-1, "type doesn't support null value"); + } + if (obj instanceof Number) return ((Number) obj).intValue(); + if (obj instanceof String) return Integer.parseInt((String) obj); + + throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); + } + + private Object convertUInt32(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 4; + if (obj == null) { + throw new ClickHouseSQLException(-1, "type doesn't support null value"); + } + if (obj instanceof Number) return ((Number) obj).longValue(); + if (obj instanceof String) return Long.parseLong((String) obj); + + throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); + } + + private Object convertInt64(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 8; if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } @@ -641,7 +724,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertUInt64(Object obj) throws ClickHouseSQLException { + private Object convertUInt64(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 8; if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } @@ -653,7 +737,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertFloat32(Object obj) throws ClickHouseSQLException { + private Object convertFloat32(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 4; if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } @@ -663,7 +748,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertFloat64(Object obj) throws ClickHouseSQLException { + private Object convertFloat64(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 8; if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } @@ -673,7 +759,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertDecimal(Object obj) throws ClickHouseSQLException { + private Object convertDecimal(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 32; if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } @@ -685,7 +772,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertUUID(Object obj) throws ClickHouseSQLException { + private Object convertUUID(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 16; if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); } @@ -697,7 +785,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun throw new ClickHouseSQLException(-1, "unhandled type: {}" + obj.getClass()); } - private Object convertNothing(Object obj) throws ClickHouseSQLException { + private Object convertNothing(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException { + sizeHelper.size += 1; return null; } @@ -705,7 +794,8 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun Object obj, IDataType eleDataType, ValueConverter eleConverter, - Object defaultValue) + Object defaultValue, + SizeHelper sizeHelper) throws ClickHouseSQLException { if (obj == null) { throw new ClickHouseSQLException(-1, "type doesn't support null value"); @@ -720,13 +810,12 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun } Object[] elements = new Object[list.size()]; for (int i = 0; i < elements.length; i++) { - elements[i] = eleConverter.convert(list.get(i)); + elements[i] = eleConverter.convert(list.get(i), sizeHelper); } return new ClickHouseArray(eleDataType, elements); } - throw new ClickHouseSQLException( - -1, "require ClickHouseArray for column, but found " + obj.getClass()); + throw new ClickHouseSQLException(-1, "require ClickHouseArray for column, but found " + obj.getClass()); } // copy from org.apache.flink.table.runtime.util.StringUtf8Utils @@ -810,6 +899,10 @@ public abstract class AbstractBatchIntervalClickHouseSink extends RichSinkFun @FunctionalInterface public interface ValueConverter extends Serializable { - Object convert(Object obj) throws ClickHouseSQLException; + Object convert(Object obj, SizeHelper sizeHelper) throws ClickHouseSQLException; + } + + public static class SizeHelper implements Serializable{ + public int size = 0; } } diff --git a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/sink/EventBatchIntervalClickHouseSink.java b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/sink/EventBatchIntervalClickHouseSink.java index c38324a..ab7a9b8 100644 --- a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/sink/EventBatchIntervalClickHouseSink.java +++ b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/sink/EventBatchIntervalClickHouseSink.java @@ -17,8 +17,8 @@ public class EventBatchIntervalClickHouseSink extends AbstractBatchIntervalClick private Set disabledFields; private String simpleName; - public EventBatchIntervalClickHouseSink(Schema schema, int batchSize, long batchIntervalMs, String host, String table, Properties connInfo) { - super(batchSize, batchIntervalMs, host, table, connInfo); + public EventBatchIntervalClickHouseSink(Schema schema, int batchSize, int batchByteSize, long batchIntervalMs, String host, String table, Properties connInfo) { + super(batchSize, batchByteSize, batchIntervalMs, host, table, connInfo); this.schema = schema; } @@ -35,7 +35,8 @@ public class EventBatchIntervalClickHouseSink extends AbstractBatchIntervalClick } @Override - boolean addBatch(Block batch, Event event) throws Exception { + int addBatch(Block batch, Event event) throws Exception { + int writeSize = 0; Map map = event.getExtractedFields(); String columnName; Object value; @@ -44,6 +45,7 @@ public class EventBatchIntervalClickHouseSink extends AbstractBatchIntervalClick if(disabledFields != null && disabledFields.contains(columnName)){ value = columnDefaultValues[i]; batch.setObject(i, value); // 默认值不用转换 + writeSize += columnDefaultSizes[i]; continue; } @@ -52,19 +54,22 @@ public class EventBatchIntervalClickHouseSink extends AbstractBatchIntervalClick if (value == null) { value = columnDefaultValues[i]; batch.setObject(i, value); // 默认值不用转换 + writeSize += columnDefaultSizes[i]; } else { // int columnIdx = batch.paramIdx2ColumnIdx(i); // batch.setObject(columnIdx, convertToCkDataType(columnTypes[i], value)); // batch.setObject(i, convertToCkDataType(dataType, value)); try { - batch.setObject(i, columnConverters[i].convert(value)); + writeSizeHelper.size = 0; + batch.setObject(i, columnConverters[i].convert(value, writeSizeHelper)); + writeSize += writeSizeHelper.size; } catch (Exception e) { throw new RuntimeException(columnNames[i] + "列转换值出错:" + value + ", event data:" + JSON.toJSONString(map), e); } } } - return true; + return writeSize; } @Override diff --git a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/util/BlockColumnsByteSizeInfo.java b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/util/BlockColumnsByteSizeInfo.java new file mode 100644 index 0000000..4e23417 --- /dev/null +++ b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/util/BlockColumnsByteSizeInfo.java @@ -0,0 +1,13 @@ +package com.geedgenetworks.connectors.clickhouse.util; + +public class BlockColumnsByteSizeInfo { + public long totalSize; + public long totalBufferSize; + public String bigColumnsInfo; + + public BlockColumnsByteSizeInfo(long totalSize, long totalBufferSize, String bigColumnsInfo) { + this.totalSize = totalSize; + this.totalBufferSize = totalBufferSize; + this.bigColumnsInfo = bigColumnsInfo; + } +} diff --git a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/util/ClickHouseUtils.java b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/util/ClickHouseUtils.java index 8efc7d0..7c42498 100644 --- a/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/util/ClickHouseUtils.java +++ b/groot-connectors/connector-clickhouse/src/main/java/com/geedgenetworks/connectors/clickhouse/util/ClickHouseUtils.java @@ -1,18 +1,17 @@ package com.geedgenetworks.connectors.clickhouse.util; -import com.github.housepower.buffer.ByteArrayWriter; +import com.geedgenetworks.connectors.clickhouse.buffer.BufferPool; +import com.geedgenetworks.connectors.clickhouse.buffer.ReusedColumnWriterBuffer; +import com.geedgenetworks.connectors.clickhouse.jdbc.DataTypeStringV2; import com.github.housepower.data.*; import com.github.housepower.data.type.*; -import com.github.housepower.data.type.complex.DataTypeArray; -import com.github.housepower.data.type.complex.DataTypeDecimal; -import com.github.housepower.data.type.complex.DataTypeFixedString; -import com.github.housepower.data.type.complex.DataTypeString; +import com.github.housepower.data.type.complex.*; import com.github.housepower.jdbc.ClickHouseArray; import com.github.housepower.jdbc.ClickHouseConnection; import com.github.housepower.misc.BytesCharSeq; import com.github.housepower.misc.DateTimeUtil; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -21,6 +20,7 @@ import java.lang.reflect.Field; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.sql.*; import java.time.ZoneId; import java.util.ArrayList; @@ -87,7 +87,7 @@ public class ClickHouseUtils { return sb.toString(); } - public static Tuple2 getInsertColumnsAndDefaultValuesForTable( + public static Tuple3 getInsertColumnsAndDefaultValuesAndDefaultSizesForTable( String[] urls, int urlIndex, Properties connInfo, String table) throws Exception { Class.forName("com.github.housepower.jdbc.ClickHouseDriver"); @@ -104,6 +104,7 @@ public class ClickHouseUtils { List columnNames = new ArrayList<>(); List columnDefaultValues = new ArrayList<>(); + List columnDefaultSizes = new ArrayList<>(); while (rst.next()) { String name = rst.getString("name"); String typeStr = rst.getString("type"); @@ -127,9 +128,10 @@ public class ClickHouseUtils { } columnNames.add(name); columnDefaultValues.add(defaultValue); + columnDefaultSizes.add(getDefaultValueSize(type, defaultExpression)); } - return new Tuple2<>(columnNames.toArray(new String[columnNames.size()]), columnDefaultValues.toArray(new Object[columnDefaultValues.size()])); + return new Tuple3<>(columnNames.toArray(new String[columnNames.size()]), columnDefaultValues.toArray(new Object[columnDefaultValues.size()]), columnDefaultSizes.stream().mapToInt(x -> x).toArray()); } catch (SQLException e) { LOG.error("ClickHouse Connection Exception url:" + urls[urlIndex], e); if (retryCount >= 3) { @@ -242,7 +244,7 @@ public class ClickHouseUtils { } public static void copyInsertBlockColumns(Block src, Block desc) throws Exception { - desc.cleanup(); + //desc.cleanup(); IColumn[] srcColumns = (IColumn[]) blockColumnsField.get(src); IColumn[] descColumns = (IColumn[]) blockColumnsField.get(desc); @@ -253,18 +255,31 @@ public class ClickHouseUtils { blockRowCntField.set(desc, blockRowCntField.get(src)); } + public static void initBlockWriteBuffer(Block block, BufferPool bufferPool) throws Exception { + IColumn[] columns = (IColumn[]) blockColumnsField.get(block); + for (int i = 0; i < columns.length; i++) { + ColumnWriterBuffer writeBuffer = columns[i].getColumnWriterBuffer(); + if(writeBuffer == null){ + writeBuffer = new ReusedColumnWriterBuffer(bufferPool); + columns[i].setColumnWriterBuffer(writeBuffer); + }else{ + writeBuffer.reset(); + } + } + } + public static void resetInsertBlockColumns(Block block) throws Exception { - block.cleanup(); blockRowCntField.set(block, 0); IColumn[] columns = (IColumn[]) blockColumnsField.get(block); for (int i = 0; i < columns.length; i++) { + ColumnWriterBuffer writeBuffer = columns[i].getColumnWriterBuffer(); String name = columns[i].name(); IDataType dataType = columns[i].type(); columns[i] = ColumnFactory.createColumn(name, dataType, null); // values用于rst读取 + writeBuffer.reset(); + columns[i].setColumnWriterBuffer(writeBuffer); } - - block.initWriteBuffer(); } private static Object parseDefaultValue(IDataType type, String defaultExpression) { @@ -294,32 +309,113 @@ public class ClickHouseUtils { return defaultValue; } - public static long getBlockColumnsByteSize(Block block) throws Exception { + private static int getDefaultValueSize(IDataType type, String defaultExpression){ + if (type instanceof DataTypeString || type instanceof DataTypeFixedString || type instanceof DataTypeStringV2) { + if(StringUtils.isBlank(defaultExpression)){ + return 1; + }else{ + return writeBytesSizeByLen(defaultExpression.getBytes(StandardCharsets.UTF_8).length); + } + } + + if (type instanceof DataTypeDate) { + return 2; + } + + if (type instanceof DataTypeDate32) { + return 4; + } + + if (type instanceof DataTypeDateTime) { + return 4; + } + + if (type instanceof DataTypeDateTime64) { + return 8; + } + + if (type instanceof DataTypeInt8 || type instanceof DataTypeUInt8) { + return 1; + } + + if (type instanceof DataTypeInt16 || type instanceof DataTypeUInt16) { + return 2; + } + + if (type instanceof DataTypeInt32 || type instanceof DataTypeUInt32) { + return 4; + } + + if (type instanceof DataTypeInt64 || type instanceof DataTypeUInt64) { + return 8; + } + + if (type instanceof DataTypeFloat32) { + return 4; + } + + if (type instanceof DataTypeFloat64) { + return 8; + } + + if (type instanceof DataTypeDecimal) { + return 32; + } + + if (type instanceof DataTypeUUID) { + return 16; + } + + if (type instanceof DataTypeNothing) { + return 1; + } + + if (type instanceof DataTypeNullable) { + return getDefaultValueSize(((DataTypeNullable)type).getNestedDataType(), defaultExpression); + } + + if (type instanceof DataTypeArray) { + return 0; + } + + throw new UnsupportedOperationException("Unsupported type: " + type); + } + + public static int writeBytesSizeByLen(final int len) { + int bytes = 1, value = len; + while ((value & 0xffffff80) != 0L) { + bytes += 1; + value >>>= 7; + } + return bytes + len; + } + + public static BlockColumnsByteSizeInfo getBlockColumnsByteSizeInfo(Block block) throws Exception { IColumn[] columns = (IColumn[]) blockColumnsField.get(block); - Field field = AbstractColumn.class.getDeclaredField("buffer"); - field.setAccessible(true); - Field columnWriterField = ColumnWriterBuffer.class.getDeclaredField("columnWriter"); - columnWriterField.setAccessible(true); - Field byteBufferListField = ByteArrayWriter.class.getDeclaredField("byteBufferList"); - byteBufferListField.setAccessible(true); - int size = 0; - int totalSize = 0; + long size = 0, bufferSize = 0; + long totalSize = 0, totalBufferSize = 0; + long sizeThreshold = Math.max(200 << 20 / columns.length * 2, 4 << 20); + StringBuilder sb = new StringBuilder(); for (int i = 0; i < columns.length; i++) { - Object columnWriter = columnWriterField.get(field.get(columns[i])); - List byteBufferList = - (List) byteBufferListField.get(columnWriter); + List byteBufferList = ((ReusedColumnWriterBuffer) columns[i].getColumnWriterBuffer()).getBufferList(); size = 0; + bufferSize = 0; for (ByteBuffer byteBuffer : byteBufferList) { size += byteBuffer.position(); + bufferSize += byteBuffer.capacity(); } totalSize += size; - /*if (size > 1000000) { - LOG.warn(columns[i].name() + "buf cnt:" + byteBufferList.size() + ", size:" + size/1000000.0 + "M"); - }*/ + totalBufferSize += bufferSize; + if (bufferSize > sizeThreshold) { + if(sb.length() > 0){ + sb.append(','); + } + sb.append(String.format("%s:%d M", columns[i].name(), (bufferSize >>> 20)) ); + } } - return totalSize; + return new BlockColumnsByteSizeInfo(totalSize, totalBufferSize, sb.toString()); } -- cgit v1.2.3 From 8e73de162cd679ba437c938fd003b1797aa7985e Mon Sep 17 00:00:00 2001 From: doufenghu Date: Thu, 25 Jul 2024 19:03:00 +0800 Subject: Release 1.5.0 --- .../com/geedgenetworks/example/GrootStreamExample.java | 2 +- .../main/resources/examples/inline_to_print_test.yaml | 17 +++++++++++++---- pom.xml | 2 +- 3 files changed, 15 insertions(+), 6 deletions(-) diff --git a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java index 2e21e49..d927133 100644 --- a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java +++ b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java @@ -13,7 +13,7 @@ import java.nio.file.Paths; public class GrootStreamExample { public static void main(String[] args) throws FileNotFoundException, URISyntaxException { - String configPath = args.length > 0 ? args[0] : "/examples/session_record_mock_to_print.yaml"; + String configPath = args.length > 0 ? args[0] : "/examples/inline_to_print_test.yaml"; String configFile = getTestConfigFile(configPath); ExecuteCommandArgs executeCommandArgs = new ExecuteCommandArgs(); executeCommandArgs.setConfigFile(configFile); diff --git a/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_test.yaml b/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_test.yaml index 1e1e13e..fb51a0e 100644 --- a/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_test.yaml +++ b/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_test.yaml @@ -23,6 +23,10 @@ sources: type: string - name: device_tag type: string + - name: sent_bytes + type: bigint + - name: received_bytes + type: bigint properties: data: '{"tcp_rtt_ms":128,"decoded_as":"HTTP", "http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.ct.cn","http_url":"www.ct.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.11.22.22","server_ip":"8.8.8.8","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":5575,"sent_pkts":97,"sent_bytes":5892,"received_pkts":250,"received_bytes":333931,"tcp_c2s_ip_fragments":0,"tcp_s2c_ip_fragments":0,"tcp_c2s_rtx_pkts":0,"tcp_c2s_rtx_bytes":0,"tcp_s2c_rtx_pkts":0,"tcp_s2c_rtx_bytes":0,"tcp_c2s_o3_pkts":0,"tcp_s2c_o3_pkts":0,"tcp_c2s_lost_bytes":0,"tcp_s2c_lost_bytes":0,"flags":26418,"flags_identify_info":[100,1,100,60,150,100,1,2],"app_transition":"http.1111.test_1_1","decoded_as":"HTTP","server_fqdn":"www.ct.cn","app":"test_1_1","decoded_path":"ETHERNET.IPv4.TCP.http","fqdn_category_list":[1767],"t_vsys_id":1,"vsys_id":1,"session_id":290538039798223400,"tcp_handshake_latency_ms":41,"client_os_desc":"Windows","server_os_desc":"Linux","data_center":"center-xxg-tsgx","device_group":"group-xxg-tsgx","device_tag":"{\"tags\":[{\"tag\":\"data_center\",\"value\":\"center-xxg-tsgx\"},{\"tag\":\"device_group\",\"value\":\"group-xxg-tsgx\"}]}","device_id":"9800165603247024","sled_ip":"192.168.40.39","dup_traffic_flag":0}' format: json @@ -37,13 +41,13 @@ filters: preprocessing_pipelines: transform_processor: type: projection - remove_fields: [client_ip] + remove_fields: [http_request_line] processing_pipelines: session_record_processor: type: projection remove_fields: [device_tag] - output_fields: [log_id, client_ip, client_geolocation, client_asn, server_domain, server_ip, server_geolocation, server_asn] + output_fields: [log_id, renamed_client_ip, c2s_bytes] functions: - function: DROP lookup_fields: [] @@ -98,8 +102,13 @@ processing_pipelines: parameters: precision: milliseconds - function: RENAME - lookup_fields: [ device_tag ] - output_fields: [ renamed_device_tag ] + parameters: + rename_fields: + client_ip: renamed_client_ip + - function: EVAL + output_fields: [ c2s_bytes ] + parameters: + value_expression: sent_bytes sinks: print_sink: diff --git a/pom.xml b/pom.xml index 5e3e128..5e2ebe0 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ - 1.5.0-SNAPSHOT + 1.5.0 11 UTF-8 ${java.version} -- cgit v1.2.3 From 7bc23562099556f8fa28a45b2481d13cc6afdba8 Mon Sep 17 00:00:00 2001 From: doufenghu Date: Tue, 27 Aug 2024 20:13:59 +0800 Subject: release 1.5.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 53bcf1f..8378a72 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ - 1.5.0-SNAPSHOT + 1.5.0 11 UTF-8 ${java.version} -- cgit v1.2.3 From 3a6e0934386f0ed6aead149786fa1d7fe9d89810 Mon Sep 17 00:00:00 2001 From: doufenghu Date: Thu, 1 Aug 2024 10:29:46 +0800 Subject: [feature][e2e-test] add e2e-clickhouse module --- groot-tests/pom.xml | 1 + groot-tests/test-e2e-clickhouse/pom.xml | 63 ++++ .../test/e2e/clickhouse/ClickHouseIT.java | 363 ++++++++++++++++++++ .../src/test/resources/init/clickhouse_init.conf | 143 ++++++++ .../test/e2e/kafka/KafkaConnectorIT.java | 375 --------------------- .../com/geedgenetworks/test/e2e/kafka/KafkaIT.java | 372 ++++++++++++++++++++ pom.xml | 46 +-- 7 files changed, 966 insertions(+), 397 deletions(-) create mode 100644 groot-tests/test-e2e-clickhouse/pom.xml create mode 100644 groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java create mode 100644 groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_init.conf delete mode 100644 groot-tests/test-e2e-kafka/src/test/java/com/geedgenetworks/test/e2e/kafka/KafkaConnectorIT.java create mode 100644 groot-tests/test-e2e-kafka/src/test/java/com/geedgenetworks/test/e2e/kafka/KafkaIT.java diff --git a/groot-tests/pom.xml b/groot-tests/pom.xml index 76f533a..b46ad10 100644 --- a/groot-tests/pom.xml +++ b/groot-tests/pom.xml @@ -16,6 +16,7 @@ test-common test-e2e-base test-e2e-kafka + test-e2e-clickhouse diff --git a/groot-tests/test-e2e-clickhouse/pom.xml b/groot-tests/test-e2e-clickhouse/pom.xml new file mode 100644 index 0000000..c576100 --- /dev/null +++ b/groot-tests/test-e2e-clickhouse/pom.xml @@ -0,0 +1,63 @@ + + + 4.0.0 + + com.geedgenetworks + groot-tests + ${revision} + + + test-e2e-clickhouse + Groot : Tests : E2E : ClickHouse + + + 11 + 11 + UTF-8 + 0.3.2-patch9 + + + + + + com.geedgenetworks + test-common + ${project.version} + test-jar + test + + + + org.testcontainers + clickhouse + ${testcontainer.version} + test + + + + com.clickhouse + clickhouse-jdbc + ${clickhouse.jdbc.version} + test + + + + com.geedgenetworks + connector-clickhouse + ${project.version} + test + + + + org.xerial.snappy + snappy-java + ${snappy-java.version} + test + + + + + + \ No newline at end of file diff --git a/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java b/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java new file mode 100644 index 0000000..f752d57 --- /dev/null +++ b/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java @@ -0,0 +1,363 @@ +package com.geedgenetworks.test.e2e.clickhouse; + +import com.geedgenetworks.test.common.TestResource; +import com.geedgenetworks.test.common.TestSuiteBase; +import com.geedgenetworks.test.common.container.ContainerUtil; +import com.geedgenetworks.test.common.container.TestContainer; +import com.geedgenetworks.test.common.container.TestContainerId; +import com.geedgenetworks.test.common.junit.DisabledOnContainer; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import lombok.extern.slf4j.Slf4j; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.ClickHouseContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.shaded.org.apache.commons.io.IOUtils; +import org.testcontainers.utility.DockerLoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.sql.*; +import java.sql.Date; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +@Slf4j +@DisabledOnContainer( + value = {TestContainerId.FLINK_1_17}, + disabledReason = "Override TestSuiteBase @DisabledOnContainer") +public class ClickHouseIT extends TestSuiteBase implements TestResource { + private static final String CLICKHOUSE_DOCKER_IMAGE = "clickhouse/clickhouse-server:23.3.19.32"; + private static final String DRIVER_CLASS = "com.clickhouse.jdbc.ClickHouseDriver"; + private static final String INIT_CLICKHOUSE_PATH = "/init/clickhouse_init.conf"; + private static final String DATABASE = "default"; + private static final String SOURCE_TABLE = "source_table"; + private static final String SINK_TABLE = "sink_table"; + private static final String INSERT_SQL = "insert_sql"; + private static final String COMPARE_SQL = "compare_sql"; + private static final String HOST = "clickhouse"; + private static final Config CONFIG = getInitClickhouseConfig(); + private ClickHouseContainer clickHouseContainer; + private Connection connection; + private static final ListTEST_DATASET = + generateTestDataSet(); + private static final String[] default_columns = new String[] { + "id", + "c_map", + "c_array_string", + "c_array_short", + "c_array_int", + "c_array_long", + "c_array_float", + "c_array_double", + "c_string", + "c_boolean", + "c_int8", + "c_int16", + "c_int32", + "c_int64", + "c_float32", + "c_float64", + "c_decimal", + "c_date", + "c_datetime", + "c_nullable", + "c_lowcardinality", + "c_nested.int", + "c_nested.double", + "c_nested.string", + "c_int128", + "c_uint128", + "c_int256", + "c_uint256", + "c_point", + "c_ring" + }; + + @BeforeAll + @Override + public void startUp() throws Exception { + this.clickHouseContainer = + new ClickHouseContainer(CLICKHOUSE_DOCKER_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(HOST) + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger(CLICKHOUSE_DOCKER_IMAGE))); + Startables.deepStart(Stream.of(this.clickHouseContainer)).join(); + log.info("Clickhouse container started"); + Awaitility.given() + .ignoreExceptions() + .await() + .atMost(360L, TimeUnit.SECONDS) + .untilAsserted(this::initConnection); + this.initializeClickhouseTable(); + this.batchInsertData(); + + } + + + @TestTemplate + public void testClickhouse(TestContainer container) throws Exception { + assertHasData(SOURCE_TABLE); + + //assertHasData(SINK_TABLE); + //compareResult(); + //clearSinkTable(); + + } + + private void assertHasData(String table) { + try (Statement statement = connection.createStatement()) { + String sql = String.format("select * from %s.%s limit 1", DATABASE, table); + ResultSet source = statement.executeQuery(sql); + Assertions.assertTrue(source.next()); + } catch (SQLException e) { + throw new RuntimeException("test clickhouse server image error", e); + } + } + + private void clearSinkTable() { + try (Statement statement = connection.createStatement()) { + statement.execute(String.format("truncate table %s.%s", DATABASE, SINK_TABLE)); + } catch (SQLException e) { + throw new RuntimeException("Test clickhouse server image error", e); + } + } + + private void compareResult() throws SQLException, IOException { + String sourceSql = "select * from " + SOURCE_TABLE + " order by id"; + String sinkSql = "select * from " + SINK_TABLE + " order by id"; + List columnList = + Arrays.stream(default_columns).collect(Collectors.toList()); + Statement sourceStatement = connection.createStatement(); + Statement sinkStatement = connection.createStatement(); + ResultSet sourceResultSet = sourceStatement.executeQuery(sourceSql); + ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql); + Assertions.assertEquals( + sourceResultSet.getMetaData().getColumnCount(), + sinkResultSet.getMetaData().getColumnCount()); + while (sourceResultSet.next()) { + if (sinkResultSet.next()) { + for (String column : columnList) { + Object source = sourceResultSet.getObject(column); + Object sink = sinkResultSet.getObject(column); + if (!Objects.deepEquals(source, sink)) { + InputStream sourceAsciiStream = sourceResultSet.getBinaryStream(column); + InputStream sinkAsciiStream = sinkResultSet.getBinaryStream(column); + String sourceValue = + IOUtils.toString(sourceAsciiStream, StandardCharsets.UTF_8); + String sinkValue = + IOUtils.toString(sinkAsciiStream, StandardCharsets.UTF_8); + Assertions.assertEquals(sourceValue, sinkValue); + } + Assertions.assertTrue(true); + } + } + } + String columns = String.join(",", default_columns); + Assertions.assertTrue( + compare(String.format(CONFIG.getString(COMPARE_SQL), columns, columns))); + } + + private Boolean compare(String sql) { + try (Statement statement = connection.createStatement()) { + ResultSet resultSet = statement.executeQuery(sql); + return !resultSet.next(); + } catch (SQLException e) { + throw new RuntimeException("result compare error", e); + } + } + + + private void batchInsertData() { + String sql = CONFIG.getString(INSERT_SQL); + PreparedStatement preparedStatement = null; + try { + this.connection.setAutoCommit(true); + preparedStatement = this.connection.prepareStatement(sql); + for (Object[] row : TEST_DATASET) { + preparedStatement.setLong(1, (Long) row[0]); + preparedStatement.setObject(2, row[1]); + preparedStatement.setArray(3, toSqlArray(row[2])); + preparedStatement.setArray(4, toSqlArray(row[3])); + preparedStatement.setArray(5, toSqlArray(row[4])); + preparedStatement.setArray(6, toSqlArray(row[5])); + preparedStatement.setArray(7, toSqlArray(row[6])); + preparedStatement.setArray(8, toSqlArray(row[7])); + preparedStatement.setString(9, (String) row[8]); + preparedStatement.setBoolean(10, (Boolean) row[9]); + preparedStatement.setByte(11, (Byte) row[10]); + preparedStatement.setShort(12, (Short) row[11]); + preparedStatement.setInt(13, (Integer) row[12]); + preparedStatement.setLong(14, (Long) row[13]); + preparedStatement.setFloat(15, (Float) row[14]); + preparedStatement.setDouble(16, (Double) row[15]); + preparedStatement.setBigDecimal(17, (BigDecimal) row[16]); + preparedStatement.setDate(18, Date.valueOf((LocalDate) row[17])); + preparedStatement.setTimestamp( + 19, Timestamp.valueOf((LocalDateTime) row[18])); + preparedStatement.setInt(20, (Integer) row[19]); + preparedStatement.setString(21, (String) row[20]); + preparedStatement.setArray(22, toSqlArray(row[21])); + preparedStatement.setArray(23, toSqlArray(row[22])); + preparedStatement.setArray(24, toSqlArray(row[23])); + preparedStatement.setObject(25, row[24]); + preparedStatement.setObject(26, row[25]); + preparedStatement.setObject(27, row[26]); + preparedStatement.setObject(28, row[27]); + preparedStatement.setObject(29, row[28]); + preparedStatement.setObject(30, row[29]); + preparedStatement.addBatch(); + } + + preparedStatement.executeBatch(); + preparedStatement.clearBatch(); + + } catch (SQLException e) { + throw new RuntimeException("Batch insert data failed!", e); + } finally { + if (preparedStatement != null) { + try { + preparedStatement.close(); + } catch (SQLException e) { + throw new RuntimeException("PreparedStatement close failed!", e); + } + } + } + + + + } + + + private Array toSqlArray(Object value) throws SQLException { + Object[] elements = null; + String sqlType = null; + if (String[].class.equals(value.getClass())) { + sqlType = "TEXT"; + elements = (String[]) value; + } else if (Boolean[].class.equals(value.getClass())) { + sqlType = "BOOLEAN"; + elements = (Boolean[]) value; + } else if (Byte[].class.equals(value.getClass())) { + sqlType = "TINYINT"; + elements = (Byte[]) value; + } else if (Short[].class.equals(value.getClass())) { + sqlType = "SMALLINT"; + elements = (Short[]) value; + } else if (Integer[].class.equals(value.getClass())) { + sqlType = "INTEGER"; + elements = (Integer[]) value; + } else if (Long[].class.equals(value.getClass())) { + sqlType = "BIGINT"; + elements = (Long[]) value; + } else if (Float[].class.equals(value.getClass())) { + sqlType = "REAL"; + elements = (Float[]) value; + } else if (Double[].class.equals(value.getClass())) { + sqlType = "DOUBLE"; + elements = (Double[]) value; + } + if (sqlType == null) { + throw new IllegalArgumentException( + "array inject error, not supported data type: " + value.getClass()); + } + return connection.createArrayOf(sqlType, elements); + } + + private static List generateTestDataSet() { + List rows = new ArrayList<>(); + for (int i = 0; i < 100; ++i) { + Object[] row = new Object[] { + (long) i, + Collections.singletonMap("key", Integer.parseInt("1")), + new String[] {"string"}, + new Short[] {Short.parseShort("1")}, + new Integer[] {Integer.parseInt("1")}, + new Long[] {Long.parseLong("1")}, + new Float[] {Float.parseFloat("1.1")}, + new Double[] {Double.parseDouble("1.1")}, + "string", + Boolean.FALSE, + Byte.parseByte("1"), + Short.parseShort("1"), + Integer.parseInt("1"), + Long.parseLong("1"), + Float.parseFloat("1.1"), + Double.parseDouble("1.1"), + BigDecimal.valueOf(11L, 1), + LocalDate.now(), + LocalDateTime.now(), + i, + "string", + new Integer[] {Integer.parseInt("1")}, + new Double[] {Double.parseDouble("1.1")}, + new String[] {"1"}, + "170141183460469231731687303715884105727", + "340282366920938463463374607431768211455", + "57896044618658097711785492504343953926634992332820282019728792003956564819967", + "115792089237316195423570985008687907853269984665640564039457584007913129639935", + new double[] {1, 2}, + new double[][] {{2, 3}, {4, 5}} + }; + rows.add(row); + } + return rows; + } + + private void initConnection() + throws SQLException, ClassNotFoundException, InstantiationException, + IllegalAccessException { + final Properties info = new Properties(); + info.put("user", this.clickHouseContainer.getUsername()); + info.put("password", this.clickHouseContainer.getPassword()); + this.connection = + ((Driver) Class.forName(DRIVER_CLASS).newInstance()) + .connect(this.clickHouseContainer.getJdbcUrl(), info); + } + + private void initializeClickhouseTable() { + try { + Statement statement = this.connection.createStatement(); + statement.execute(CONFIG.getString(SOURCE_TABLE)); + statement.execute(CONFIG.getString(SINK_TABLE)); + } catch (SQLException e) { + throw new RuntimeException("Initializing Clickhouse table failed!", e); + } + } + + private static Config getInitClickhouseConfig() { + File file = ContainerUtil.getResourcesFile(INIT_CLICKHOUSE_PATH); + Config config = ConfigFactory.parseFile(file); + assert config.hasPath(SOURCE_TABLE) + && config.hasPath(SINK_TABLE) + && config.hasPath(INSERT_SQL) + && config.hasPath(COMPARE_SQL); + return config; + } + + @AfterAll + @Override + public void tearDown() throws Exception { + if (this.connection != null) { + this.connection.close(); + } + if (this.clickHouseContainer != null) { + this.clickHouseContainer.stop(); + } + + } +} diff --git a/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_init.conf b/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_init.conf new file mode 100644 index 0000000..78f2daa --- /dev/null +++ b/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_init.conf @@ -0,0 +1,143 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +source_table = """ +set allow_experimental_geo_types = 1; +create table if not exists `default`.source_table( + `id` Int64, + `c_map` Map(String, Int32), + `c_array_string` Array(String), + `c_array_short` Array(Int16), + `c_array_int` Array(Int32), + `c_array_long` Array(Int64), + `c_array_float` Array(Float32), + `c_array_double` Array(Float64), + `c_string` String, + `c_boolean` Boolean, + `c_int8` Int8, + `c_int16` Int16, + `c_int32` Int32, + `c_int64` Int64, + `c_float32` Float32, + `c_float64` Float64, + `c_decimal` Decimal(9,4), + `c_date` Date, + `c_datetime` DateTime64, + `c_nullable` Nullable(Int32), + `c_lowcardinality` LowCardinality(String), + `c_nested` Nested + ( + `int` UInt32, + `double` Float64, + `string` String + ), + `c_int128` Int128, + `c_uint128` UInt128, + `c_int256` Int256, + `c_uint256` UInt256, + `c_point` Point, + `c_ring` Ring +)engine=Memory; +""" + +sink_table = """ +create table if not exists `default`.sink_table( + `id` Int64, + `c_map` Map(String, Int32), + `c_array_string` Array(String), + `c_array_short` Array(Int16), + `c_array_int` Array(Int32), + `c_array_long` Array(Int64), + `c_array_float` Array(Float32), + `c_array_double` Array(Float64), + `c_string` String, + `c_boolean` Boolean, + `c_int8` Int8, + `c_int16` Int16, + `c_int32` Int32, + `c_int64` Int64, + `c_float32` Float32, + `c_float64` Float64, + `c_decimal` Decimal(9,4), + `c_date` Date, + `c_datetime` DateTime64, + `c_nullable` Nullable(Int32), + `c_lowcardinality` LowCardinality(String), + `c_nested` Nested + ( + `int` UInt32, + `double` Float64, + `string` String + ), + `c_int128` Int128, + `c_uint128` UInt128, + `c_int256` Int256, + `c_uint256` UInt256, + `c_point` Point, + `c_ring` Ring +)engine=Memory; +""" + +insert_sql = """ +insert into `default`.source_table +( + `id`, + `c_map`, + `c_array_string`, + `c_array_short`, + `c_array_int`, + `c_array_long`, + `c_array_float`, + `c_array_double`, + `c_string`, + `c_boolean`, + `c_int8`, + `c_int16`, + `c_int32`, + `c_int64`, + `c_float32`, + `c_float64`, + `c_decimal`, + `c_date`, + `c_datetime`, + `c_nullable`, + `c_lowcardinality`, + `c_nested.int`, + `c_nested.double`, + `c_nested.string`, + `c_int128`, + `c_uint128`, + `c_int256`, + `c_uint256`, + `c_point`, + `c_ring` +) +values +(?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?) +""" + +compare_sql = """ +select + %s + from ( + select * from default.source_table +union all + select * from default.sink_table + ) +group by %s +having count(*) < 2 +""" \ No newline at end of file diff --git a/groot-tests/test-e2e-kafka/src/test/java/com/geedgenetworks/test/e2e/kafka/KafkaConnectorIT.java b/groot-tests/test-e2e-kafka/src/test/java/com/geedgenetworks/test/e2e/kafka/KafkaConnectorIT.java deleted file mode 100644 index 56108c5..0000000 --- a/groot-tests/test-e2e-kafka/src/test/java/com/geedgenetworks/test/e2e/kafka/KafkaConnectorIT.java +++ /dev/null @@ -1,375 +0,0 @@ -package com.geedgenetworks.test.e2e.kafka; - -import com.geedgenetworks.core.types.StructType; -import com.geedgenetworks.core.types.Types; -import com.geedgenetworks.formats.json.JsonSerializer; -import com.geedgenetworks.test.common.TestResource; -import com.geedgenetworks.test.common.TestSuiteBase; -import com.geedgenetworks.test.common.container.TestContainer; -import com.geedgenetworks.test.common.container.TestContainerId; -import com.geedgenetworks.test.common.junit.DisabledOnContainer; -import com.google.common.collect.Lists; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; -import org.apache.kafka.clients.admin.AdminClientConfig; -import org.apache.kafka.clients.consumer.*; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.config.SaslConfigs; -import org.apache.kafka.common.security.auth.SecurityProtocol; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.TestTemplate; -import org.testcontainers.containers.Container; -import org.testcontainers.containers.KafkaContainer; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.lifecycle.Startables;; -import org.testcontainers.utility.DockerImageName; -import org.testcontainers.utility.DockerLoggerFactory; -import org.testcontainers.utility.MountableFile; - -import java.io.IOException; -import java.time.Duration; -import java.util.*; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; -import java.util.stream.Stream; - -import static org.awaitility.Awaitility.await; - -@Slf4j -@DisabledOnContainer( - value = {TestContainerId.FLINK_1_17}, - disabledReason = "Override TestSuiteBase @DisabledOnContainer") -public class KafkaConnectorIT extends TestSuiteBase implements TestResource { - - private KafkaContainer kafkaContainer; - - private static final String KAFKA_IMAGE_NAME = "confluentinc/cp-kafka:7.4.0"; - private static final String KAFKA_HOST = "kafkaCluster"; - private KafkaProducer producer; - private static final String DEFAULT_TEST_TOPIC_SOURCE = "test_topic_source"; - private static final String DEFAULT_TEST_TOPIC_CONSUME_GROUP = "test-consume-group"; - - @Override - @BeforeAll - public void startUp() { - kafkaContainer = new KafkaContainer(DockerImageName.parse(KAFKA_IMAGE_NAME)) - .withNetwork(NETWORK) - .withNetworkAliases(KAFKA_HOST) - .withEnv("KAFKA_AUTO_CREATE_TOPICS_ENABLE", "true") - .withEnv("KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR", "1") - .withEnv("KAFKA_LISTENER_SECURITY_PROTOCOL_MAP", "PLAINTEXT:SASL_PLAINTEXT,BROKER:SASL_PLAINTEXT") - .withEnv("KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL", "PLAIN") - .withEnv("KAFKA_LISTENER_NAME_PLAINTEXT_SASL_ENABLED_MECHANISMS", "PLAIN") - .withEnv("KAFKA_LISTENER_NAME_BROKER_SASL_ENABLED_MECHANISMS", "PLAIN") - // .withEnv("KAFKA_AUTHORIZER_CLASS_NAME", "kafka.security.authorizer.AclAuthorizer") - .withEnv("KAFKA_SUPER_USERS", "User:admin") - .withEnv("KAFKA_OPTS", "-Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf") - .withCopyFileToContainer(MountableFile.forClasspathResource("kafka_server_jaas.conf"), "/etc/kafka/kafka_server_jaas.conf") - .withCopyFileToContainer(MountableFile.forClasspathResource("kafka_client_jass_cli.properties"), "/etc/kafka/kafka_client_jass_cli.properties") - .withLogConsumer(new Slf4jLogConsumer(DockerLoggerFactory.getLogger(KAFKA_IMAGE_NAME))); - Startables.deepStart(Stream.of(kafkaContainer)).join(); - log.info("Kafka container started successfully"); - Awaitility.given() - .ignoreExceptions() - .atLeast(100, TimeUnit.MILLISECONDS) - .pollInterval(500, TimeUnit.MILLISECONDS) - .atMost(180, TimeUnit.SECONDS) - .untilAsserted(this::initKafkaProducer); - - log.info("Write 100 records to topic test_topic_source"); - generateTestData(DEFAULT_TEST_TOPIC_SOURCE,0, 100); - - - } - - @TestTemplate - public void testKafkaAsSourceConsume(TestContainer container) { - generateTestData("test_topic_json", 0, 10); - CompletableFuture.supplyAsync( - () -> { - try { - return container.executeJob("/kafka_source.yaml"); - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - }); - - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - String logs = container.getServerLogs(); - Assertions.assertEquals(StringUtils.countMatches(logs, "PrintSinkFunction"), 10); - }); - } - - @TestTemplate - public void testKafkaAsSourceConsumeErrorSchema(TestContainer container) { - generateTestData("test_topic_error_json", 0, 10); - CompletableFuture.supplyAsync( - () -> { - try { - Container.ExecResult execResult = container.executeJob("/kafka_source_error_schema.yaml"); - Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); - return execResult; - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - }); - - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - String logs = container.getServerLogs(); - Assertions.assertTrue(StringUtils.contains(logs, "NumberFormatException")); - }); - } - - @TestTemplate - public void testKafkaAsSink(TestContainer container) throws IOException, InterruptedException { - CompletableFuture.supplyAsync( - () -> { - try { - Container.ExecResult execResult = container.executeJob("/kafka_sink.yaml"); - Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); - return execResult; - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - }); - - - List data = Lists.newArrayList(); - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - data.addAll(getKafkaConsumerListData("test_sink_topic")); - Assertions.assertEquals(10, data.size()); // Check if all 10 records are consumed - }); - - } - - @TestTemplate - public void testKafkaAsSinkProducerQuota(TestContainer container) throws IOException, InterruptedException { - //Create topic with 3 partitions - executeShell("kafka-topics --create --topic SESSION-RECORD-QUOTA-TEST --bootstrap-server kafkaCluster:9092 --partitions 3 --replication-factor 1 --command-config /etc/kafka/kafka_client_jass_cli.properties"); - //Set producer quota to 5KB/s - executeShell("kafka-configs --bootstrap-server kafkaCluster:9092 --alter --add-config 'producer_byte_rate=5120' --entity-type users --entity-name admin --entity-type clients --entity-name SESSION-RECORD-QUOTA-TEST --command-config /etc/kafka/kafka_client_jass_cli.properties "); - - CompletableFuture.supplyAsync( - () -> { - try { - Container.ExecResult execResult = container.executeJob("/kafka_producer_quota.yaml"); - Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); - return execResult; - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - }); - - List data = Lists.newArrayList(); - await().atMost(300000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - data.addAll(getKafkaConsumerListData("SESSION-RECORD-QUOTA-TEST")); - Assertions.assertTrue(StringUtils.contains(container.getServerLogs(), "TimeoutException") && data.size()>1000); - }); - - } - - - - @TestTemplate - public void testKafkaAsSinkHandleErrorJsonFormat(TestContainer container) throws IOException, InterruptedException { - CompletableFuture. supplyAsync( - () -> { - try { - Container.ExecResult execResult = container.executeJob("/kafka_sink_handle_error_json_format.yaml"); - Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); - return execResult; - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - }); - - - List data = Lists.newArrayList(); - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - data.addAll(getKafkaConsumerListData("test_handle_error_json_format_topic")); - Assertions.assertTrue(StringUtils.contains(container.getServerLogs(), "UnsupportedOperationException")); - Assertions.assertEquals(0, data.size()); - }); - - - - } - - @TestTemplate - public void testKafkaSinkSkipErrorJsonFormat(TestContainer container) throws IOException, InterruptedException { - CompletableFuture.supplyAsync( - () -> { - try { - Container.ExecResult execResult = container.executeJob("/kafka_sink_skip_error_json_format.yaml"); - Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); - return execResult; - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - }); - - - List data = Lists.newArrayList(); - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - data.addAll(getKafkaConsumerListData("test_skip_error_json_format_topic")); - Assertions.assertTrue(StringUtils.contains(container.getServerLogs(), "NullPointerException")); - Assertions.assertEquals(0, data.size()); - }); - } - - - - private void generateTestData(String topic, int start, int end) { - StructType dataType = Types.parseStructType("id: int, client_ip: string, server_ip: string, flag: string"); - JsonSerializer serializer = new JsonSerializer(dataType); - for (int i = start; i < end; i++) { - Map row = Map - .of("id", i, - "client_ip", "192.168.40.12", - "server_ip", "8.8.8.8" , - "flag", Boolean.FALSE.booleanValue()); - ProducerRecord record = - new ProducerRecord<>(topic, serializer.serialize(row)); - producer.send(record); - } - - } - - - @AfterAll - @Override - public void tearDown() throws Exception { - if (producer != null) { - producer.close(); - } - if (kafkaContainer != null) { - kafkaContainer.close(); - } - - } - - private void initKafkaProducer() { - Properties properties = new Properties(); - String bootstrapServers = kafkaContainer.getBootstrapServers(); - properties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); - properties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - properties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - properties.put(AdminClientConfig.SECURITY_PROTOCOL_CONFIG, "SASL_PLAINTEXT"); - properties.put(SaslConfigs.SASL_MECHANISM, "PLAIN"); - properties.put(SaslConfigs.SASL_JAAS_CONFIG, "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"admin\" password=\"admin\";"); - producer = new KafkaProducer<>(properties); - } - - private Properties kafkaConsumerConfig(String consumeGroup) { - Properties properties = new Properties(); - properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaContainer.getBootstrapServers()); - properties.put(ConsumerConfig.GROUP_ID_CONFIG, consumeGroup); - properties.put(AdminClientConfig.SECURITY_PROTOCOL_CONFIG, "SASL_PLAINTEXT"); - properties.put(SaslConfigs.SASL_MECHANISM, "PLAIN"); - properties.put(SaslConfigs.SASL_JAAS_CONFIG, "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"admin\" password=\"admin\";"); - properties.put( - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, - OffsetResetStrategy.EARLIEST.toString().toLowerCase()); - properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - return properties; - } - - private Properties kafkaByteConsumerConfig() { - Properties props = new Properties(); - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaContainer.getBootstrapServers()); - props.put(ConsumerConfig.GROUP_ID_CONFIG, "test-consume-group"); - props.put( - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, - OffsetResetStrategy.EARLIEST.toString().toLowerCase()); - props.setProperty( - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, - ByteArrayDeserializer.class.getName()); - props.setProperty( - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, - ByteArrayDeserializer.class.getName()); - return props; - } - - private Map getKafkaConsumerData(String topicName) { - Map data = new HashMap<>(); - try (KafkaConsumer consumer = new KafkaConsumer<>(kafkaConsumerConfig(DEFAULT_TEST_TOPIC_CONSUME_GROUP))) { - consumer.subscribe(Arrays.asList(topicName)); - Map offsets = - consumer.endOffsets(Arrays.asList(new TopicPartition(topicName, 0))); - Long endOffset = offsets.entrySet().iterator().next().getValue(); - Long lastProcessedOffset = -1L; - - do { - ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); - for (ConsumerRecord record : records) { - if (lastProcessedOffset < record.offset()) { - data.put(record.key(), record.value()); - } - lastProcessedOffset = record.offset(); - } - } while (lastProcessedOffset < endOffset - 1); - } - return data; - } - - private List getKafkaConsumerListData(String topicName) { - List data = new ArrayList<>(); - try (KafkaConsumer consumer = new KafkaConsumer<>(kafkaConsumerConfig(DEFAULT_TEST_TOPIC_CONSUME_GROUP))) { - consumer.subscribe(Arrays.asList(topicName)); - Map offsets = - consumer.endOffsets(Arrays.asList(new TopicPartition(topicName, 0))); - Long endOffset = offsets.entrySet().iterator().next().getValue(); - Long lastProcessedOffset = -1L; - - do { - ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); - for (ConsumerRecord record : records) { - if (lastProcessedOffset < record.offset()) { - data.add(record.value()); - } - lastProcessedOffset = record.offset(); - } - } while (lastProcessedOffset < endOffset - 1); - } - return data; - } - - private void executeShell(String command) { - try { - Container.ExecResult result = kafkaContainer.execInContainer("/bin/sh", "-c", command); - log.info("Execute shell command result: {},{}", result.getStdout(), result.getStderr()); - - } catch (Exception e) { - log.error("Execute shell command error: {}", e.getMessage()); - } - } - -} diff --git a/groot-tests/test-e2e-kafka/src/test/java/com/geedgenetworks/test/e2e/kafka/KafkaIT.java b/groot-tests/test-e2e-kafka/src/test/java/com/geedgenetworks/test/e2e/kafka/KafkaIT.java new file mode 100644 index 0000000..1c40e01 --- /dev/null +++ b/groot-tests/test-e2e-kafka/src/test/java/com/geedgenetworks/test/e2e/kafka/KafkaIT.java @@ -0,0 +1,372 @@ +package com.geedgenetworks.test.e2e.kafka; + +import com.geedgenetworks.core.types.StructType; +import com.geedgenetworks.core.types.Types; +import com.geedgenetworks.formats.json.JsonSerializer; +import com.geedgenetworks.test.common.TestResource; +import com.geedgenetworks.test.common.TestSuiteBase; +import com.geedgenetworks.test.common.container.TestContainer; +import com.geedgenetworks.test.common.container.TestContainerId; +import com.geedgenetworks.test.common.junit.DisabledOnContainer; +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.consumer.*; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables;; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; +import org.testcontainers.utility.MountableFile; + +import java.io.IOException; +import java.time.Duration; +import java.util.*; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +import static org.awaitility.Awaitility.await; + +@Slf4j +@DisabledOnContainer( + value = {TestContainerId.FLINK_1_17}, + disabledReason = "Override TestSuiteBase @DisabledOnContainer") +public class KafkaIT extends TestSuiteBase implements TestResource { + + private KafkaContainer kafkaContainer; + + private static final String KAFKA_IMAGE_NAME = "confluentinc/cp-kafka:7.4.0"; + private static final String KAFKA_HOST = "kafkaCluster"; + private KafkaProducer producer; + private static final String DEFAULT_TEST_TOPIC_SOURCE = "test_topic_source"; + private static final String DEFAULT_TEST_TOPIC_CONSUME_GROUP = "test-consume-group"; + + @Override + @BeforeAll + public void startUp() { + kafkaContainer = new KafkaContainer(DockerImageName.parse(KAFKA_IMAGE_NAME)) + .withNetwork(NETWORK) + .withNetworkAliases(KAFKA_HOST) + .withEnv("KAFKA_AUTO_CREATE_TOPICS_ENABLE", "true") + .withEnv("KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR", "1") + .withEnv("KAFKA_LISTENER_SECURITY_PROTOCOL_MAP", "PLAINTEXT:SASL_PLAINTEXT,BROKER:SASL_PLAINTEXT") + .withEnv("KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL", "PLAIN") + .withEnv("KAFKA_LISTENER_NAME_PLAINTEXT_SASL_ENABLED_MECHANISMS", "PLAIN") + .withEnv("KAFKA_LISTENER_NAME_BROKER_SASL_ENABLED_MECHANISMS", "PLAIN") + // .withEnv("KAFKA_AUTHORIZER_CLASS_NAME", "kafka.security.authorizer.AclAuthorizer") + .withEnv("KAFKA_SUPER_USERS", "User:admin") + .withEnv("KAFKA_OPTS", "-Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf") + .withCopyFileToContainer(MountableFile.forClasspathResource("kafka_server_jaas.conf"), "/etc/kafka/kafka_server_jaas.conf") + .withCopyFileToContainer(MountableFile.forClasspathResource("kafka_client_jass_cli.properties"), "/etc/kafka/kafka_client_jass_cli.properties") + .withLogConsumer(new Slf4jLogConsumer(DockerLoggerFactory.getLogger(KAFKA_IMAGE_NAME))); + Startables.deepStart(Stream.of(kafkaContainer)).join(); + log.info("Kafka container started successfully"); + Awaitility.given() + .ignoreExceptions() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(180, TimeUnit.SECONDS) + .untilAsserted(this::initKafkaProducer); + + log.info("Write 100 records to topic test_topic_source"); + generateTestData(DEFAULT_TEST_TOPIC_SOURCE,0, 100); + + + } + + @TestTemplate + public void testKafkaAsSourceConsume(TestContainer container) { + generateTestData("test_topic_json", 0, 10); + CompletableFuture.supplyAsync( + () -> { + try { + return container.executeJob("/kafka_source.yaml"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + String logs = container.getServerLogs(); + Assertions.assertEquals(StringUtils.countMatches(logs, "PrintSinkFunction"), 10); + }); + } + + @TestTemplate + public void testKafkaAsSourceConsumeErrorSchema(TestContainer container) { + generateTestData("test_topic_error_json", 0, 10); + CompletableFuture.supplyAsync( + () -> { + try { + Container.ExecResult execResult = container.executeJob("/kafka_source_error_schema.yaml"); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + return execResult; + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + String logs = container.getServerLogs(); + Assertions.assertTrue(StringUtils.contains(logs, "NumberFormatException")); + }); + } + + @TestTemplate + public void testKafkaAsSink(TestContainer container) throws IOException, InterruptedException { + CompletableFuture.supplyAsync( + () -> { + try { + Container.ExecResult execResult = container.executeJob("/kafka_sink.yaml"); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + return execResult; + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + + List data = Lists.newArrayList(); + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + data.addAll(getKafkaConsumerListData("test_sink_topic")); + Assertions.assertEquals(10, data.size()); // Check if all 10 records are consumed + }); + + } + + @TestTemplate + public void testKafkaAsSinkProducerQuota(TestContainer container) throws IOException, InterruptedException { + //Create topic with 3 partitions + executeShell("kafka-topics --create --topic SESSION-RECORD-QUOTA-TEST --bootstrap-server kafkaCluster:9092 --partitions 3 --replication-factor 1 --command-config /etc/kafka/kafka_client_jass_cli.properties"); + //Set producer quota to 5KB/s + executeShell("kafka-configs --bootstrap-server kafkaCluster:9092 --alter --add-config 'producer_byte_rate=5120' --entity-type users --entity-name admin --entity-type clients --entity-name SESSION-RECORD-QUOTA-TEST --command-config /etc/kafka/kafka_client_jass_cli.properties "); + + CompletableFuture.supplyAsync( + () -> { + try { + Container.ExecResult execResult = container.executeJob("/kafka_producer_quota.yaml"); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + return execResult; + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + List data = Lists.newArrayList(); + await().atMost(300000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + data.addAll(getKafkaConsumerListData("SESSION-RECORD-QUOTA-TEST")); + Assertions.assertTrue(StringUtils.contains(container.getServerLogs(), "TimeoutException") && data.size()>1000); + }); + + } + + + + @TestTemplate + public void testKafkaAsSinkHandleErrorJsonFormat(TestContainer container) throws IOException, InterruptedException { + CompletableFuture. supplyAsync( + () -> { + try { + Container.ExecResult execResult = container.executeJob("/kafka_sink_handle_error_json_format.yaml"); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + return execResult; + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + + List data = Lists.newArrayList(); + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + data.addAll(getKafkaConsumerListData("test_handle_error_json_format_topic")); + Assertions.assertTrue(StringUtils.contains(container.getServerLogs(), "UnsupportedOperationException")); + Assertions.assertEquals(0, data.size()); + }); + + + + } + + @TestTemplate + public void testKafkaSinkSkipErrorJsonFormat(TestContainer container) throws IOException, InterruptedException { + CompletableFuture.supplyAsync( + () -> { + try { + Container.ExecResult execResult = container.executeJob("/kafka_sink_skip_error_json_format.yaml"); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + return execResult; + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + + List data = Lists.newArrayList(); + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + data.addAll(getKafkaConsumerListData("test_skip_error_json_format_topic")); + Assertions.assertTrue(StringUtils.contains(container.getServerLogs(), "NullPointerException")); + Assertions.assertEquals(0, data.size()); + }); + } + + private void generateTestData(String topic, int start, int end) { + StructType dataType = Types.parseStructType("id: int, client_ip: string, server_ip: string, flag: string"); + JsonSerializer serializer = new JsonSerializer(dataType); + for (int i = start; i < end; i++) { + Map row = Map + .of("id", i, + "client_ip", "192.168.40.12", + "server_ip", "8.8.8.8" , + "flag", Boolean.FALSE.booleanValue()); + ProducerRecord record = + new ProducerRecord<>(topic, serializer.serialize(row)); + producer.send(record); + } + + } + + + @AfterAll + @Override + public void tearDown() throws Exception { + if (producer != null) { + producer.close(); + } + if (kafkaContainer != null) { + kafkaContainer.close(); + } + + } + + private void initKafkaProducer() { + Properties properties = new Properties(); + String bootstrapServers = kafkaContainer.getBootstrapServers(); + properties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + properties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + properties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + properties.put(AdminClientConfig.SECURITY_PROTOCOL_CONFIG, "SASL_PLAINTEXT"); + properties.put(SaslConfigs.SASL_MECHANISM, "PLAIN"); + properties.put(SaslConfigs.SASL_JAAS_CONFIG, "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"admin\" password=\"admin\";"); + producer = new KafkaProducer<>(properties); + } + + private Properties kafkaConsumerConfig(String consumeGroup) { + Properties properties = new Properties(); + properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaContainer.getBootstrapServers()); + properties.put(ConsumerConfig.GROUP_ID_CONFIG, consumeGroup); + properties.put(AdminClientConfig.SECURITY_PROTOCOL_CONFIG, "SASL_PLAINTEXT"); + properties.put(SaslConfigs.SASL_MECHANISM, "PLAIN"); + properties.put(SaslConfigs.SASL_JAAS_CONFIG, "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"admin\" password=\"admin\";"); + properties.put( + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + OffsetResetStrategy.EARLIEST.toString().toLowerCase()); + properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + return properties; + } + + private Properties kafkaByteConsumerConfig() { + Properties props = new Properties(); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaContainer.getBootstrapServers()); + props.put(ConsumerConfig.GROUP_ID_CONFIG, "test-consume-group"); + props.put( + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + OffsetResetStrategy.EARLIEST.toString().toLowerCase()); + props.setProperty( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName()); + props.setProperty( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName()); + return props; + } + + private Map getKafkaConsumerData(String topicName) { + Map data = new HashMap<>(); + try (KafkaConsumer consumer = new KafkaConsumer<>(kafkaConsumerConfig(DEFAULT_TEST_TOPIC_CONSUME_GROUP))) { + consumer.subscribe(Arrays.asList(topicName)); + Map offsets = + consumer.endOffsets(Arrays.asList(new TopicPartition(topicName, 0))); + Long endOffset = offsets.entrySet().iterator().next().getValue(); + Long lastProcessedOffset = -1L; + + do { + ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); + for (ConsumerRecord record : records) { + if (lastProcessedOffset < record.offset()) { + data.put(record.key(), record.value()); + } + lastProcessedOffset = record.offset(); + } + } while (lastProcessedOffset < endOffset - 1); + } + return data; + } + + private List getKafkaConsumerListData(String topicName) { + List data = new ArrayList<>(); + try (KafkaConsumer consumer = new KafkaConsumer<>(kafkaConsumerConfig(DEFAULT_TEST_TOPIC_CONSUME_GROUP))) { + consumer.subscribe(Arrays.asList(topicName)); + Map offsets = + consumer.endOffsets(Arrays.asList(new TopicPartition(topicName, 0))); + Long endOffset = offsets.entrySet().iterator().next().getValue(); + Long lastProcessedOffset = -1L; + + do { + ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); + for (ConsumerRecord record : records) { + if (lastProcessedOffset < record.offset()) { + data.add(record.value()); + } + lastProcessedOffset = record.offset(); + } + } while (lastProcessedOffset < endOffset - 1); + } + return data; + } + + private void executeShell(String command) { + try { + Container.ExecResult result = kafkaContainer.execInContainer("/bin/sh", "-c", command); + log.info("Execute shell command result: {},{}", result.getStdout(), result.getStderr()); + + } catch (Exception e) { + log.error("Execute shell command error: {}", e.getMessage()); + } + } + +} diff --git a/pom.xml b/pom.xml index 5e2ebe0..7c36d91 100644 --- a/pom.xml +++ b/pom.xml @@ -84,6 +84,30 @@ + + + nexus3 + Team Nexus Repository + http://192.168.40.153:8081/repository/public/ + + + cloudera + https://repository.cloudera.com/artifactory/cloudera-repos/ + + + + + + true + platform-releases + http://192.168.40.153:8081/repository/platform-release/ + + + platform-snapshots + http://192.168.40.153:8081/repository/platform-snapshot/ + + + @@ -967,28 +991,6 @@ - - - nexus3 - Team Nexus Repository - http://192.168.40.153:8081/repository/public/ - - - cloudera - https://repository.cloudera.com/artifactory/cloudera-repos/ - - - - - true - platform-releases - http://192.168.40.153:8081/repository/platform-release/ - - - platform-snapshots - http://192.168.40.153:8081/repository/platform-snapshot/ - - -- cgit v1.2.3 From 9b0d191d86574e4246f192511e0bbd5000169322 Mon Sep 17 00:00:00 2001 From: wangkuan Date: Thu, 1 Aug 2024 15:25:42 +0800 Subject: [fix][core]修复mean函数必须配置parameters的问题 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../core/udf/UnixTimestampConverter.java | 3 --- .../com/geedgenetworks/core/udf/udaf/Mean.java | 2 +- .../core/udf/test/aggregate/MeanTest.java | 22 ++++++++++++++++++++++ 3 files changed, 23 insertions(+), 4 deletions(-) diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java index a8171b3..bdb41e0 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java @@ -32,9 +32,6 @@ public class UnixTimestampConverter implements ScalarFunction { if(udfContext.getLookup_fields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); } - if(udfContext.getParameters() == null){ - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function must contain parameters"); - } if(!udfContext.getParameters().containsKey("precision")){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must containkey precision"); } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java index 380f598..88e4be6 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java @@ -29,7 +29,7 @@ public class Mean implements AggregateFunction { else { outputField = lookupField; } - if(!udfContext.getParameters().isEmpty()) { + if(udfContext.getParameters()!= null && !udfContext.getParameters().isEmpty()) { precision = Integer.parseInt(udfContext.getParameters().getOrDefault("precision", "-1").toString()); if (precision > 0) { StringBuilder pattern = new StringBuilder("#."); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java index 48c4e0f..807b7db 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java @@ -41,6 +41,7 @@ public class MeanTest { Integer[] intArr2 = new Integer[]{1, 6, 3}; excute(intArr1, 0); excute2(intArr2, 2); + excute3(intArr1); } private static void excute(Number[] arr,int precision) throws ParseException { @@ -93,5 +94,26 @@ public class MeanTest { assertEquals(NumberFormat.getInstance().parse((result.getMetricsFields().get("field_mean").toString())),NumberFormat.getInstance().parse("3.33")); } + private static void excute3(Number[] arr) throws ParseException { + UDFContext udfContext = new UDFContext(); + udfContext.setLookup_fields(List.of("field")); + udfContext.setOutput_fields(Collections.singletonList("field_mean")); + Mean mean = new Mean(); + Map metricsFields = new HashMap<>(); + Accumulator accumulator = new Accumulator(); + accumulator.setMetricsFields(metricsFields); + Accumulator agg = mean.open(udfContext,accumulator); + + for (Number o : arr) { + Event event = new Event(); + Map extractedFields = new HashMap<>(); + extractedFields.put("field", o); + event.setExtractedFields(extractedFields); + agg = mean.add(event, agg); + + } + Accumulator result = mean.getResult(agg); + assertEquals(NumberFormat.getInstance().parse((result.getMetricsFields().get("field_mean").toString())),NumberFormat.getInstance().parse("2.5")); + } } \ No newline at end of file -- cgit v1.2.3 From 80c4ca2f3f0662e0c702fa9ff12a03de85abc73d Mon Sep 17 00:00:00 2001 From: wangkuan Date: Thu, 1 Aug 2024 18:22:54 +0800 Subject: [improve][core]aggregate processor增加window_timestamp_field配置 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- config/grootstream_job_example.yaml | 1 + .../com/geedgenetworks/common/config/AggregateConfigOptions.java | 6 ++++++ .../src/main/java/com/geedgenetworks/core/pojo/AggregateConfig.java | 3 +-- .../core/processor/aggregate/ProcessWindowFunctionImpl.java | 4 +++- 4 files changed, 11 insertions(+), 3 deletions(-) diff --git a/config/grootstream_job_example.yaml b/config/grootstream_job_example.yaml index 74239c9..4726af0 100644 --- a/config/grootstream_job_example.yaml +++ b/config/grootstream_job_example.yaml @@ -24,6 +24,7 @@ processing_pipelines: output_fields: group_by_fields: [server_ip,server_port] window_type: tumbling_processing_time # tumbling_event_time,sliding_processing_time,sliding_event_time + window_timestamp_field: recv_time window_size: 60 window_slide: 10 #滑动窗口步长 functions: diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/AggregateConfigOptions.java b/groot-common/src/main/java/com/geedgenetworks/common/config/AggregateConfigOptions.java index 3998a3b..0b0379d 100644 --- a/groot-common/src/main/java/com/geedgenetworks/common/config/AggregateConfigOptions.java +++ b/groot-common/src/main/java/com/geedgenetworks/common/config/AggregateConfigOptions.java @@ -5,6 +5,8 @@ import com.geedgenetworks.common.udf.UDFContext; import java.util.List; +import static com.geedgenetworks.common.Event.WINDOW_START_TIMESTAMP; + public interface AggregateConfigOptions { Option TYPE = Options.key("type") .stringType() @@ -46,4 +48,8 @@ public interface AggregateConfigOptions { .noDefaultValue() .withDescription("The size of sliding window."); + Option WINDOW_TIMESTAMP_FIELD = Options.key("window_timestamp_field") + .stringType() + .noDefaultValue() + .withDescription("which field to be set the start time of window."); } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/pojo/AggregateConfig.java b/groot-core/src/main/java/com/geedgenetworks/core/pojo/AggregateConfig.java index 8cccbbd..d3cbaac 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/pojo/AggregateConfig.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/pojo/AggregateConfig.java @@ -13,10 +13,9 @@ public class AggregateConfig extends ProcessorConfig { private List group_by_fields; - private String timestamp_field; + private String window_timestamp_field; private String window_type; private Integer window_size; - private Integer max_out_of_orderness; private Integer window_slide; private List functions; diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/ProcessWindowFunctionImpl.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/ProcessWindowFunctionImpl.java index eaa712a..cd5c485 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/ProcessWindowFunctionImpl.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/ProcessWindowFunctionImpl.java @@ -44,7 +44,9 @@ public class ProcessWindowFunctionImpl extends org.apache.flink.streaming.api.fu internalMetrics.incrementOutEvents(); internalMetrics.incrementErrorEvents(accumulator.getErrorCount()); internalMetrics.incrementInEvents(accumulator.getInEvents()); - + if (aggregateConfig.getWindow_timestamp_field() != null) { + event.getExtractedFields().put(aggregateConfig.getWindow_timestamp_field(), context.window().getStart()); + } if (aggregateConfig.getOutput_fields() != null && !aggregateConfig.getOutput_fields().isEmpty()) { event.setExtractedFields( -- cgit v1.2.3 From ad9dfd01f268581ca6780d20a4c6601f8998d42c Mon Sep 17 00:00:00 2001 From: doufenghu Date: Thu, 1 Aug 2024 19:27:35 +0800 Subject: [docs][core] add aggregate processor documents. --- docs/connector/connector.md | 16 +- docs/filter/aviator.md | 2 +- docs/processor/aggregate-processor.md | 71 +++++++++ docs/processor/udaf.md | 149 ++++++++++++++++++ docs/processor/udf.md | 74 ++++----- groot-common/src/main/resources/udf.plugins | 4 +- .../geedgenetworks/example/GrootStreamExample.java | 2 +- .../examples/inline_to_print_with_aggregation.yaml | 41 +++++ ...sion_record_mock_to_print_with_aggregation.yaml | 167 +++++++++++++++++++++ 9 files changed, 478 insertions(+), 48 deletions(-) create mode 100644 docs/processor/aggregate-processor.md create mode 100644 docs/processor/udaf.md create mode 100644 groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_with_aggregation.yaml create mode 100644 groot-examples/end-to-end-example/src/main/resources/examples/session_record_mock_to_print_with_aggregation.yaml diff --git a/docs/connector/connector.md b/docs/connector/connector.md index 08ec673..1123385 100644 --- a/docs/connector/connector.md +++ b/docs/connector/connector.md @@ -19,14 +19,14 @@ sources: ${prop_key}: ${prop_value} ``` -| Name | Type | Required | Default | Description | -|--------------------------|---------------|----------|---------|--------------------------------------------------------------------------------------------------------------------------| -| type | String | Yes | (none) | The type of the source connector. The `SourceTableFactory` will use this value as identifier to create source connector. | -| schema | Map | No | (none) | The source table schema, config through fields or local_file or url. | -| watermark_timestamp | String | No | (none) | watermark timestamp field name, if need use eventTime. | -| watermark_timestamp_unit | String | No | ms | watermark field timestamp unit, options:ms(milliseconds),s(seconds). is required if watermark_timestamp is not none. | -| watermark_lag | Long | No | (none) | watermark out-of-order milliseconds. is required if watermark_timestamp is not none. | -| properties | Map of String | Yes | (none) | The source connector customize properties, more details see the [Source](source) documentation. | +| Name | Type | Required | Default | Description | +|--------------------------|---------------|----------|---------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| type | String | Yes | (none) | The type of the source connector. The `SourceTableFactory` will use this value as identifier to create source connector. | +| schema | Map | No | (none) | The source table schema, config through fields or local_file or url. | +| watermark_timestamp | String | No | (none) | Specify the field name as the watermark field. It is used to track event time and generate watermarks. | +| watermark_timestamp_unit | String | No | ms | The watermark field timestamp unit. The optional values are `ms`, `s`. | +| watermark_lag | Long | No | (none) | The watermark out-of-order milliseconds (Allowed Latenness). It defines the maximum amount of time (in milliseconds) by which events can be late but still be considered for processing. | +| properties | Map of String | Yes | (none) | The source connector customize properties, more details see the [Source](source) documentation. | ## Schema Field Projection diff --git a/docs/filter/aviator.md b/docs/filter/aviator.md index 54fe24b..e7f6c2b 100644 --- a/docs/filter/aviator.md +++ b/docs/filter/aviator.md @@ -30,7 +30,7 @@ This example read data from inline source and print to console. It will filter t filters: # [object] Define filter operator filter_operator: # [object] AviatorFilter operator name - type: com.geedgenetworks.core.filter.AviatorFilter + type: aviator properties: expression: event.server_ip == '8.8.8.8' || event.decoded_as == 'HTTP' # [string] Aviator expression, it return true or false. diff --git a/docs/processor/aggregate-processor.md b/docs/processor/aggregate-processor.md new file mode 100644 index 0000000..d9bcdb0 --- /dev/null +++ b/docs/processor/aggregate-processor.md @@ -0,0 +1,71 @@ +# Aggregate Processor + +> Processing pipelines for aggregate processor + +## Description + +Aggregate processor is used to aggregate the data from source to sink. It is a part of the processing pipeline. It can be used in the pre-processing, processing, and post-processing pipeline. Each processor can assemble UDAFs(User-defined Aggregate functions) into a pipeline. +Within the pipeline, events are processed by each Function in order, top‑>down. The UDAF usage detail can be found in [UDAF](udaf.md). + +## Options +Note:Default will output internal fields `__window_start_timestamp` and `__window_end_timestamp` if not set output_fields. + +| name | type | required | default value | +|--------------------------|--------|----------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| type | String | Yes | The type of the processor, now only support `com.geedgenetworks.core.processor.projection.AggregateProcessor` | +| output_fields | Array | No | Array of String. The list of fields that need to be kept. Fields not in the list will be removed. | +| remove_fields | Array | No | Array of String. The list of fields that need to be removed. | +| group_by_fields | Array | yes | Array of String. The list of fields that need to be grouped. | +| window_type | String | yes | The type of window, now only support `tumbling_processing_time`, `tumbling_event_time`, `sliding_processing_time`, `sliding_event_time`. if window_type is `tumbling/sliding_event_time,` you need to set watermark. | +| window_size | Long | yes | The duration of the window in seconds. | +| window_slide | Long | yes | The duration of the window slide in seconds. | +| window_timestamp_field | String | No | Set the output timestamp field name, with the unit in seconds. It is mapped to the internal field __window_start_timestamp. | +| functions | Array | No | Array of Object. The list of functions that need to be applied to the data. | + +## Usage Example + +This example use aggregate processor to aggregate the fields `received_bytes` by `client_ip` and using NUMBER_SUM function to sum all `received_bytes` in 10 seconds window. + +```yaml +sources: + inline_source: + type: inline + properties: + data: '[{"tcp_rtt_ms":128,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.ct.cn","http_url":"www.ct.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.11.22.22","server_ip":"8.8.8.8","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":5575,"sent_pkts":97,"sent_bytes":5892,"received_pkts":250,"received_bytes":333931},{"tcp_rtt_ms":256,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.abc.cn","http_url":"www.cabc.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.168.10.198","server_ip":"4.4.4.4","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":2575,"sent_pkts":197,"sent_bytes":5892,"received_pkts":350,"received_bytes":533931}]' + format: json + json.ignore.parse.errors: false + + +processing_pipelines: + aggregate_processor: + type: aggregate + group_by_fields: [ client_ip ] + window_type: tumbling_processing_time + window_size: 10 + functions: + - function: NUMBER_SUM + lookup_fields: [ received_bytes ] + output_fields: [ received_bytes_sum ] + +sinks: + print_sink: + type: print + properties: + format: json + mode: log_warn + +application: + env: + name: example-inline-to-print-with-aggregation + parallelism: 3 + pipeline: + object-reuse: true + topology: + - name: inline_source + downstream: [aggregate_processor] + - name: aggregate_processor + downstream: [ print_sink ] + - name: print_sink + downstream: [] +``` + diff --git a/docs/processor/udaf.md b/docs/processor/udaf.md new file mode 100644 index 0000000..e22846f --- /dev/null +++ b/docs/processor/udaf.md @@ -0,0 +1,149 @@ +# UDAF + +> The functions for aggregate processors. + +## Function of content + +- [Collect List](#Collect-List) +- [Collect Set](#Collect-Set) +- [First Value](#First-Value) +- [Last Value](#Last-Value) +- [Long Count](#Long-Count) +- [MEAN](#Mean) +- [Number SUM](#Number-SUM) + + +## Description + +UDF(User Defined Aggregate Function) is used to extend the functions of aggregate processor. It is a part of the processing pipeline. It can be used in the pre-processing, processing, and post-processing pipeline. Each processor can assemble UDAFs into a pipeline. Within the pipeline, events are processed by each Function in order, top‑>down. +The deference between UDF and UDAF is: +- UDF is used to process each event, and the output is also an event. UDAF is used to process a group of events, and the output is also an event. +- A UDF is designed to perform a transformation or calculation on a single event. A UDAF is designed to perform an aggregation over a group of events, such as summing values, calculating an average, or finding a maximum. It processes multiple events of input data and produces a single aggregated result. + +## UDAF Definition + The UDAF basic properties are the same as UDF, such as `name`, `event`, `context`,more detail can be found in [UDF](udf.md). But Aggregate Processor have some methods to process the data is: +- `void add()`: Add a new event to the aggregation. +- `void getResult()`: Get the result of the aggregation. + +## Functions + +### Collect List + +COLLECT_LIST is used to collect the value of the field in the group of events. + +```COLLECT_LIST(filter, lookup_fields, output_fields)``` + +- filter: optional +- lookup_fields: required. Now only support one field. +- output_fields: optional. If not set, the output field name is `lookup_field_name`. + +### Example + +```yaml +- function: COLLECT_LIST + lookup_fields: [client_ip] + output_fields: [client_ip_list] +``` + +### Collect Set + +COLLECT_SET is used to collect the unique value of the field in the group of events. + +```COLLECT_SET(filter, lookup_fields, output_fields)``` + +- filter: optional +- lookup_fields: required. Now only support one field. +- output_fields: optional. If not set, the output field name is `lookup_field_name`. + +### Example + +```yaml +- function: COLLECT_SET + lookup_fields: [client_ip] + output_fields: [client_ip_set] +``` + +### First Value + +FIRST_VALUE is used to get the first value of the field in the group of events. + +```FIRST_VALUE(filter, lookup_fields, output_fields)``` +- filter: optional +- lookup_fields: required. Now only support one field. +- output_fields: optional. If not set, the output field name is `lookup_field_name`. + +### Example + +```yaml +- function: FIRST_VALUE + lookup_fields: [client_ip] + output_fields: [first_client_ip] +``` +### Last Value + +LAST_VALUE is used to get the last value of the field in the group of events. + +```LAST_VALUE(filter, lookup_fields, output_fields)``` +- filter: optional +- lookup_fields: required. Now only support one field. +- output_fields: optional. If not set, the output field name is `lookup_field_name`. + +### Example + +```yaml +- function: LAST_VALUE + lookup_fields: [client_ip] + output_fields: [last_client_ip] +``` + +### Long Count + +LONG_COUNT is used to count the number of events in the group of events. + +```LONG_COUNT(filter, lookup_fields, output_fields)``` +- filter: optional +- lookup_fields: optional. +- output_fields: required. + +### Example + +```yaml +- function: LONG_COUNT + output_fields: [sessions] +``` + +### Mean + +MEAN is used to calculate the mean value of the field in the group of events. The lookup field value must be a number. + +```MEAN(filter, lookup_fields, output_fields[, parameters])``` +- filter: optional +- lookup_fields: required. Now only support one field. +- output_fields: optional. If not set, the output field name is `lookup_field_name`. +- parameters: optional. + - precision: `` required. The precision of the mean value. Default is 2. + +### Example + +```yaml +- function: MEAN + lookup_fields: [received_bytes] + output_fields: [received_bytes_mean] +``` + +### Number SUM + +NUMBER_SUM is used to sum the value of the field in the group of events. The lookup field value must be a number. + +```NUMBER_SUM(filter, lookup_fields, output_fields)``` +- filter: optional +- lookup_fields: required. Now only support one field. +- output_fields: optional. If not set, the output field name is `lookup_field_name`. + +### Example + +```yaml +- function: NUMBER_SUM + lookup_fields: [received_bytes] + output_fields: [received_bytes_sum] +``` \ No newline at end of file diff --git a/docs/processor/udf.md b/docs/processor/udf.md index 2a705fd..cf305ef 100644 --- a/docs/processor/udf.md +++ b/docs/processor/udf.md @@ -1,7 +1,7 @@ # UDF > The functions for projection processors. -> + ## Function of content - [Asn Lookup](#asn-lookup) @@ -40,7 +40,7 @@ A UDF includes the following parts: name, event(processing data), context, evalu - open function: Initialize the resources used by the function. - close function: Release the resources used by the function. -### Functions +## Functions Function define common parameters: `filter`, `lookup_fields`, `output_fields`, `parameters`, and will return a Map value of the event. ``` FUNCTION_NAME(filter, lookup_fields, output_fields[, parameters])``` @@ -54,8 +54,8 @@ Asn lookup function is used to lookup the asn information by ip address. You nee - lookup_fields: required - output_fields: required - parameters: required -- kb_name: required. The name of the knowledge base. -- option: required. Now only support `IP_TO_ASN`. + - kb_name: required. The name of the knowledge base. + - option: required. Now only support `IP_TO_ASN`. Example: @@ -77,8 +77,8 @@ Base64 decode function is used to decode the base64 encoded string. - lookup_fields: not required - output_fields: required - parameters: required -- value_field: `` required. -- charset_field:`` optional. Default is `UTF-8`. + - value_field: `` required. + - charset_field:`` optional. Default is `UTF-8`. Example: @@ -99,7 +99,7 @@ Base64 encode function is commonly used to encode the binary data to base64 stri - lookup_fields: not required - output_fields: required - parameters: required -- value_field: `` required. + - value_field: `` required. Example: @@ -119,7 +119,7 @@ Current unix timestamp function is used to get the current unix timestamp. - lookup_fields: not required - output_fields: required - parameters: optional -- precision: `` optional. Default is `seconds`. Enum: `milliseconds`, `seconds`. + - precision: `` optional. Default is `seconds`. Enum: `milliseconds`, `seconds`. Example: @@ -139,7 +139,7 @@ Domain function is used to extract the domain from the url. - lookup_fields: required. Support more than one fields. All fields will be processed from left to right, and the result will be overwritten if the field processed value is not null. - output_fields: required - parameters: required -- option: `` required. Enum: `TOP_LEVEL_DOMAIN`, `FIRST_SIGNIFICANT_SUBDOMAIN`. + - option: `` required. Enum: `TOP_LEVEL_DOMAIN`, `FIRST_SIGNIFICANT_SUBDOMAIN`. #### Option @@ -182,7 +182,7 @@ Eval function is used to adds or removes fields from events by evaluating an val - lookup_fields: not required - output_fields: required - parameters: required -- value_expression: `` required. Enter a value expression to set the field’s value – this can be a constant. + - value_expression: `` required. Enter a value expression to set the field’s value – this can be a constant. Example 1: Add a field `ingestion_time` with value `recv_time`: @@ -213,10 +213,10 @@ Flatten the fields of nested structure to the top level. The new fields name are - lookup_fields: optional - output_fields: not required - parameters: optional -- prefix: `` optional. Prefix string for flattened field names. Default is empty. -- depth: `` optional. Number representing the nested levels to consider for flattening. Minimum 1. Default is `5`. -- delimiter: `` optional. The string used to join nested keys Default is `.`. -- json_string_keys: `` optional. The keys of the json string fields. It indicates keys that contain JSON strings and should be parsed and flattened. Default is empty. + - prefix: `` optional. Prefix string for flattened field names. Default is empty. + - depth: `` optional. Number representing the nested levels to consider for flattening. Minimum 1. Default is `5`. + - delimiter: `` optional. The string used to join nested keys Default is `.`. + - json_string_keys: `` optional. The keys of the json string fields. It indicates keys that contain JSON strings and should be parsed and flattened. Default is empty. Example 1: @@ -259,7 +259,7 @@ From unix timestamp function is used to convert the unix timestamp to date time - lookup_fields: required - output_fields: required - parameters: optional -- precision: `` optional. Default is `seconds`. Enum: `milliseconds`, `seconds`. + - precision: `` optional. Default is `seconds`. Enum: `milliseconds`, `seconds`. #### Precision @@ -303,16 +303,16 @@ GeoIP lookup function is used to lookup the geoip information by ip address. You - lookup_fields: required - output_fields: optional - parameters: required -- kb_name: `` required. The name of the knowledge base. -- option: `` required. Enum: `IP_TO_COUNTRY`, `IP_TO_PROVINCE`, `IP_TO_CITY`, `IP_TO_SUBDIVISION_ADDR`, `IP_TO_DETAIL`, `IP_TO_LATLNG`, `IP_TO_PROVIDER`, `IP_TO_JSON`, `IP_TO_OBJECT`. -- geolocation_field_mapping : `>` optional. The option is required when the option is `IP_TO_OBJECT`. The mapping of the geolocation fields. The key is the field name of the knowledge base , and the value is the field name of the event. -- COUNTRY: `` optional. -- PROVINCE: `` optional. -- CITY: `` optional. -- LONGITUDE: `` optional. -- LATITUDE: `` optional. -- ISP: `` optional. -- ORGANIZATION: `` optional. + - kb_name: `` required. The name of the knowledge base. + - option: `` required. Enum: `IP_TO_COUNTRY`, `IP_TO_PROVINCE`, `IP_TO_CITY`, `IP_TO_SUBDIVISION_ADDR`, `IP_TO_DETAIL`, `IP_TO_LATLNG`, `IP_TO_PROVIDER`, `IP_TO_JSON`, `IP_TO_OBJECT`. + - geolocation_field_mapping : `>` optional. The option is required when the option is `IP_TO_OBJECT`. The mapping of the geolocation fields. The key is the field name of the knowledge base , and the value is the field name of the event. + - COUNTRY: `` optional. + - PROVINCE: `` optional. + - CITY: `` optional. + - LONGITUDE: `` optional. + - LATITUDE: `` optional. + - ISP: `` optional. + - ORGANIZATION: `` optional. #### Option @@ -369,7 +369,7 @@ JSON extract function is used to extract the value from json string. - lookup_fields: required - output_fields: required - parameters: required -- value_expression: `` required. The json path expression. + - value_expression: `` required. The json path expression. Example: @@ -390,7 +390,7 @@ Path combine function is used to combine the file path. The path value can be co - lookup_fields: required - output_fields: required - parameters: required -- path: `` required. + - path: `` required. Example: @@ -411,11 +411,11 @@ Rename function is used to rename or reformat(e.g. by replacing character unders - lookup_fields: not required - output_fields: not required - parameters: required -- parent_fields: `` optional. Specify fields whose children will inherit the Rename fields and Rename expression operations. -- rename_fields: `Map` required. The key is the original field name, and the value is the new field name. -- current_field_name: `` required. The original field name. -- new_field_name: `` required. The new field name. -- rename_expression: `` optional. AviatorScript expression whose returned value will be used to rename fields. + - parent_fields: `` optional. Specify fields whose children will inherit the Rename fields and Rename expression operations. + - rename_fields: `Map` required. The key is the original field name, and the value is the new field name. + - current_field_name: `` required. The original field name. + - new_field_name: `` required. The new field name. + - rename_expression: `` optional. AviatorScript expression whose returned value will be used to rename fields. ``` A single Function can include both rename_fields (to rename specified field names) and rename_expression (to globally rename fields). However, the Rename fields strategy will execute first. @@ -462,7 +462,7 @@ Snowflake ID function is used to generate the snowflake id. The snowflake id is - lookup_fields: not required - output_fields: required - parameters: optional -- data_center_id_num: `` optional. Default is `0`, range is `0-31`. + - data_center_id_num: `` optional. Default is `0`, range is `0-31`. Example: @@ -482,9 +482,9 @@ String joiner function joins multiple string fields using a delimiter, prefix, a - lookup_fields: required. Support more than one fields. - output_fields: required - parameters: optional -- delimiter: `` optional. Default is `,`. -- prefix: `` optional. Default is empty string. -- suffix: `` optional. Default is empty string. + - delimiter: `` optional. Default is `,`. + - prefix: `` optional. Default is empty string. + - suffix: `` optional. Default is empty string. Example: @@ -507,7 +507,7 @@ Unix timestamp converter function is used to convert the unix timestamp precisio - lookup_fields: required - output_fields: required - parameters: required -- precision: `` required. Enum: `milliseconds`, `seconds`, `minutes`. The minutes precision is used to generate Unix timestamp, round it to the minute level, and output it in seconds format. + - precision: `` required. Enum: `milliseconds`, `seconds`, `minutes`. The minutes precision is used to generate Unix timestamp, round it to the minute level, and output it in seconds format. - Example: _`__timestamp` Internal field, from source ingestion time or current unix timestamp. diff --git a/groot-common/src/main/resources/udf.plugins b/groot-common/src/main/resources/udf.plugins index 0eb24cb..1b7fca4 100644 --- a/groot-common/src/main/resources/udf.plugins +++ b/groot-common/src/main/resources/udf.plugins @@ -18,4 +18,6 @@ com.geedgenetworks.core.udf.udaf.NumberSum com.geedgenetworks.core.udf.udaf.CollectList com.geedgenetworks.core.udf.udaf.CollectSet com.geedgenetworks.core.udf.udaf.LongCount -com.geedgenetworks.core.udf.udaf.Mean \ No newline at end of file +com.geedgenetworks.core.udf.udaf.Mean +com.geedgenetworks.core.udf.udaf.LastValue +com.geedgenetworks.core.udf.udaf.FirstValue \ No newline at end of file diff --git a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java index d927133..690f21c 100644 --- a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java +++ b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java @@ -13,7 +13,7 @@ import java.nio.file.Paths; public class GrootStreamExample { public static void main(String[] args) throws FileNotFoundException, URISyntaxException { - String configPath = args.length > 0 ? args[0] : "/examples/inline_to_print_test.yaml"; + String configPath = args.length > 0 ? args[0] : "/examples/session_record_mock_to_print_with_aggregation.yaml"; String configFile = getTestConfigFile(configPath); ExecuteCommandArgs executeCommandArgs = new ExecuteCommandArgs(); executeCommandArgs.setConfigFile(configFile); diff --git a/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_with_aggregation.yaml b/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_with_aggregation.yaml new file mode 100644 index 0000000..6f08be2 --- /dev/null +++ b/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_with_aggregation.yaml @@ -0,0 +1,41 @@ +sources: + inline_source: + type: inline + properties: + data: '[{"tcp_rtt_ms":128,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.ct.cn","http_url":"www.ct.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.11.22.22","server_ip":"8.8.8.8","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":5575,"sent_pkts":97,"sent_bytes":5892,"received_pkts":250,"received_bytes":333931},{"tcp_rtt_ms":256,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.abc.cn","http_url":"www.cabc.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.168.10.198","server_ip":"4.4.4.4","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":2575,"sent_pkts":197,"sent_bytes":5892,"received_pkts":350,"received_bytes":533931}]' + format: json + json.ignore.parse.errors: false + + + +processing_pipelines: + aggregate_processor: + type: aggregate + group_by_fields: [ client_ip ] + window_type: tumbling_processing_time + window_size: 10 + functions: + - function: NUMBER_SUM + lookup_fields: [ received_bytes] + output_fields: [ received_bytes_sum ] + +sinks: + print_sink: + type: print + properties: + format: json + mode: log_warn + +application: + env: + name: example-inline-to-print-with-aggregation + parallelism: 3 + pipeline: + object-reuse: true + topology: + - name: inline_source + downstream: [aggregate_processor] + - name: aggregate_processor + downstream: [ print_sink ] + - name: print_sink + downstream: [] \ No newline at end of file diff --git a/groot-examples/end-to-end-example/src/main/resources/examples/session_record_mock_to_print_with_aggregation.yaml b/groot-examples/end-to-end-example/src/main/resources/examples/session_record_mock_to_print_with_aggregation.yaml new file mode 100644 index 0000000..a3629c1 --- /dev/null +++ b/groot-examples/end-to-end-example/src/main/resources/examples/session_record_mock_to_print_with_aggregation.yaml @@ -0,0 +1,167 @@ +sources: # [object] Define connector source + mock_source: + type: mock + #watermark_timestamp: __timestamp + #watermark_timestamp_unit: s + #watermark_lag: 10000 + properties: + mock.desc.file.path: ./config/template/mock_schema/session_record_mock_desc.json + rows.per.second: 10 + +preprocessing_pipelines: + etl_processor: + type: projection + functions: + - function: SNOWFLAKE_ID + lookup_fields: [''] + output_fields: [log_id] + parameters: + data_center_id_num: 1 + - function: UNIX_TIMESTAMP_CONVERTER + lookup_fields: [ __timestamp ] + output_fields: [ recv_time ] + parameters: + precision: seconds + - function: SNOWFLAKE_ID + lookup_fields: [ '' ] + output_fields: [ session_id ] + parameters: + data_center_id_num: 2 + - function: EVAL + output_fields: [ ingestion_time ] + parameters: + value_expression: recv_time + + - function: DOMAIN + lookup_fields: [ http_host, ssl_sni, dtls_sni, quic_sni ] + output_fields: [ server_domain ] + parameters: + option: FIRST_SIGNIFICANT_SUBDOMAIN + + + - function: ASN_LOOKUP + lookup_fields: [ client_ip ] + output_fields: [ client_asn ] + parameters: + kb_name: tsg_ip_asn + option: IP_TO_ASN + + - function: ASN_LOOKUP + lookup_fields: [ server_ip ] + output_fields: [ server_asn ] + parameters: + kb_name: tsg_ip_asn + option: IP_TO_ASN + + - function: GEOIP_LOOKUP + lookup_fields: [ client_ip ] + output_fields: [] + parameters: + kb_name: tsg_ip_location + option: IP_TO_OBJECT + geolocation_field_mapping: + COUNTRY: client_country + PROVINCE: client_super_administrative_area + CITY: client_administrative_area + + - function: GEOIP_LOOKUP + lookup_fields: [ server_ip ] + output_fields: [] + parameters: + kb_name: tsg_ip_location + option: IP_TO_OBJECT + geolocation_field_mapping: + COUNTRY: server_country + PROVINCE: server_super_administrative_area + CITY: server_administrative_area + + - function: CURRENT_UNIX_TIMESTAMP + output_fields: [ processing_time ] + parameters: + precision: seconds + + - function: UNIX_TIMESTAMP_CONVERTER + lookup_fields: [recv_time] + output_fields: [recv_time] + parameters: + precision: seconds + interval: 60 + + +processing_pipelines: + aggregate_processor: + type: aggregate + group_by_fields: [recv_time, sled_ip] + window_type: tumbling_processing_time + window_size: 60 + functions: + - function: NUMBER_SUM + lookup_fields: [received_bytes, sent_bytes] + output_fields: [received_bytes_sum] + + - function: LONG_COUNT + lookup_fields: [received_bytes] + output_fields: [sessions] + + - function: MEAN + lookup_fields: [received_bytes] + output_fields: [received_bytes_mean] + parameters: + precision: 2 + + - function: FIRST_VALUE + lookup_fields: [ received_bytes ] + output_fields: [ received_bytes_first ] + + - function: LAST_VALUE + lookup_fields: [ received_bytes ] + output_fields: [ received_bytes_last ] + + - function: COLLECT_LIST + lookup_fields: [received_bytes] + output_fields: [received_bytes_set] + + +sinks: + print_sink: + type: print + properties: + mode: log_info + format: json + + kafka_sink: + type: kafka + properties: + topic: SESSION-RECORD + kafka.bootstrap.servers: 192.168.44.12:9094 + kafka.retries: 0 + kafka.linger.ms: 10 + kafka.request.timeout.ms: 30000 + kafka.batch.size: 262144 + kafka.buffer.memory: 134217728 + kafka.max.request.size: 10485760 + kafka.compression.type: snappy + kafka.security.protocol: SASL_PLAINTEXT + kafka.sasl.mechanism: PLAIN + kafka.sasl.jaas.config: 454f65ea6eef1256e3067104f82730e737b68959560966b811e7ff364116b03124917eb2b0f3596f14733aa29ebad9352644ce1a5c85991c6f01ba8a5e8f177a7ff0b2d3889a424249967b3870b50993d9644f239f0de82cdb13bdb502959e16afadffa49ef1e1d2b9c9b5113e619817 + format: json + json.ignore.parse.errors: false + log.failures.only: true + + +application: # [object] Define job configuration + env: + name: session_record_mock_to_print_with_aggregation + parallelism: 3 + shade.identifier: aes + pipeline: + object-reuse: true + topology: + - name: mock_source + downstream: [ etl_processor ] + - name: etl_processor + downstream: [ aggregate_processor ] + - name: aggregate_processor + downstream: [ print_sink ] + - name: print_sink + downstream: [] \ No newline at end of file -- cgit v1.2.3 From 6710ce6bf79836a99176fd75d1e15719a31cd02c Mon Sep 17 00:00:00 2001 From: wangkuan Date: Fri, 2 Aug 2024 10:16:58 +0800 Subject: [improve][core]优化部分函数校验逻辑 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java | 3 +++ groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java | 1 - groot-core/src/main/java/com/geedgenetworks/core/udf/Rename.java | 4 +++- 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java index 3153ef7..84c2c2a 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java @@ -24,6 +24,9 @@ public class Flatten implements ScalarFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + if(udfContext.getParameters()==null){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } prefix = udfContext.getParameters().getOrDefault("prefix", "").toString(); delimiter = udfContext.getParameters().getOrDefault("delimiter", ".").toString(); flattenKeys = new HashSet<>(); diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java index e48a503..874735d 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java @@ -31,7 +31,6 @@ public class PathCombine implements ScalarFunction { if (udfContext.getParameters() != null && !udfContext.getParameters().isEmpty()) { String paths = udfContext.getParameters().getOrDefault("path","").toString(); - // 使用逗号分隔项并转换为数组 if (!paths.isEmpty()) { List pathList; try { diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Rename.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Rename.java index ba9b4d2..6a77c3a 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/Rename.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Rename.java @@ -26,7 +26,9 @@ public class Rename implements ScalarFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - + if(udfContext.getParameters()==null ){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } String parentFields = udfContext.getParameters().getOrDefault("parent_fields", "").toString(); this.parentFields = new HashSet<>(); if (!parentFields.isEmpty()) { -- cgit v1.2.3 From fe1dc0328be1cc291660cc22a3048fe2800b52ea Mon Sep 17 00:00:00 2001 From: wangkuan Date: Fri, 2 Aug 2024 11:27:24 +0800 Subject: [improve][core][bootstrap]调整结构,合并Processor接口 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../bootstrap/execution/AbstractExecutor.java | 14 +++++--------- .../bootstrap/execution/AbstractProcessorExecutor.java | 14 +++++++------- .../java/com/geedgenetworks/core/processor/Processor.java | 14 ++++++++++++++ .../core/processor/aggregate/AggregateProcessor.java | 13 ++----------- .../core/processor/aggregate/AggregateProcessorImpl.java | 9 ++------- .../core/processor/projection/ProjectionProcessor.java | 11 ++--------- .../core/processor/projection/ProjectionProcessorImpl.java | 8 +++----- .../services/com.geedgenetworks.core.processor.Processor | 2 ++ ...dgenetworks.core.processor.aggregate.AggregateProcessor | 1 - ...enetworks.core.processor.projection.ProjectionProcessor | 1 - 10 files changed, 37 insertions(+), 50 deletions(-) create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/processor/Processor.java create mode 100644 groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.Processor delete mode 100644 groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.aggregate.AggregateProcessor delete mode 100644 groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.projection.ProjectionProcessor diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractExecutor.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractExecutor.java index 04a6a94..64c66b6 100644 --- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractExecutor.java +++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractExecutor.java @@ -2,6 +2,7 @@ package com.geedgenetworks.bootstrap.execution; import com.geedgenetworks.common.utils.ReflectionUtils; import com.geedgenetworks.core.filter.Filter; +import com.geedgenetworks.core.processor.Processor; import com.geedgenetworks.core.processor.aggregate.AggregateProcessor; import com.geedgenetworks.core.processor.projection.ProjectionProcessor; import com.typesafe.config.Config; @@ -20,8 +21,7 @@ public abstract class AbstractExecutor protected final Config operatorConfig; protected final Map operatorMap; protected final Map filterMap = new HashMap<>(); - protected final Map projectionProcessorMap = new HashMap<>(); - protected final Map aggregateProcessorMap = new HashMap<>(); + protected final Map processorMap = new HashMap<>(); protected AbstractExecutor(List jarPaths, Config operatorConfig) { this.operatorConfig = operatorConfig; @@ -30,13 +30,9 @@ public abstract class AbstractExecutor for (Filter filter : filters) { this.filterMap.put(filter.type(), filter); } - ServiceLoader projectionProcessors = ServiceLoader.load(ProjectionProcessor.class); - for (ProjectionProcessor projectionProcessor : projectionProcessors) { - this.projectionProcessorMap.put(projectionProcessor.type(), projectionProcessor); - } - ServiceLoader aggregateProcessors = ServiceLoader.load(AggregateProcessor.class); - for (AggregateProcessor aggregateProcessor : aggregateProcessors) { - this.aggregateProcessorMap.put(aggregateProcessor.type(), aggregateProcessor); + ServiceLoader processors = ServiceLoader.load(Processor.class); + for (Processor processor : processors) { + this.processorMap.put(processor.type(), processor); } } diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java index 42a4828..bd8b75c 100644 --- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java +++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java @@ -47,8 +47,8 @@ public abstract class AbstractProcessorExecutor extends AbstractExecutor { + + SingleOutputStreamOperator processorFunction( + SingleOutputStreamOperator singleOutputStreamOperator, + T processorConfig, ExecutionConfig config) + throws Exception; + String type(); +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessor.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessor.java index 9acf8fc..0846ffe 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessor.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessor.java @@ -1,16 +1,7 @@ package com.geedgenetworks.core.processor.aggregate; import com.geedgenetworks.core.pojo.AggregateConfig; -import com.geedgenetworks.common.Event; +import com.geedgenetworks.core.processor.Processor; +public interface AggregateProcessor extends Processor { -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; - -public interface AggregateProcessor { - - SingleOutputStreamOperator aggregateProcessorFunction( - SingleOutputStreamOperator singleOutputStreamOperator, - AggregateConfig aggregateConfig, ExecutionConfig config) - throws Exception; - String type(); } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java index 2f086b0..bc87c32 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java @@ -11,17 +11,12 @@ import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeW import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows; import org.apache.flink.streaming.api.windowing.time.Time; - import static com.geedgenetworks.common.Constants.*; -public class AggregateProcessorImpl implements AggregateProcessor { +public class AggregateProcessorImpl implements AggregateProcessor { @Override - public SingleOutputStreamOperator aggregateProcessorFunction( - SingleOutputStreamOperator grootEventSingleOutputStreamOperator, - AggregateConfig aggregateConfig, ExecutionConfig config) - throws Exception { - + public SingleOutputStreamOperator processorFunction(SingleOutputStreamOperator grootEventSingleOutputStreamOperator, AggregateConfig aggregateConfig, ExecutionConfig config) throws Exception { if (aggregateConfig.getParallelism() != 0) { switch (aggregateConfig.getWindow_type()) { case TUMBLING_PROCESSING_TIME: diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessor.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessor.java index 862ba5e..f15d481 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessor.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessor.java @@ -1,15 +1,8 @@ package com.geedgenetworks.core.processor.projection; -import com.geedgenetworks.common.Event; import com.geedgenetworks.core.pojo.ProjectionConfig; +import com.geedgenetworks.core.processor.Processor; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +public interface ProjectionProcessor extends Processor{ -public interface ProjectionProcessor { - - SingleOutputStreamOperator projectionProcessorFunction( - SingleOutputStreamOperator grootEventSingleOutputStreamOperator, - ProjectionConfig projectionConfig) - throws Exception; - String type(); } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java index 79b0e0d..6b46a7b 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java @@ -3,16 +3,14 @@ package com.geedgenetworks.core.processor.projection; import com.geedgenetworks.common.Event; import com.geedgenetworks.core.pojo.ProjectionConfig; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; public class ProjectionProcessorImpl implements ProjectionProcessor { - @Override - public SingleOutputStreamOperator projectionProcessorFunction( - SingleOutputStreamOperator grootEventSingleOutputStreamOperator, - ProjectionConfig projectionConfig) - throws Exception{ + @Override + public SingleOutputStreamOperator processorFunction(SingleOutputStreamOperator grootEventSingleOutputStreamOperator, ProjectionConfig projectionConfig, ExecutionConfig config) throws Exception { if (projectionConfig.getParallelism() != 0) { return grootEventSingleOutputStreamOperator .process(new ProjectionProcessFunction(projectionConfig)) diff --git a/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.Processor b/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.Processor new file mode 100644 index 0000000..727b42b --- /dev/null +++ b/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.Processor @@ -0,0 +1,2 @@ +com.geedgenetworks.core.processor.aggregate.AggregateProcessorImpl +com.geedgenetworks.core.processor.projection.ProjectionProcessorImpl \ No newline at end of file diff --git a/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.aggregate.AggregateProcessor b/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.aggregate.AggregateProcessor deleted file mode 100644 index 426a1a9..0000000 --- a/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.aggregate.AggregateProcessor +++ /dev/null @@ -1 +0,0 @@ -com.geedgenetworks.core.processor.aggregate.AggregateProcessorImpl \ No newline at end of file diff --git a/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.projection.ProjectionProcessor b/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.projection.ProjectionProcessor deleted file mode 100644 index ede2c8c..0000000 --- a/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.projection.ProjectionProcessor +++ /dev/null @@ -1 +0,0 @@ -com.geedgenetworks.core.processor.projection.ProjectionProcessorImpl \ No newline at end of file -- cgit v1.2.3 From 9dba0761c940b390581ab606d0859ccf3669e9ca Mon Sep 17 00:00:00 2001 From: wangkuan Date: Fri, 2 Aug 2024 14:57:15 +0800 Subject: [improve][core]修改初始化聚合函数逻辑,使函数只构造一次,优化性能 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../aggregate/AggregateProcessorFunction.java | 23 +++++++++++----------- .../core/processor/projection/UdfEntity.java | 2 ++ 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorFunction.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorFunction.java index b535faf..c07374e 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorFunction.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorFunction.java @@ -37,19 +37,11 @@ public class AggregateProcessorFunction implements org.apache.flink.api.common.f public AggregateProcessorFunction(AggregateConfig aggregateConfig, ExecutionConfig config) { udfClassNameLists = JSON.parseObject(config.getGlobalJobParameters().toMap().get(Constants.SYSPROP_UDF_PLUGIN_CONFIG), List.class); udfContexts = aggregateConfig.getFunctions(); - groupByFields = aggregateConfig.getGroup_by_fields(); - } - - @Override - public Accumulator createAccumulator() { - - functions = Lists.newLinkedList(); if (udfContexts == null || udfContexts.isEmpty()) { throw new RuntimeException(); } - Map map = new HashMap<>(); - Accumulator accumulator = new Accumulator(); - accumulator.setMetricsFields(map); + groupByFields = aggregateConfig.getGroup_by_fields(); + functions = Lists.newLinkedList(); Map udfClassReflect = getClassReflect(udfClassNameLists); try { for (UDFContext udfContext : udfContexts) { @@ -59,7 +51,6 @@ public class AggregateProcessorFunction implements org.apache.flink.api.common.f if (udfClassReflect.containsKey(udfContext.getFunction())) { Class cls = Class.forName(udfClassReflect.get(udfContext.getFunction())); AggregateFunction aggregateFunction = (AggregateFunction) cls.getConstructor().newInstance(); - aggregateFunction.open(udfContext, accumulator); // 函数如果包含filter,对表达式进行编译 if (udfContext.getFilter() != null) { AviatorEvaluatorInstance instance = AviatorEvaluator.getInstance(); @@ -72,6 +63,7 @@ public class AggregateProcessorFunction implements org.apache.flink.api.common.f udfEntity.setFilterExpression(filterExpression); udfEntity.setName(udfContext.getFunction()); udfEntity.setClassName(udfClassReflect.get(udfContext.getFunction())); + udfEntity.setUdfContext(udfContext); functions.add(udfEntity); } else { throw new GrootStreamRuntimeException(CommonErrorCode.UNSUPPORTED_OPERATION, @@ -83,7 +75,16 @@ public class AggregateProcessorFunction implements org.apache.flink.api.common.f throw new GrootStreamRuntimeException(CommonErrorCode.UNSUPPORTED_OPERATION, "Initialization UDAF failed!", e); } + } + @Override + public Accumulator createAccumulator() { + Map map = new HashMap<>(); + Accumulator accumulator = new Accumulator(); + accumulator.setMetricsFields(map); + for (UdfEntity udfEntity : functions) { + udfEntity.getAggregateFunction().open(udfEntity.getUdfContext(), accumulator); + } return accumulator; } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/UdfEntity.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/UdfEntity.java index c36a785..34267a6 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/UdfEntity.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/UdfEntity.java @@ -3,6 +3,7 @@ package com.geedgenetworks.core.processor.projection; import com.geedgenetworks.common.udf.AggregateFunction; import com.geedgenetworks.common.udf.ScalarFunction; +import com.geedgenetworks.common.udf.UDFContext; import com.googlecode.aviator.Expression; import lombok.Data; @@ -15,4 +16,5 @@ public class UdfEntity implements Serializable { private Expression filterExpression; private String name; private String className; + private UDFContext udfContext; } -- cgit v1.2.3 From d4bdba7dd0f86bce7abe77c85bc5880dbd324f78 Mon Sep 17 00:00:00 2001 From: doufenghu Date: Fri, 2 Aug 2024 15:19:49 +0800 Subject: [tests][e2e-clickhouse] Replace the dependency library clickhouse.jdbc.version with version 0.6.3. --- groot-tests/test-e2e-clickhouse/pom.xml | 28 +++++++++++++++++++++- .../test/e2e/clickhouse/ClickHouseIT.java | 3 +-- 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/groot-tests/test-e2e-clickhouse/pom.xml b/groot-tests/test-e2e-clickhouse/pom.xml index c576100..aef4470 100644 --- a/groot-tests/test-e2e-clickhouse/pom.xml +++ b/groot-tests/test-e2e-clickhouse/pom.xml @@ -16,7 +16,9 @@ 11 11 UTF-8 - 0.3.2-patch9 + 0.6.3 + 4.0.3 + 5.2.1 @@ -36,6 +38,21 @@ test + + com.zaxxer + HikariCP + ${hikaricp.version} + test + + + + org.lz4 + lz4-java + 1.8.0 + test + + + com.clickhouse clickhouse-jdbc @@ -43,6 +60,14 @@ test + + + org.apache.httpcomponents.client5 + httpclient5 + ${apache-httpclient.version} + test + + com.geedgenetworks connector-clickhouse @@ -50,6 +75,7 @@ test + org.xerial.snappy snappy-java diff --git a/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java b/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java index f752d57..6cc01b4 100644 --- a/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java +++ b/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java @@ -51,8 +51,7 @@ public class ClickHouseIT extends TestSuiteBase implements TestResource { private static final Config CONFIG = getInitClickhouseConfig(); private ClickHouseContainer clickHouseContainer; private Connection connection; - private static final ListTEST_DATASET = - generateTestDataSet(); + private static final ListTEST_DATASET = generateTestDataSet(); private static final String[] default_columns = new String[] { "id", "c_map", -- cgit v1.2.3 From e3e5ecd85a76318fd91e29782dc345cb6c9471cf Mon Sep 17 00:00:00 2001 From: doufenghu Date: Sun, 11 Aug 2024 23:31:30 +0800 Subject: [tests][e2e-clickhouse] Support the ingestion of common data types by a Flink job --- .../bootstrap/utils/ConfigShadeTest.java | 1 + .../inline_all_data_type_to_clickhouse.yaml | 78 +++++++ .../test/e2e/clickhouse/ClickHouseIT.java | 227 ++++++++++----------- .../test/resources/clickhouse_data_type_sink.yaml | 78 +++++++ .../src/test/resources/init/clickhouse_init.conf | 143 ------------- .../test/resources/init/clickhouse_test_sql.conf | 98 +++++++++ .../src/test/resources/init/init-clickhouse.sql | 4 + .../src/test/resources/init/users.xml | 29 +++ plugin-mapping.properties | 2 +- pom.xml | 2 +- 10 files changed, 400 insertions(+), 262 deletions(-) create mode 100644 groot-examples/end-to-end-example/src/main/resources/examples/inline_all_data_type_to_clickhouse.yaml create mode 100644 groot-tests/test-e2e-clickhouse/src/test/resources/clickhouse_data_type_sink.yaml delete mode 100644 groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_init.conf create mode 100644 groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_test_sql.conf create mode 100644 groot-tests/test-e2e-clickhouse/src/test/resources/init/init-clickhouse.sql create mode 100644 groot-tests/test-e2e-clickhouse/src/test/resources/init/users.xml diff --git a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java index ccdd224..336842f 100644 --- a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java +++ b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java @@ -67,5 +67,6 @@ public class ConfigShadeTest { Assertions.assertEquals(decryptPassword, PASSWORD); System.out.println( ConfigShadeUtils.encryptOption("aes", "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"admin\" password=\"galaxy2019\";")); System.out.println( ConfigShadeUtils.decryptOption("aes", "454f65ea6eef1256e3067104f82730e737b68959560966b811e7ff364116b03124917eb2b0f3596f14733aa29ebad9352644ce1a5c85991c6f01ba8a5e8f177a7ff0b2d3889a424249967b3870b50993d9644f239f0de82cdb13bdb502959e16afadffa49ef1e1d2b9c9b5113e619817")); + System.out.println( ConfigShadeUtils.encryptOption("aes", "testuser")); } } diff --git a/groot-examples/end-to-end-example/src/main/resources/examples/inline_all_data_type_to_clickhouse.yaml b/groot-examples/end-to-end-example/src/main/resources/examples/inline_all_data_type_to_clickhouse.yaml new file mode 100644 index 0000000..e449a08 --- /dev/null +++ b/groot-examples/end-to-end-example/src/main/resources/examples/inline_all_data_type_to_clickhouse.yaml @@ -0,0 +1,78 @@ +sources: + inline_source: + type: inline + schema: + fields: + - name: id + type: bigint + - name: c_array_string + type: array + - name: c_array_short + type: array + - name: c_array_int + type: array + - name: c_array_long + type: array + - name: c_array_float + type: array + - name: c_array_double + type: array + - name: c_string + type: string + - name: c_int8 + type: int + - name: c_int16 + type: int + - name: c_int32 + type: int + - name: c_int64 + type: int + - name: c_float32 + type: float + - name: c_float64 + type: double + - name: c_decimal + type: double + - name: c_date + type: string + - name: c_datetime + type: string + - name: c_nullable + type: int + - name: c_lowcardinality + type: string + properties: + # + # [string] Event Data, it will be parsed to Map by the specified format. + # + data: '[{"id":0,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":0,"c_lowcardinality":"string"},{"id":1,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":1,"c_lowcardinality":"string"},{"id":2,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":2,"c_lowcardinality":"string"},{"id":3,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":3,"c_lowcardinality":"string"},{"id":4,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":4,"c_lowcardinality":"string"},{"id":5,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":5,"c_lowcardinality":"string"},{"id":6,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":6,"c_lowcardinality":"string"},{"id":7,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":7,"c_lowcardinality":"string"},{"id":8,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":8,"c_lowcardinality":"string"},{"id":9,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":9,"c_lowcardinality":"string"},{"id":10,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":10,"c_lowcardinality":"string"},{"id":11,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":11,"c_lowcardinality":"string"},{"id":12,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":12,"c_lowcardinality":"string"},{"id":13,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":13,"c_lowcardinality":"string"},{"id":14,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":14,"c_lowcardinality":"string"},{"id":15,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":15,"c_lowcardinality":"string"},{"id":16,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":16,"c_lowcardinality":"string"},{"id":17,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":17,"c_lowcardinality":"string"},{"id":18,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":18,"c_lowcardinality":"string"},{"id":19,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":19,"c_lowcardinality":"string"},{"id":20,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":20,"c_lowcardinality":"string"},{"id":21,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":21,"c_lowcardinality":"string"},{"id":22,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":22,"c_lowcardinality":"string"},{"id":23,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":23,"c_lowcardinality":"string"},{"id":24,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":24,"c_lowcardinality":"string"},{"id":25,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":25,"c_lowcardinality":"string"},{"id":26,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":26,"c_lowcardinality":"string"},{"id":27,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":27,"c_lowcardinality":"string"},{"id":28,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":28,"c_lowcardinality":"string"},{"id":29,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":29,"c_lowcardinality":"string"},{"id":30,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":30,"c_lowcardinality":"string"},{"id":31,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":31,"c_lowcardinality":"string"},{"id":32,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":32,"c_lowcardinality":"string"},{"id":33,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":33,"c_lowcardinality":"string"},{"id":34,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":34,"c_lowcardinality":"string"},{"id":35,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":35,"c_lowcardinality":"string"},{"id":36,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":36,"c_lowcardinality":"string"},{"id":37,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":37,"c_lowcardinality":"string"},{"id":38,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":38,"c_lowcardinality":"string"},{"id":39,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":39,"c_lowcardinality":"string"},{"id":40,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":40,"c_lowcardinality":"string"},{"id":41,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":41,"c_lowcardinality":"string"},{"id":42,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":42,"c_lowcardinality":"string"},{"id":43,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":43,"c_lowcardinality":"string"},{"id":44,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":44,"c_lowcardinality":"string"},{"id":45,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":45,"c_lowcardinality":"string"},{"id":46,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":46,"c_lowcardinality":"string"},{"id":47,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":47,"c_lowcardinality":"string"},{"id":48,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":48,"c_lowcardinality":"string"},{"id":49,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":49,"c_lowcardinality":"string"},{"id":50,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":50,"c_lowcardinality":"string"},{"id":51,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":51,"c_lowcardinality":"string"},{"id":52,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":52,"c_lowcardinality":"string"},{"id":53,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":53,"c_lowcardinality":"string"},{"id":54,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":54,"c_lowcardinality":"string"},{"id":55,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":55,"c_lowcardinality":"string"},{"id":56,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":56,"c_lowcardinality":"string"},{"id":57,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":57,"c_lowcardinality":"string"},{"id":58,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":58,"c_lowcardinality":"string"},{"id":59,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":59,"c_lowcardinality":"string"},{"id":60,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":60,"c_lowcardinality":"string"},{"id":61,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":61,"c_lowcardinality":"string"},{"id":62,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":62,"c_lowcardinality":"string"},{"id":63,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":63,"c_lowcardinality":"string"},{"id":64,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":64,"c_lowcardinality":"string"},{"id":65,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":65,"c_lowcardinality":"string"},{"id":66,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":66,"c_lowcardinality":"string"},{"id":67,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":67,"c_lowcardinality":"string"},{"id":68,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":68,"c_lowcardinality":"string"},{"id":69,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":69,"c_lowcardinality":"string"},{"id":70,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":70,"c_lowcardinality":"string"},{"id":71,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":71,"c_lowcardinality":"string"},{"id":72,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":72,"c_lowcardinality":"string"},{"id":73,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":73,"c_lowcardinality":"string"},{"id":74,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":74,"c_lowcardinality":"string"},{"id":75,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":75,"c_lowcardinality":"string"},{"id":76,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":76,"c_lowcardinality":"string"},{"id":77,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":77,"c_lowcardinality":"string"},{"id":78,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":78,"c_lowcardinality":"string"},{"id":79,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":79,"c_lowcardinality":"string"},{"id":80,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":80,"c_lowcardinality":"string"},{"id":81,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":81,"c_lowcardinality":"string"},{"id":82,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":82,"c_lowcardinality":"string"},{"id":83,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":83,"c_lowcardinality":"string"},{"id":84,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":84,"c_lowcardinality":"string"},{"id":85,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":85,"c_lowcardinality":"string"},{"id":86,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":86,"c_lowcardinality":"string"},{"id":87,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":87,"c_lowcardinality":"string"},{"id":88,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":88,"c_lowcardinality":"string"},{"id":89,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":89,"c_lowcardinality":"string"},{"id":90,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":90,"c_lowcardinality":"string"},{"id":91,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":91,"c_lowcardinality":"string"},{"id":92,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":92,"c_lowcardinality":"string"},{"id":93,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":93,"c_lowcardinality":"string"},{"id":94,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":94,"c_lowcardinality":"string"},{"id":95,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":95,"c_lowcardinality":"string"},{"id":96,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":96,"c_lowcardinality":"string"},{"id":97,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":97,"c_lowcardinality":"string"},{"id":98,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":98,"c_lowcardinality":"string"},{"id":99,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":99,"c_lowcardinality":"string"}]' + format: json + interval.per.row: 1s + repeat.count: 10 + json.ignore.parse.errors: false + + +sinks: + clickhouse_sink: + type: clickhouse + properties: + host: 192.168.44.12:9001 + table: default.sink_table + batch.size: 10 + batch.byte.size: 200MB + batch.interval: 1s + connection.user: e54c9568586180eede1506eecf3574e9 + connection.password: 86cf0e2ffba3f541a6c6761313e5cc7e + +application: # [object] Define job configuration + env: + name: example-inline-to-clickhouse + parallelism: 1 + shade.identifier: aes + pipeline: + object-reuse: true + topology: + - name: inline_source + downstream: [ clickhouse_sink ] + - name: clickhouse_sink + downstream: [] \ No newline at end of file diff --git a/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java b/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java index 6cc01b4..8eab377 100644 --- a/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java +++ b/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java @@ -1,11 +1,13 @@ package com.geedgenetworks.test.e2e.clickhouse; +import com.alibaba.fastjson2.JSON; import com.geedgenetworks.test.common.TestResource; import com.geedgenetworks.test.common.TestSuiteBase; import com.geedgenetworks.test.common.container.ContainerUtil; import com.geedgenetworks.test.common.container.TestContainer; import com.geedgenetworks.test.common.container.TestContainerId; import com.geedgenetworks.test.common.junit.DisabledOnContainer; +import com.google.common.collect.Maps; import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; import lombok.extern.slf4j.Slf4j; @@ -15,10 +17,12 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; import org.testcontainers.containers.ClickHouseContainer; +import org.testcontainers.containers.Container; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.lifecycle.Startables; import org.testcontainers.shaded.org.apache.commons.io.IOUtils; import org.testcontainers.utility.DockerLoggerFactory; +import org.testcontainers.utility.MountableFile; import java.io.File; import java.io.IOException; @@ -26,14 +30,14 @@ import java.io.InputStream; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; import java.sql.*; -import java.sql.Date; -import java.time.LocalDate; -import java.time.LocalDateTime; import java.util.*; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.awaitility.Awaitility.await; + @Slf4j @DisabledOnContainer( value = {TestContainerId.FLINK_1_17}, @@ -41,7 +45,7 @@ import java.util.stream.Stream; public class ClickHouseIT extends TestSuiteBase implements TestResource { private static final String CLICKHOUSE_DOCKER_IMAGE = "clickhouse/clickhouse-server:23.3.19.32"; private static final String DRIVER_CLASS = "com.clickhouse.jdbc.ClickHouseDriver"; - private static final String INIT_CLICKHOUSE_PATH = "/init/clickhouse_init.conf"; + private static final String INIT_CLICKHOUSE_PATH = "/init/clickhouse_test_sql.conf"; private static final String DATABASE = "default"; private static final String SOURCE_TABLE = "source_table"; private static final String SINK_TABLE = "sink_table"; @@ -51,39 +55,27 @@ public class ClickHouseIT extends TestSuiteBase implements TestResource { private static final Config CONFIG = getInitClickhouseConfig(); private ClickHouseContainer clickHouseContainer; private Connection connection; - private static final ListTEST_DATASET = generateTestDataSet(); private static final String[] default_columns = new String[] { - "id", - "c_map", - "c_array_string", - "c_array_short", - "c_array_int", - "c_array_long", - "c_array_float", - "c_array_double", - "c_string", - "c_boolean", - "c_int8", - "c_int16", - "c_int32", - "c_int64", - "c_float32", - "c_float64", - "c_decimal", - "c_date", - "c_datetime", - "c_nullable", - "c_lowcardinality", - "c_nested.int", - "c_nested.double", - "c_nested.string", - "c_int128", - "c_uint128", - "c_int256", - "c_uint256", - "c_point", - "c_ring" + "id", + "c_array_string", + "c_array_short", + "c_array_int", + "c_array_long", + "c_array_float", + "c_array_double", + "c_string", + "c_int8", + "c_int16", + "c_int32", + "c_int64", + "c_float32", + "c_float64", + "c_decimal", + "c_nullable", + "c_lowcardinality" }; + private static final List>TEST_DATASET = generateTestDataSet(); + @BeforeAll @Override @@ -92,10 +84,17 @@ public class ClickHouseIT extends TestSuiteBase implements TestResource { new ClickHouseContainer(CLICKHOUSE_DOCKER_IMAGE) .withNetwork(NETWORK) .withNetworkAliases(HOST) + .withCopyFileToContainer(MountableFile.forClasspathResource("init/users.xml"), "/etc/clickhouse-server/users.xml") + .withCopyFileToContainer(MountableFile.forClasspathResource("init/init-clickhouse.sql"), "/docker-entrypoint-initdb.d/init-clickhouse.sql") .withLogConsumer( new Slf4jLogConsumer( DockerLoggerFactory.getLogger(CLICKHOUSE_DOCKER_IMAGE))); + Startables.deepStart(Stream.of(this.clickHouseContainer)).join(); + System.out.println("Clickhouse JDBC URL: " + this.clickHouseContainer.getJdbcUrl()); + System.out.println("Clickhouse username: " + this.clickHouseContainer.getUsername()); + System.out.println("Clickhouse password: " + this.clickHouseContainer.getPassword()); + log.info("Clickhouse container started"); Awaitility.given() .ignoreExceptions() @@ -104,20 +103,51 @@ public class ClickHouseIT extends TestSuiteBase implements TestResource { .untilAsserted(this::initConnection); this.initializeClickhouseTable(); this.batchInsertData(); + log.info(JSON.toJSONString(TEST_DATASET)); } + private void initConnection() + throws SQLException, ClassNotFoundException, InstantiationException, + IllegalAccessException { + final Properties info = new Properties(); + info.put("user", this.clickHouseContainer.getUsername()); + info.put("password", this.clickHouseContainer.getPassword()); + this.connection = + ((Driver) Class.forName(DRIVER_CLASS).newInstance()) + .connect(this.clickHouseContainer.getJdbcUrl(), info); + + } @TestTemplate - public void testClickhouse(TestContainer container) throws Exception { + public void testClickHouse(TestContainer container) throws Exception { assertHasData(SOURCE_TABLE); + clearTable(SOURCE_TABLE); + } - //assertHasData(SINK_TABLE); - //compareResult(); - //clearSinkTable(); + @TestTemplate + public void testClickHouseDataTypeSinkTable(TestContainer container) throws Exception { + CompletableFuture.supplyAsync( + () -> { + try { + Container.ExecResult execResult = container.executeJob("/clickhouse_data_type_sink.yaml"); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + return execResult; + } catch (Exception e) { + log.error("Commit task exception:" + e.getMessage()); + throw new RuntimeException(e); + } + }); + await().atMost(300000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + assertHasData(SINK_TABLE); + clearTable(SINK_TABLE); + }); } + private void assertHasData(String table) { try (Statement statement = connection.createStatement()) { String sql = String.format("select * from %s.%s limit 1", DATABASE, table); @@ -128,9 +158,9 @@ public class ClickHouseIT extends TestSuiteBase implements TestResource { } } - private void clearSinkTable() { + private void clearTable(String table) { try (Statement statement = connection.createStatement()) { - statement.execute(String.format("truncate table %s.%s", DATABASE, SINK_TABLE)); + statement.execute(String.format("truncate table %s.%s", DATABASE, table)); } catch (SQLException e) { throw new RuntimeException("Test clickhouse server image error", e); } @@ -187,38 +217,24 @@ public class ClickHouseIT extends TestSuiteBase implements TestResource { try { this.connection.setAutoCommit(true); preparedStatement = this.connection.prepareStatement(sql); - for (Object[] row : TEST_DATASET) { - preparedStatement.setLong(1, (Long) row[0]); - preparedStatement.setObject(2, row[1]); - preparedStatement.setArray(3, toSqlArray(row[2])); - preparedStatement.setArray(4, toSqlArray(row[3])); - preparedStatement.setArray(5, toSqlArray(row[4])); - preparedStatement.setArray(6, toSqlArray(row[5])); - preparedStatement.setArray(7, toSqlArray(row[6])); - preparedStatement.setArray(8, toSqlArray(row[7])); - preparedStatement.setString(9, (String) row[8]); - preparedStatement.setBoolean(10, (Boolean) row[9]); - preparedStatement.setByte(11, (Byte) row[10]); - preparedStatement.setShort(12, (Short) row[11]); - preparedStatement.setInt(13, (Integer) row[12]); - preparedStatement.setLong(14, (Long) row[13]); - preparedStatement.setFloat(15, (Float) row[14]); - preparedStatement.setDouble(16, (Double) row[15]); - preparedStatement.setBigDecimal(17, (BigDecimal) row[16]); - preparedStatement.setDate(18, Date.valueOf((LocalDate) row[17])); - preparedStatement.setTimestamp( - 19, Timestamp.valueOf((LocalDateTime) row[18])); - preparedStatement.setInt(20, (Integer) row[19]); - preparedStatement.setString(21, (String) row[20]); - preparedStatement.setArray(22, toSqlArray(row[21])); - preparedStatement.setArray(23, toSqlArray(row[22])); - preparedStatement.setArray(24, toSqlArray(row[23])); - preparedStatement.setObject(25, row[24]); - preparedStatement.setObject(26, row[25]); - preparedStatement.setObject(27, row[26]); - preparedStatement.setObject(28, row[27]); - preparedStatement.setObject(29, row[28]); - preparedStatement.setObject(30, row[29]); + for (Map row : TEST_DATASET) { + preparedStatement.setLong(1, (Long) row.get(default_columns[0])); + preparedStatement.setArray(2, toSqlArray(row.get(default_columns[1]))); + preparedStatement.setArray(3, toSqlArray(row.get(default_columns[2]))); + preparedStatement.setArray(4, toSqlArray(row.get(default_columns[3]))); + preparedStatement.setArray(5, toSqlArray(row.get(default_columns[4]))); + preparedStatement.setArray(6, toSqlArray(row.get(default_columns[5]))); + preparedStatement.setArray(7, toSqlArray(row.get(default_columns[6]))); + preparedStatement.setString(8, (String) row.get(default_columns[7])); + preparedStatement.setByte(9, (Byte) row.get(default_columns[8])); + preparedStatement.setShort(10, (Short) row.get(default_columns[9])); + preparedStatement.setInt(11, (Integer) row.get(default_columns[10])); + preparedStatement.setLong(12, (Long) row.get(default_columns[11])); + preparedStatement.setFloat(13, (Float) row.get(default_columns[12])); + preparedStatement.setDouble(14, (Double) row.get(default_columns[13])); + preparedStatement.setBigDecimal(15, (BigDecimal) row.get(default_columns[14])); + preparedStatement.setInt(16, (Integer) row.get(default_columns[15])); + preparedStatement.setString(17, (String) row.get(default_columns[16])); preparedStatement.addBatch(); } @@ -277,56 +293,33 @@ public class ClickHouseIT extends TestSuiteBase implements TestResource { return connection.createArrayOf(sqlType, elements); } - private static List generateTestDataSet() { - List rows = new ArrayList<>(); + private static List> generateTestDataSet() { + List> rows = new ArrayList<>(); for (int i = 0; i < 100; ++i) { - Object[] row = new Object[] { - (long) i, - Collections.singletonMap("key", Integer.parseInt("1")), - new String[] {"string"}, - new Short[] {Short.parseShort("1")}, - new Integer[] {Integer.parseInt("1")}, - new Long[] {Long.parseLong("1")}, - new Float[] {Float.parseFloat("1.1")}, - new Double[] {Double.parseDouble("1.1")}, - "string", - Boolean.FALSE, - Byte.parseByte("1"), - Short.parseShort("1"), - Integer.parseInt("1"), - Long.parseLong("1"), - Float.parseFloat("1.1"), - Double.parseDouble("1.1"), - BigDecimal.valueOf(11L, 1), - LocalDate.now(), - LocalDateTime.now(), - i, - "string", - new Integer[] {Integer.parseInt("1")}, - new Double[] {Double.parseDouble("1.1")}, - new String[] {"1"}, - "170141183460469231731687303715884105727", - "340282366920938463463374607431768211455", - "57896044618658097711785492504343953926634992332820282019728792003956564819967", - "115792089237316195423570985008687907853269984665640564039457584007913129639935", - new double[] {1, 2}, - new double[][] {{2, 3}, {4, 5}} - }; + Map row = Maps.newLinkedHashMap(); + row.put(default_columns[0], (long) i); + row.put(default_columns[1], new String[] {"string"}); + row.put(default_columns[2], new Short[] {Short.parseShort("1")}); + row.put(default_columns[3], new Integer[] {Integer.parseInt("1")}); + row.put(default_columns[4], new Long[] {Long.parseLong("1")}); + row.put(default_columns[5], new Float[] {Float.parseFloat("1.1")}); + row.put(default_columns[6], new Double[] {Double.parseDouble("1.1")}); + row.put(default_columns[7], "string"); + row.put(default_columns[8], Byte.parseByte("1")); + row.put(default_columns[9], Short.parseShort("1")); + row.put(default_columns[10], Integer.parseInt("1")); + row.put(default_columns[11], Long.parseLong("1")); + row.put(default_columns[12], Float.parseFloat("1.1")); + row.put(default_columns[13], Double.parseDouble("1.1")); + row.put(default_columns[14], BigDecimal.valueOf(11L, 1)); + row.put(default_columns[15], i); + row.put(default_columns[16], "string"); rows.add(row); } return rows; } - private void initConnection() - throws SQLException, ClassNotFoundException, InstantiationException, - IllegalAccessException { - final Properties info = new Properties(); - info.put("user", this.clickHouseContainer.getUsername()); - info.put("password", this.clickHouseContainer.getPassword()); - this.connection = - ((Driver) Class.forName(DRIVER_CLASS).newInstance()) - .connect(this.clickHouseContainer.getJdbcUrl(), info); - } + private void initializeClickhouseTable() { try { diff --git a/groot-tests/test-e2e-clickhouse/src/test/resources/clickhouse_data_type_sink.yaml b/groot-tests/test-e2e-clickhouse/src/test/resources/clickhouse_data_type_sink.yaml new file mode 100644 index 0000000..eb020c1 --- /dev/null +++ b/groot-tests/test-e2e-clickhouse/src/test/resources/clickhouse_data_type_sink.yaml @@ -0,0 +1,78 @@ +sources: + inline_source: + type: inline + schema: + fields: + - name: id + type: bigint + - name: c_array_string + type: array + - name: c_array_short + type: array + - name: c_array_int + type: array + - name: c_array_long + type: array + - name: c_array_float + type: array + - name: c_array_double + type: array + - name: c_string + type: string + - name: c_int8 + type: int + - name: c_int16 + type: int + - name: c_int32 + type: int + - name: c_int64 + type: int + - name: c_float32 + type: float + - name: c_float64 + type: double + - name: c_decimal + type: double + - name: c_date + type: string + - name: c_datetime + type: string + - name: c_nullable + type: int + - name: c_lowcardinality + type: string + properties: + # + # [string] Event Data, it will be parsed to Map by the specified format. + # + data: '[{"id":0,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":0,"c_lowcardinality":"string"},{"id":1,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":1,"c_lowcardinality":"string"},{"id":2,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":2,"c_lowcardinality":"string"},{"id":3,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":3,"c_lowcardinality":"string"},{"id":4,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":4,"c_lowcardinality":"string"},{"id":5,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":5,"c_lowcardinality":"string"},{"id":6,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":6,"c_lowcardinality":"string"},{"id":7,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":7,"c_lowcardinality":"string"},{"id":8,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":8,"c_lowcardinality":"string"},{"id":9,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":9,"c_lowcardinality":"string"},{"id":10,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":10,"c_lowcardinality":"string"},{"id":11,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":11,"c_lowcardinality":"string"},{"id":12,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":12,"c_lowcardinality":"string"},{"id":13,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":13,"c_lowcardinality":"string"},{"id":14,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":14,"c_lowcardinality":"string"},{"id":15,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":15,"c_lowcardinality":"string"},{"id":16,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":16,"c_lowcardinality":"string"},{"id":17,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":17,"c_lowcardinality":"string"},{"id":18,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":18,"c_lowcardinality":"string"},{"id":19,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":19,"c_lowcardinality":"string"},{"id":20,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":20,"c_lowcardinality":"string"},{"id":21,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":21,"c_lowcardinality":"string"},{"id":22,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":22,"c_lowcardinality":"string"},{"id":23,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":23,"c_lowcardinality":"string"},{"id":24,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":24,"c_lowcardinality":"string"},{"id":25,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":25,"c_lowcardinality":"string"},{"id":26,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":26,"c_lowcardinality":"string"},{"id":27,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":27,"c_lowcardinality":"string"},{"id":28,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":28,"c_lowcardinality":"string"},{"id":29,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":29,"c_lowcardinality":"string"},{"id":30,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":30,"c_lowcardinality":"string"},{"id":31,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":31,"c_lowcardinality":"string"},{"id":32,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":32,"c_lowcardinality":"string"},{"id":33,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":33,"c_lowcardinality":"string"},{"id":34,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":34,"c_lowcardinality":"string"},{"id":35,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":35,"c_lowcardinality":"string"},{"id":36,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":36,"c_lowcardinality":"string"},{"id":37,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":37,"c_lowcardinality":"string"},{"id":38,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":38,"c_lowcardinality":"string"},{"id":39,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":39,"c_lowcardinality":"string"},{"id":40,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":40,"c_lowcardinality":"string"},{"id":41,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":41,"c_lowcardinality":"string"},{"id":42,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":42,"c_lowcardinality":"string"},{"id":43,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":43,"c_lowcardinality":"string"},{"id":44,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":44,"c_lowcardinality":"string"},{"id":45,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":45,"c_lowcardinality":"string"},{"id":46,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":46,"c_lowcardinality":"string"},{"id":47,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":47,"c_lowcardinality":"string"},{"id":48,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":48,"c_lowcardinality":"string"},{"id":49,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":49,"c_lowcardinality":"string"},{"id":50,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":50,"c_lowcardinality":"string"},{"id":51,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":51,"c_lowcardinality":"string"},{"id":52,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":52,"c_lowcardinality":"string"},{"id":53,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":53,"c_lowcardinality":"string"},{"id":54,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":54,"c_lowcardinality":"string"},{"id":55,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":55,"c_lowcardinality":"string"},{"id":56,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":56,"c_lowcardinality":"string"},{"id":57,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":57,"c_lowcardinality":"string"},{"id":58,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":58,"c_lowcardinality":"string"},{"id":59,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":59,"c_lowcardinality":"string"},{"id":60,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":60,"c_lowcardinality":"string"},{"id":61,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":61,"c_lowcardinality":"string"},{"id":62,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":62,"c_lowcardinality":"string"},{"id":63,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":63,"c_lowcardinality":"string"},{"id":64,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":64,"c_lowcardinality":"string"},{"id":65,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":65,"c_lowcardinality":"string"},{"id":66,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":66,"c_lowcardinality":"string"},{"id":67,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":67,"c_lowcardinality":"string"},{"id":68,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":68,"c_lowcardinality":"string"},{"id":69,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":69,"c_lowcardinality":"string"},{"id":70,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":70,"c_lowcardinality":"string"},{"id":71,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":71,"c_lowcardinality":"string"},{"id":72,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":72,"c_lowcardinality":"string"},{"id":73,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":73,"c_lowcardinality":"string"},{"id":74,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":74,"c_lowcardinality":"string"},{"id":75,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":75,"c_lowcardinality":"string"},{"id":76,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":76,"c_lowcardinality":"string"},{"id":77,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":77,"c_lowcardinality":"string"},{"id":78,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":78,"c_lowcardinality":"string"},{"id":79,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":79,"c_lowcardinality":"string"},{"id":80,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":80,"c_lowcardinality":"string"},{"id":81,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":81,"c_lowcardinality":"string"},{"id":82,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":82,"c_lowcardinality":"string"},{"id":83,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":83,"c_lowcardinality":"string"},{"id":84,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":84,"c_lowcardinality":"string"},{"id":85,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":85,"c_lowcardinality":"string"},{"id":86,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":86,"c_lowcardinality":"string"},{"id":87,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":87,"c_lowcardinality":"string"},{"id":88,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":88,"c_lowcardinality":"string"},{"id":89,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":89,"c_lowcardinality":"string"},{"id":90,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":90,"c_lowcardinality":"string"},{"id":91,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":91,"c_lowcardinality":"string"},{"id":92,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":92,"c_lowcardinality":"string"},{"id":93,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":93,"c_lowcardinality":"string"},{"id":94,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":94,"c_lowcardinality":"string"},{"id":95,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":95,"c_lowcardinality":"string"},{"id":96,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":96,"c_lowcardinality":"string"},{"id":97,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":97,"c_lowcardinality":"string"},{"id":98,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":98,"c_lowcardinality":"string"},{"id":99,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":99,"c_lowcardinality":"string"}]' + format: json + interval.per.row: 1s + repeat.count: 100 + json.ignore.parse.errors: false + + +sinks: + clickhouse_sink: + type: clickhouse + properties: + host: clickhouse:9000 + table: default.sink_table + batch.size: 10 + batch.byte.size: 200MB + batch.interval: 1s + connection.user: ee9b0016824d59c8c191aa9633e4b61e + connection.password: ee9b0016824d59c8c191aa9633e4b61e + +application: # [object] Define job configuration + env: + name: example-inline-to-clickhouse + parallelism: 1 + shade.identifier: aes + pipeline: + object-reuse: true + topology: + - name: inline_source + downstream: [ clickhouse_sink ] + - name: clickhouse_sink + downstream: [] \ No newline at end of file diff --git a/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_init.conf b/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_init.conf deleted file mode 100644 index 78f2daa..0000000 --- a/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_init.conf +++ /dev/null @@ -1,143 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -source_table = """ -set allow_experimental_geo_types = 1; -create table if not exists `default`.source_table( - `id` Int64, - `c_map` Map(String, Int32), - `c_array_string` Array(String), - `c_array_short` Array(Int16), - `c_array_int` Array(Int32), - `c_array_long` Array(Int64), - `c_array_float` Array(Float32), - `c_array_double` Array(Float64), - `c_string` String, - `c_boolean` Boolean, - `c_int8` Int8, - `c_int16` Int16, - `c_int32` Int32, - `c_int64` Int64, - `c_float32` Float32, - `c_float64` Float64, - `c_decimal` Decimal(9,4), - `c_date` Date, - `c_datetime` DateTime64, - `c_nullable` Nullable(Int32), - `c_lowcardinality` LowCardinality(String), - `c_nested` Nested - ( - `int` UInt32, - `double` Float64, - `string` String - ), - `c_int128` Int128, - `c_uint128` UInt128, - `c_int256` Int256, - `c_uint256` UInt256, - `c_point` Point, - `c_ring` Ring -)engine=Memory; -""" - -sink_table = """ -create table if not exists `default`.sink_table( - `id` Int64, - `c_map` Map(String, Int32), - `c_array_string` Array(String), - `c_array_short` Array(Int16), - `c_array_int` Array(Int32), - `c_array_long` Array(Int64), - `c_array_float` Array(Float32), - `c_array_double` Array(Float64), - `c_string` String, - `c_boolean` Boolean, - `c_int8` Int8, - `c_int16` Int16, - `c_int32` Int32, - `c_int64` Int64, - `c_float32` Float32, - `c_float64` Float64, - `c_decimal` Decimal(9,4), - `c_date` Date, - `c_datetime` DateTime64, - `c_nullable` Nullable(Int32), - `c_lowcardinality` LowCardinality(String), - `c_nested` Nested - ( - `int` UInt32, - `double` Float64, - `string` String - ), - `c_int128` Int128, - `c_uint128` UInt128, - `c_int256` Int256, - `c_uint256` UInt256, - `c_point` Point, - `c_ring` Ring -)engine=Memory; -""" - -insert_sql = """ -insert into `default`.source_table -( - `id`, - `c_map`, - `c_array_string`, - `c_array_short`, - `c_array_int`, - `c_array_long`, - `c_array_float`, - `c_array_double`, - `c_string`, - `c_boolean`, - `c_int8`, - `c_int16`, - `c_int32`, - `c_int64`, - `c_float32`, - `c_float64`, - `c_decimal`, - `c_date`, - `c_datetime`, - `c_nullable`, - `c_lowcardinality`, - `c_nested.int`, - `c_nested.double`, - `c_nested.string`, - `c_int128`, - `c_uint128`, - `c_int256`, - `c_uint256`, - `c_point`, - `c_ring` -) -values -(?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?) -""" - -compare_sql = """ -select - %s - from ( - select * from default.source_table -union all - select * from default.sink_table - ) -group by %s -having count(*) < 2 -""" \ No newline at end of file diff --git a/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_test_sql.conf b/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_test_sql.conf new file mode 100644 index 0000000..c24e632 --- /dev/null +++ b/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_test_sql.conf @@ -0,0 +1,98 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +source_table = """ +set allow_experimental_geo_types = 1; +create table if not exists `default`.source_table( + `id` Int64, + `c_array_string` Array(String), + `c_array_short` Array(Int16), + `c_array_int` Array(Int32), + `c_array_long` Array(Int64), + `c_array_float` Array(Float32), + `c_array_double` Array(Float64), + `c_string` String, + `c_int8` Int8, + `c_int16` Int16, + `c_int32` Int32, + `c_int64` Int64, + `c_float32` Float32, + `c_float64` Float64, + `c_decimal` Decimal(9,4), + `c_nullable` Nullable(Int32), + `c_lowcardinality` LowCardinality(String) +)engine=Memory; +""" + +sink_table = """ +create table if not exists `default`.sink_table( + `id` Int64, + `c_array_string` Array(String), + `c_array_short` Array(Int16), + `c_array_int` Array(Int32), + `c_array_long` Array(Int64), + `c_array_float` Array(Float32), + `c_array_double` Array(Float64), + `c_string` String, + `c_int8` Int8, + `c_int16` Int16, + `c_int32` Int32, + `c_int64` Int64, + `c_float32` Float32, + `c_float64` Float64, + `c_decimal` Decimal(9,4), + `c_nullable` Nullable(Int32), + `c_lowcardinality` LowCardinality(String) +)engine=Memory; +""" + +insert_sql = """ +insert into `default`.source_table +( + `id`, + `c_array_string`, + `c_array_short`, + `c_array_int`, + `c_array_long`, + `c_array_float`, + `c_array_double`, + `c_string`, + `c_int8`, + `c_int16`, + `c_int32`, + `c_int64`, + `c_float32`, + `c_float64`, + `c_decimal`, + `c_nullable`, + `c_lowcardinality` +) +values +(?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?) +""" + +compare_sql = """ +select + %s + from ( + select * from default.source_table +union all + select * from default.sink_table + ) +group by %s +having count(*) < 2 +""" \ No newline at end of file diff --git a/groot-tests/test-e2e-clickhouse/src/test/resources/init/init-clickhouse.sql b/groot-tests/test-e2e-clickhouse/src/test/resources/init/init-clickhouse.sql new file mode 100644 index 0000000..fd9daac --- /dev/null +++ b/groot-tests/test-e2e-clickhouse/src/test/resources/init/init-clickhouse.sql @@ -0,0 +1,4 @@ +show databases; +-- ALTER USER default IDENTIFIED WITH plaintext_password BY 'testuser'; +CREATE USER testuser IDENTIFIED WITH plaintext_password BY 'testuser'; +GRANT ALL ON *.* TO testuser; \ No newline at end of file diff --git a/groot-tests/test-e2e-clickhouse/src/test/resources/init/users.xml b/groot-tests/test-e2e-clickhouse/src/test/resources/init/users.xml new file mode 100644 index 0000000..86a590d --- /dev/null +++ b/groot-tests/test-e2e-clickhouse/src/test/resources/init/users.xml @@ -0,0 +1,29 @@ + + + + + default + 1 + 1 + 1 + 1 + + ::/0 + + + ALTER TABLE ON *.* + CREATE USER ON *.* + GRANT ON *.* + + + + + + + 10000000000 + 1 + random + 8 + + + diff --git a/plugin-mapping.properties b/plugin-mapping.properties index 4e2eb81..7cc144d 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -1,7 +1,7 @@ #Connectors grootstream.source.kafka = connector-kafka grootstream.sink.kafka = connector-kafka -grootstream.source.clickhouse= connector-clickhouse +grootstream.sink.clickhouse = connector-clickhouse grootstream.source.ipfix = connector-ipfix-collector grootstream.source.mock = connector-mock grootstream.source.file = connector-file \ No newline at end of file diff --git a/pom.xml b/pom.xml index 7c36d91..177aa11 100644 --- a/pom.xml +++ b/pom.xml @@ -36,7 +36,7 @@ 3.1.1 1.3.0 4.0.4 - 1.20.0 + 1.20.1 4.2.0 2.40.0 1.7.25 -- cgit v1.2.3 From e19538c5bee90c140a65c742738048d12dc1e0ad Mon Sep 17 00:00:00 2001 From: wangkuan Date: Tue, 13 Aug 2024 11:10:40 +0800 Subject: [feature][core][common][bootstrap]支持tableProcessor,增加UnRoll函数 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- config/udf.plugins | 3 +- .../execution/AbstractProcessorExecutor.java | 57 ++++++++- .../common/config/TableConfigOptions.java | 34 +++++ .../geedgenetworks/common/udf/TableFunction.java | 20 +++ groot-common/src/main/resources/udf.plugins | 3 +- .../com/geedgenetworks/core/pojo/TableConfig.java | 15 +++ .../core/processor/projection/UdfEntity.java | 2 + .../core/processor/table/TableProcessor.java | 7 ++ .../processor/table/TableProcessorFunction.java | 138 +++++++++++++++++++++ .../core/processor/table/TableProcessorImpl.java | 33 +++++ .../com/geedgenetworks/core/udf/udtf/UnRoll.java | 132 ++++++++++++++++++++ .../com.geedgenetworks.core.processor.Processor | 3 +- .../core/udf/test/table/UnRollFunctionTest.java | 132 ++++++++++++++++++++ 13 files changed, 572 insertions(+), 7 deletions(-) create mode 100644 groot-common/src/main/java/com/geedgenetworks/common/config/TableConfigOptions.java create mode 100644 groot-common/src/main/java/com/geedgenetworks/common/udf/TableFunction.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/pojo/TableConfig.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessor.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorFunction.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/UnRoll.java create mode 100644 groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnRollFunctionTest.java diff --git a/config/udf.plugins b/config/udf.plugins index 2978bbe..9eb32c4 100644 --- a/config/udf.plugins +++ b/config/udf.plugins @@ -21,4 +21,5 @@ com.geedgenetworks.core.udf.udaf.CollectSet com.geedgenetworks.core.udf.udaf.LongCount com.geedgenetworks.core.udf.udaf.Mean com.geedgenetworks.core.udf.udaf.LastValue -com.geedgenetworks.core.udf.udaf.FirstValue \ No newline at end of file +com.geedgenetworks.core.udf.udaf.FirstValue +com.geedgenetworks.core.udf.udtf.UnRoll \ No newline at end of file diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java index bd8b75c..66c0b0f 100644 --- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java +++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java @@ -2,15 +2,14 @@ package com.geedgenetworks.bootstrap.execution; import com.alibaba.fastjson.JSONObject; import com.geedgenetworks.bootstrap.exception.JobExecuteException; -import com.geedgenetworks.common.config.AggregateConfigOptions; -import com.geedgenetworks.common.config.CheckConfigUtil; -import com.geedgenetworks.common.config.CheckResult; -import com.geedgenetworks.common.config.ProjectionConfigOptions; +import com.geedgenetworks.common.config.*; import com.geedgenetworks.common.exception.CommonErrorCode; import com.geedgenetworks.common.exception.ConfigValidationException; import com.geedgenetworks.core.pojo.AggregateConfig; import com.geedgenetworks.core.pojo.ProcessorConfig; import com.geedgenetworks.core.pojo.ProjectionConfig; +import com.geedgenetworks.core.pojo.TableConfig; +import com.geedgenetworks.core.processor.table.TableProcessor; import com.geedgenetworks.core.processor.aggregate.AggregateProcessor; import com.geedgenetworks.core.processor.projection.ProjectionProcessor; import com.typesafe.config.Config; @@ -35,6 +34,9 @@ public abstract class AbstractProcessorExecutor extends AbstractExecutor 0) { + tableConfig.setParallelism(node.getParallelism()); + } + try { + dataStream = + tableProcessor.processorFunction( + dataStream, tableConfig, jobRuntimeEnvironment.getStreamExecutionEnvironment().getConfig()); + } catch (Exception e) { + throw new JobExecuteException("Create orderby pipeline instance failed!", e); + } + return dataStream; + } protected SingleOutputStreamOperator executeAggregateProcessor(SingleOutputStreamOperator dataStream, Node node, AggregateConfig aggregateConfig) throws JobExecuteException { AggregateProcessor aggregateProcessor; @@ -114,6 +141,9 @@ public abstract class AbstractProcessorExecutor extends AbstractExecutor value, Config config) { + + CheckResult result = CheckConfigUtil.checkAtLeastOneExists(config.getConfig(key), + TableConfigOptions.OUTPUT_FIELDS.key(), + TableConfigOptions.REMOVE_FIELDS.key(), + TableConfigOptions.FUNCTIONS.key()); + if (!result.isSuccess()) { + throw new ConfigValidationException(CommonErrorCode.CONFIG_VALIDATION_FAILED, String.format( + "Table processor: %s, At least one of [%s] should be specified.", + key, String.join(",", + TableConfigOptions.OUTPUT_FIELDS.key(), + TableConfigOptions.REMOVE_FIELDS.key(), + TableConfigOptions.FUNCTIONS.key()))); + } + + TableConfig tableConfig = new JSONObject(value).toJavaObject(TableConfig.class); + tableConfig.setName(key); + return tableConfig; + } } diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/TableConfigOptions.java b/groot-common/src/main/java/com/geedgenetworks/common/config/TableConfigOptions.java new file mode 100644 index 0000000..480496d --- /dev/null +++ b/groot-common/src/main/java/com/geedgenetworks/common/config/TableConfigOptions.java @@ -0,0 +1,34 @@ +package com.geedgenetworks.common.config; + +import com.alibaba.fastjson2.TypeReference; +import com.geedgenetworks.common.udf.UDFContext; + +import java.util.List; + +public interface TableConfigOptions { + Option TYPE = Options.key("type") + .stringType() + .noDefaultValue() + .withDescription("The type of processor."); + + Option> OUTPUT_FIELDS = Options.key("output_fields") + .listType() + .noDefaultValue() + .withDescription("The fields to be outputted."); + + Option> REMOVE_FIELDS = Options.key("remove_fields") + .listType() + .noDefaultValue() + .withDescription("The fields to be removed."); + + Option> FUNCTIONS = Options.key("functions") + .type(new TypeReference>() {}) + .noDefaultValue() + .withDescription("The functions to be executed."); + + + + + + +} diff --git a/groot-common/src/main/java/com/geedgenetworks/common/udf/TableFunction.java b/groot-common/src/main/java/com/geedgenetworks/common/udf/TableFunction.java new file mode 100644 index 0000000..e602291 --- /dev/null +++ b/groot-common/src/main/java/com/geedgenetworks/common/udf/TableFunction.java @@ -0,0 +1,20 @@ +package com.geedgenetworks.common.udf; + +import com.geedgenetworks.common.Event; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.util.Collector; + +import java.io.Serializable; +import java.util.List; + +public interface TableFunction extends Serializable { + + void open(RuntimeContext runtimeContext, UDFContext udfContext); + + List evaluate(Event event); + + String functionName(); + + void close(); + +} diff --git a/groot-common/src/main/resources/udf.plugins b/groot-common/src/main/resources/udf.plugins index 1b7fca4..f5a4c3f 100644 --- a/groot-common/src/main/resources/udf.plugins +++ b/groot-common/src/main/resources/udf.plugins @@ -20,4 +20,5 @@ com.geedgenetworks.core.udf.udaf.CollectSet com.geedgenetworks.core.udf.udaf.LongCount com.geedgenetworks.core.udf.udaf.Mean com.geedgenetworks.core.udf.udaf.LastValue -com.geedgenetworks.core.udf.udaf.FirstValue \ No newline at end of file +com.geedgenetworks.core.udf.udaf.FirstValue +com.geedgenetworks.core.udf.udtf.UnRoll \ No newline at end of file diff --git a/groot-core/src/main/java/com/geedgenetworks/core/pojo/TableConfig.java b/groot-core/src/main/java/com/geedgenetworks/core/pojo/TableConfig.java new file mode 100644 index 0000000..3efb8e1 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/pojo/TableConfig.java @@ -0,0 +1,15 @@ +package com.geedgenetworks.core.pojo; + +import com.geedgenetworks.common.udf.UDFContext; +import lombok.Data; +import lombok.EqualsAndHashCode; + +import java.util.List; + +@EqualsAndHashCode(callSuper = true) +@Data +public class TableConfig extends ProcessorConfig { + + private List functions; + private String format; +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/UdfEntity.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/UdfEntity.java index 34267a6..ab6a6f5 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/UdfEntity.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/UdfEntity.java @@ -3,6 +3,7 @@ package com.geedgenetworks.core.processor.projection; import com.geedgenetworks.common.udf.AggregateFunction; import com.geedgenetworks.common.udf.ScalarFunction; +import com.geedgenetworks.common.udf.TableFunction; import com.geedgenetworks.common.udf.UDFContext; import com.googlecode.aviator.Expression; import lombok.Data; @@ -13,6 +14,7 @@ import java.io.Serializable; public class UdfEntity implements Serializable { private ScalarFunction scalarFunction; private AggregateFunction aggregateFunction; + private TableFunction tableFunction; private Expression filterExpression; private String name; private String className; diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessor.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessor.java new file mode 100644 index 0000000..4078997 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessor.java @@ -0,0 +1,7 @@ +package com.geedgenetworks.core.processor.table; + +import com.geedgenetworks.core.pojo.TableConfig; +import com.geedgenetworks.core.processor.Processor; +public interface TableProcessor extends Processor { + +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorFunction.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorFunction.java new file mode 100644 index 0000000..7b6a5e2 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorFunction.java @@ -0,0 +1,138 @@ +package com.geedgenetworks.core.processor.table; + +import com.alibaba.fastjson.JSON; +import com.geedgenetworks.common.Constants; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.TableFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.common.utils.ColumnUtil; +import com.geedgenetworks.core.metrics.InternalMetrics; +import com.geedgenetworks.core.pojo.TableConfig; +import com.geedgenetworks.core.processor.projection.UdfEntity; +import com.google.common.collect.Lists; +import com.googlecode.aviator.AviatorEvaluator; +import com.googlecode.aviator.AviatorEvaluatorInstance; +import com.googlecode.aviator.Expression; +import com.googlecode.aviator.Options; +import com.googlecode.aviator.exception.ExpressionRuntimeException; +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Collector; +import org.checkerframework.checker.units.qual.A; + +import java.util.*; + +import static com.geedgenetworks.core.utils.UDFUtils.filterExecute; +import static com.geedgenetworks.core.utils.UDFUtils.getClassReflect; +@Slf4j +public class TableProcessorFunction extends RichFlatMapFunction { + private LinkedList functions; + private final TableConfig tableConfig; + private transient InternalMetrics internalMetrics; + + public TableProcessorFunction(TableConfig tableConfig) { + this.tableConfig = tableConfig; + } + + @Override + public void open(Configuration parameters) { + functions = Lists.newLinkedList(); + try { + this.internalMetrics = new InternalMetrics(getRuntimeContext()); + List udfContexts = tableConfig.getFunctions(); + if (udfContexts == null || udfContexts.isEmpty()) { + return; + } + Configuration configuration = (Configuration) getRuntimeContext().getExecutionConfig().getGlobalJobParameters(); + List udfClassNameLists = JSON.parseObject(configuration.toMap().get(Constants.SYSPROP_UDF_PLUGIN_CONFIG), List.class); + Map udfClassReflect = getClassReflect(udfClassNameLists); + for (UDFContext udfContext : udfContexts) { + Expression filterExpression = null; + UdfEntity udfEntity = new UdfEntity(); + // 平台注册的函数包含任务中配置的函数则对函数进行实例化 + if (udfClassReflect.containsKey(udfContext.getFunction())) { + Class cls = Class.forName(udfClassReflect.get(udfContext.getFunction())); + TableFunction tableFunction = (TableFunction) cls.getConstructor().newInstance(); + tableFunction.open(getRuntimeContext(), udfContext); + // 函数如果包含filter,对表达式进行编译 + if (udfContext.getFilter() != null) { + AviatorEvaluatorInstance instance = AviatorEvaluator.getInstance(); + instance.setCachedExpressionByDefault(true); + instance.setOption(Options.OPTIMIZE_LEVEL, AviatorEvaluator.EVAL); + instance.setFunctionMissing(null); + filterExpression = instance.compile(udfContext.getFilter(), true); + } + udfEntity.setTableFunction(tableFunction); + udfEntity.setFilterExpression(filterExpression); + udfEntity.setName(udfContext.getFunction()); + udfEntity.setClassName(udfClassReflect.get(udfContext.getFunction())); + functions.add(udfEntity); + } else { + throw new GrootStreamRuntimeException(CommonErrorCode.UNSUPPORTED_OPERATION, + "Unsupported UDTF: " + udfContext.getFunction()); + } + + } + } catch (Exception e) { + throw new GrootStreamRuntimeException(CommonErrorCode.UNSUPPORTED_OPERATION, "Initialization UDTF failed!", e); + } + } + + + @Override + public void flatMap(Event event, Collector out) throws Exception { + internalMetrics.incrementInEvents(); + int errorCount = 0; + List events = new ArrayList<>(); + events.add(event); + for (UdfEntity udfEntity : functions) { + List newEvents = new ArrayList<>(); + for(int i=0;i0){ + internalMetrics.incrementErrorEvents(); + } + for(Event newEvent:events){ + if (tableConfig.getOutput_fields() != null + && !tableConfig.getOutput_fields().isEmpty()) { + newEvent.setExtractedFields( + ColumnUtil.columnSelector( + newEvent.getExtractedFields(), tableConfig.getOutput_fields())); + } + if (tableConfig.getRemove_fields() != null + && !tableConfig.getRemove_fields().isEmpty()) { + newEvent.setExtractedFields( + ColumnUtil.columnRemover( + newEvent.getExtractedFields(), tableConfig.getRemove_fields())); + } + if (!event.isDropped()) { + out.collect(newEvent); + internalMetrics.incrementOutEvents(); + } else { + internalMetrics.incrementDroppedEvents(); + } + + } + + + } +} \ No newline at end of file diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java new file mode 100644 index 0000000..f36f8db --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java @@ -0,0 +1,33 @@ +package com.geedgenetworks.core.processor.table; + +import com.geedgenetworks.common.Event; +import com.geedgenetworks.core.pojo.TableConfig; +import com.geedgenetworks.core.processor.projection.ProjectionProcessFunction; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; + +import java.time.Duration; + +public class TableProcessorImpl implements TableProcessor { + + @Override + public SingleOutputStreamOperator processorFunction(SingleOutputStreamOperator grootEventSingleOutputStreamOperator, TableConfig tableConfig, ExecutionConfig config) throws Exception { + + if (tableConfig.getParallelism() != 0) { + return grootEventSingleOutputStreamOperator + .flatMap(new TableProcessorFunction(tableConfig)) + .setParallelism(tableConfig.getParallelism()) + .name(tableConfig.getName()); + } else { + return grootEventSingleOutputStreamOperator + .flatMap(new TableProcessorFunction(tableConfig)) + .name(tableConfig.getName()); + } + } + + @Override + public String type() { + return "table"; + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/UnRoll.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/UnRoll.java new file mode 100644 index 0000000..82bea7b --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/UnRoll.java @@ -0,0 +1,132 @@ +package com.geedgenetworks.core.udf.udtf; + +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.TableFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.googlecode.aviator.AviatorEvaluator; +import com.googlecode.aviator.AviatorEvaluatorInstance; +import com.googlecode.aviator.Expression; +import com.googlecode.aviator.Options; +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.api.common.functions.RuntimeContext; + +import java.util.*; + + +@Slf4j +public class UnRoll implements TableFunction { + + private String lookupFieldName; + private String outputFieldName; + private Expression compiledExp; + private String expression; + private String outputFieldType; + + + @Override + public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } + this.lookupFieldName = udfContext.getLookup_fields().get(0); + this.outputFieldName = udfContext.getOutput_fields().get(0); + this.outputFieldType="object"; + if(udfContext.getParameters()==null ){ + expression=""; + } + else { + this.outputFieldType=udfContext.getParameters().getOrDefault("output_field_type", "object").toString().trim(); + expression=udfContext.getParameters().getOrDefault("path", "").toString().trim(); + } + if(!expression.isEmpty()){ + AviatorEvaluatorInstance instance = AviatorEvaluator.getInstance(); + instance.setCachedExpressionByDefault(true); + instance.setOption(Options.OPTIMIZE_LEVEL, AviatorEvaluator.EVAL); + instance.setFunctionMissing(null); + compiledExp = instance.compile("event."+expression, true); + } + + } + + @Override + public List evaluate(Event event) { + try { + if(event.getExtractedFields().containsKey(lookupFieldName)){ + Object object; + if(event.getExtractedFields().get(lookupFieldName) instanceof String){ + object = JSONObject.parseObject((String) event.getExtractedFields().get(lookupFieldName), Object.class); + } + else { + object = event.getExtractedFields().get(lookupFieldName); + } + try { + if(compiledExp!=null){ + Object obj = compiledExp.execute(compiledExp.newEnv("event", object)); + if(obj instanceof List) { + return parseList(obj,event); + } + else if(obj instanceof String){ + object = JSONObject.parseObject((String)obj, Object.class); + if(object instanceof List){ + return parseList(object,event); + } + else { + log.error("Invalid unroll ! Object is not instance of list. expression=" +expression); + } + }else { + log.error("Invalid unroll ! Object is not instance of String or List. expression=" +expression); + } + } + else { + if(object instanceof List){ + return parseList(object,event); + } + else { + log.error("Invalid unroll ! Object is not instance of list. "); + } + } + }catch (Exception e) { + log.error("Invalid unroll ! expression=" +expression + " Exception :" + e.getMessage()); + } + } + }catch (Exception e) { + log.error("Invalid parseObject ! expression=" +expression + " Exception :" + e.getMessage()); + } + return Collections.singletonList(event); + } + + private List parseList(Object object,Event event) { + List list = (List) object; + List eventList = new ArrayList<>(); + for (Object obj : list) { + Event newEvent = new Event(); + newEvent.setExtractedFields(new HashMap<>()); + newEvent.getExtractedFields().putAll(event.getExtractedFields()); + newEvent.getExtractedFields().remove(lookupFieldName); + if("string".equals(outputFieldType)) { + String jsonString = JSON.toJSONString(obj); + newEvent.getExtractedFields().put(outputFieldName, jsonString); + } + else { + newEvent.getExtractedFields().put(outputFieldName, obj); + } + eventList.add(newEvent); + } + return eventList; + } + + @Override + public String functionName() { + return "UNROLL"; + } + + @Override + public void close() { + + } + +} diff --git a/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.Processor b/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.Processor index 727b42b..1f32ffa 100644 --- a/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.Processor +++ b/groot-core/src/main/resources/META-INF/services/com.geedgenetworks.core.processor.Processor @@ -1,2 +1,3 @@ com.geedgenetworks.core.processor.aggregate.AggregateProcessorImpl -com.geedgenetworks.core.processor.projection.ProjectionProcessorImpl \ No newline at end of file +com.geedgenetworks.core.processor.projection.ProjectionProcessorImpl +com.geedgenetworks.core.processor.table.TableProcessorImpl \ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnRollFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnRollFunctionTest.java new file mode 100644 index 0000000..5686a42 --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnRollFunctionTest.java @@ -0,0 +1,132 @@ +package com.geedgenetworks.core.udf.test.table; + +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.core.udf.udtf.UnRoll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class UnRollFunctionTest { + + private static Map nestedMap; + @BeforeAll + public static void setUp() { + nestedMap = Map.of( + "k1", List.of( + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "52:d4:18:c7:e5:11"), + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") + ), + "k2", Map.of("name", "[{\"tunnels_schema_type\":\"MULTIPATH_ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"c2s_destination_mac\":\"10:70:fd:03:c2:6d\",\"s2c_source_mac\":\"10:70:fd:03:c2:6d\",\"s2c_destination_mac\":\"10:70:fd:03:c2:6c\"},{\"tunnels_schema_type\":\"MULTIPATH_ETHERNET\",\"source_mac\":\"ff:ff:ff:ff:ff:ff\",\"c2s_destination_mac\":\"10:70:fd:03:c2:6d\",\"s2c_source_mac\":\"10:70:fd:03:c2:6d\",\"s2c_destination_mac\":\"10:70:fd:03:c2:6c\"}]", "lastName", "{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}"), + "k3","[{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"},{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"ff:ff:ff:ff:ff:ff\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}]", + "k4","{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}", + "k5", Map.of("name",List.of( + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "52:d4:18:c7:e5:11"), + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") + ) + ), "k6", Map.of("name",List.of( + Map.of("name2", List.of( + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "52:d4:18:c7:e5:11"), + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") + ), "source_mac", "52:d4:18:c7:e5:11"), + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") + ) + ) + ); + } + + // 测试方法 + @Test + public void testUnrollFunction1() { + + + UDFContext udfContext = new UDFContext(); + udfContext.setLookup_fields(List.of("k1")); + udfContext.setOutput_fields(List.of("newk1")); + Map params = new HashMap<>(); + params.put("path", ""); + udfContext.setParameters(params); + UnRoll unroll = new UnRoll(); + unroll.open(null, udfContext); + Event event = new Event(); + event.setExtractedFields(nestedMap); + List result = unroll.evaluate(event); + assertEquals(2, result.size()); + Map map1 = (Map) result.get(0).getExtractedFields().get("newk1"); + assertEquals("52:d4:18:c7:e5:11", map1.get("source_mac")); + Map map2 = (Map) result.get(1).getExtractedFields().get("newk1"); + assertEquals("ff:ff:ff:ff:ff:ff", map2.get("source_mac")); + + + + } + + @Test + public void testUnrollFunction2() { + UDFContext udfContext = new UDFContext(); + UnRoll unroll = new UnRoll(); + Event event = new Event(); + event.setExtractedFields(nestedMap); + udfContext.setLookup_fields(List.of("k3")); + udfContext.setOutput_fields(List.of("newk3")); + unroll.open(null, udfContext); + List result3 = unroll.evaluate(event); + assertEquals(2, result3.size()); + Map map3 = (Map) result3.get(0).getExtractedFields().get("newk3"); + assertEquals("52:d4:18:c7:e5:11", map3.get("source_mac")); + Map map4 = (Map) result3.get(1).getExtractedFields().get("newk3"); + assertEquals("ff:ff:ff:ff:ff:ff", map4.get("source_mac")); + + + Map params = new HashMap<>(); + params.put("path", "name"); + udfContext.setParameters(params); + event.setExtractedFields(nestedMap); + udfContext.setLookup_fields(List.of("k2")); + udfContext.setOutput_fields(List.of("newk2")); + unroll.open(null, udfContext); + List result2 = unroll.evaluate(event); + assertEquals(2, result2.size()); + Map map5 = (Map) result2.get(0).getExtractedFields().get("newk2"); + assertEquals("52:d4:18:c7:e5:11", map5.get("source_mac")); + Map map6 = (Map) result2.get(1).getExtractedFields().get("newk2"); + assertEquals("ff:ff:ff:ff:ff:ff", map6.get("source_mac")); + + + Map params1 = new HashMap<>(); + params.put("path", "name.0.name2"); + udfContext.setParameters(params1); + event.setExtractedFields(nestedMap); + udfContext.setLookup_fields(List.of("k6")); + udfContext.setOutput_fields(List.of("newk6")); + unroll.open(null, udfContext); + List result6 = unroll.evaluate(event); + assertEquals(2, result6.size()); + Map map9 = (Map) result6.get(0).getExtractedFields().get("newk6"); + assertEquals("52:d4:18:c7:e5:11", map9.get("source_mac")); + Map map10 = (Map) result6.get(1).getExtractedFields().get("newk6"); + assertEquals("ff:ff:ff:ff:ff:ff", map10.get("source_mac")); + } + @Test + public void testUnrollFunction4() { + + + UDFContext udfContext = new UDFContext(); + UnRoll unroll = new UnRoll(); + Event event = new Event(); + Map params = new HashMap<>(); + udfContext.setParameters(params); + event.setExtractedFields(nestedMap); + udfContext.setLookup_fields(List.of("k4")); + udfContext.setOutput_fields(List.of("newk4")); + unroll.open(null, udfContext); + List result2 = unroll.evaluate(event); + assertEquals(1, result2.size()); + + } +} -- cgit v1.2.3 From c144317f1d1a0f643ccfae93a9da1d96a5d7c835 Mon Sep 17 00:00:00 2001 From: wangkuan Date: Tue, 13 Aug 2024 17:14:57 +0800 Subject: [improve][core]拆分聚合函数open方法,较少调用次数,优化性能 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/geedgenetworks/common/udf/AggregateFunction.java | 7 ++++--- .../processor/aggregate/AggregateProcessorFunction.java | 7 ++++--- .../java/com/geedgenetworks/core/udf/udaf/CollectList.java | 14 +++++++------- .../java/com/geedgenetworks/core/udf/udaf/CollectSet.java | 10 +++++----- .../java/com/geedgenetworks/core/udf/udaf/FirstValue.java | 13 +++++++------ .../java/com/geedgenetworks/core/udf/udaf/LastValue.java | 10 ++++------ .../java/com/geedgenetworks/core/udf/udaf/LongCount.java | 11 ++++------- .../main/java/com/geedgenetworks/core/udf/udaf/Mean.java | 12 +++++------- .../java/com/geedgenetworks/core/udf/udaf/NumberSum.java | 9 ++++++--- .../core/udf/test/aggregate/CollectListTest.java | 5 +++-- .../core/udf/test/aggregate/CollectSetTest.java | 4 ++-- .../core/udf/test/aggregate/FirstValueTest.java | 6 +++--- .../core/udf/test/aggregate/LastValueTest.java | 6 +++--- .../core/udf/test/aggregate/LongCountTest.java | 6 +++--- .../geedgenetworks/core/udf/test/aggregate/MeanTest.java | 14 +++++++------- .../core/udf/test/aggregate/NumberSumTest.java | 6 +++--- 16 files changed, 70 insertions(+), 70 deletions(-) diff --git a/groot-common/src/main/java/com/geedgenetworks/common/udf/AggregateFunction.java b/groot-common/src/main/java/com/geedgenetworks/common/udf/AggregateFunction.java index 98450fd..455073f 100644 --- a/groot-common/src/main/java/com/geedgenetworks/common/udf/AggregateFunction.java +++ b/groot-common/src/main/java/com/geedgenetworks/common/udf/AggregateFunction.java @@ -7,7 +7,9 @@ import java.io.Serializable; public interface AggregateFunction extends Serializable { - Accumulator open(UDFContext udfContext,Accumulator acc); + void open(UDFContext udfContext); + + Accumulator initAccumulator(Accumulator acc); Accumulator add(Event val, Accumulator acc); @@ -15,6 +17,5 @@ public interface AggregateFunction extends Serializable { Accumulator getResult(Accumulator acc); - void close(); - + default void close(){}; } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorFunction.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorFunction.java index c07374e..803fefc 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorFunction.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorFunction.java @@ -71,23 +71,24 @@ public class AggregateProcessorFunction implements org.apache.flink.api.common.f } } + for (UdfEntity udfEntity : functions) { + udfEntity.getAggregateFunction().open(udfEntity.getUdfContext()); + } } catch (Exception e) { throw new GrootStreamRuntimeException(CommonErrorCode.UNSUPPORTED_OPERATION, "Initialization UDAF failed!", e); } } - @Override public Accumulator createAccumulator() { Map map = new HashMap<>(); Accumulator accumulator = new Accumulator(); accumulator.setMetricsFields(map); for (UdfEntity udfEntity : functions) { - udfEntity.getAggregateFunction().open(udfEntity.getUdfContext(), accumulator); + udfEntity.getAggregateFunction().initAccumulator(accumulator); } return accumulator; } - @Override public Accumulator add(Event event, Accumulator accumulator) { diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java index 4a43163..423eff9 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java @@ -22,6 +22,8 @@ import com.geedgenetworks.common.exception.CommonErrorCode; import com.geedgenetworks.common.exception.GrootStreamRuntimeException; import com.geedgenetworks.common.udf.AggregateFunction; import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.core.processor.projection.UdfEntity; + import java.util.*; /** @@ -32,9 +34,8 @@ public class CollectList implements AggregateFunction { private String lookupField; private String outputField; - @Override - public Accumulator open(UDFContext udfContext,Accumulator acc) { + public void open(UDFContext udfContext) { if(udfContext.getLookup_fields()==null ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } @@ -45,11 +46,14 @@ public class CollectList implements AggregateFunction { else { outputField = lookupField; } + + } + @Override + public Accumulator initAccumulator(Accumulator acc) { acc.getMetricsFields().put(outputField, new ArrayList<>()); return acc; } - @Override public Accumulator add(Event event, Accumulator acc) { if(event.getExtractedFields().containsKey(lookupField)){ @@ -71,8 +75,4 @@ public class CollectList implements AggregateFunction { return acc; } - @Override - public void close() { - - } } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java index a425118..b4dfb14 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java @@ -8,6 +8,7 @@ import com.geedgenetworks.common.exception.GrootStreamRuntimeException; import com.geedgenetworks.common.udf.AggregateFunction; import com.geedgenetworks.common.udf.UDFContext; +import java.util.ArrayList; import java.util.HashSet; import java.util.Set; @@ -21,7 +22,7 @@ public class CollectSet implements AggregateFunction { @Override - public Accumulator open(UDFContext udfContext,Accumulator acc) { + public void open(UDFContext udfContext) { if(udfContext.getLookup_fields()==null ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } @@ -32,11 +33,13 @@ public class CollectSet implements AggregateFunction { else { outputField = lookupField; } + } + @Override + public Accumulator initAccumulator(Accumulator acc) { acc.getMetricsFields().put(outputField, new HashSet<>()); return acc; } - @Override public Accumulator add(Event event, Accumulator acc) { if(event.getExtractedFields().containsKey(lookupField)){ @@ -58,8 +61,5 @@ public class CollectSet implements AggregateFunction { return acc; } - @Override - public void close() { - } } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java index 27490ef..6301a01 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java @@ -23,6 +23,8 @@ import com.geedgenetworks.common.exception.GrootStreamRuntimeException; import com.geedgenetworks.common.udf.AggregateFunction; import com.geedgenetworks.common.udf.UDFContext; +import java.util.ArrayList; + /** * Collects elements within a group and returns the list of aggregated objects */ @@ -33,7 +35,7 @@ public class FirstValue implements AggregateFunction { @Override - public Accumulator open(UDFContext udfContext,Accumulator acc) { + public void open(UDFContext udfContext) { if(udfContext.getLookup_fields()==null ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } @@ -44,9 +46,12 @@ public class FirstValue implements AggregateFunction { else { outputField = lookupField; } - return acc; } + @Override + public Accumulator initAccumulator(Accumulator acc) { + return acc; + } @Override public Accumulator add(Event event, Accumulator acc) { @@ -66,8 +71,4 @@ public class FirstValue implements AggregateFunction { return acc; } - @Override - public void close() { - - } } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java index 4adafd4..f27a2e6 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java @@ -36,7 +36,7 @@ public class LastValue implements AggregateFunction { @Override - public Accumulator open(UDFContext udfContext,Accumulator acc) { + public void open(UDFContext udfContext) { if(udfContext.getLookup_fields()==null ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } @@ -47,10 +47,12 @@ public class LastValue implements AggregateFunction { else { outputField = lookupField; } + } + @Override + public Accumulator initAccumulator(Accumulator acc) { return acc; } - @Override public Accumulator add(Event event, Accumulator acc) { if(event.getExtractedFields().containsKey(lookupField)){ @@ -69,8 +71,4 @@ public class LastValue implements AggregateFunction { return acc; } - @Override - public void close() { - - } } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java index 5662935..ea33271 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java @@ -13,15 +13,16 @@ public class LongCount implements AggregateFunction { @Override - public Accumulator open(UDFContext udfContext,Accumulator acc){ + public void open(UDFContext udfContext){ if(udfContext.getOutput_fields()==null ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } outputField = udfContext.getOutput_fields().get(0); + } + @Override + public Accumulator initAccumulator(Accumulator acc) { return acc; } - - @Override public Accumulator add(Event event, Accumulator acc) { @@ -39,9 +40,5 @@ public class LongCount implements AggregateFunction { return acc; } - @Override - public void close() { - - } } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java index 88e4be6..2a615ef 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java @@ -17,7 +17,7 @@ public class Mean implements AggregateFunction { private Integer precision; private DecimalFormat df; @Override - public Accumulator open(UDFContext udfContext,Accumulator acc){ + public void open(UDFContext udfContext){ if(udfContext.getLookup_fields()==null ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); @@ -41,11 +41,14 @@ public class Mean implements AggregateFunction { }else { precision = -1; } + + } + @Override + public Accumulator initAccumulator(Accumulator acc) { acc.getMetricsFields().put(outputField,new OnlineStatistics()); return acc; } - @Override public Accumulator add(Event event, Accumulator acc) { @@ -76,9 +79,4 @@ public class Mean implements AggregateFunction { return acc; } - @Override - public void close() { - - } - } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java index 4ed3143..01e9a5b 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java @@ -6,6 +6,7 @@ import com.geedgenetworks.common.exception.CommonErrorCode; import com.geedgenetworks.common.exception.GrootStreamRuntimeException; import com.geedgenetworks.common.udf.AggregateFunction; import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.core.pojo.OnlineStatistics; public class NumberSum implements AggregateFunction { @@ -14,7 +15,7 @@ public class NumberSum implements AggregateFunction { @Override - public Accumulator open(UDFContext udfContext,Accumulator acc){ + public void open(UDFContext udfContext){ if(udfContext.getLookup_fields()==null ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } @@ -25,10 +26,12 @@ public class NumberSum implements AggregateFunction { else { outputField = lookupField; } - return acc; } - + @Override + public Accumulator initAccumulator(Accumulator acc) { + return acc; + } @Override public Accumulator add(Event event, Accumulator acc) { diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java index a01edb3..b0d846b 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java @@ -33,7 +33,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; public class CollectListTest { @Test - public void testNumberSumTest() throws ParseException { + public void test() throws ParseException { List arr = List.of("192.168.1.1", "192.168.1.2", "192.168.1.3", "192.168.1.4"); excute(arr); @@ -49,7 +49,8 @@ public class CollectListTest { Map metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); accumulator.setMetricsFields(metricsFields); - Accumulator agg = collectList.open(udfContext,accumulator); + collectList.open(udfContext); + Accumulator agg = collectList.initAccumulator(accumulator); for (String o : arr) { Event event = new Event(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java index ae69d7c..ea4fe8d 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java @@ -47,8 +47,8 @@ public class CollectSetTest { Map metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); accumulator.setMetricsFields(metricsFields); - Accumulator agg = collectSet.open(udfContext,accumulator); - + collectSet.open(udfContext); + Accumulator agg = collectSet.initAccumulator(accumulator); for (String o : arr) { Event event = new Event(); Map extractedFields = new HashMap<>(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java index 2c4d460..506f6de 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java @@ -31,7 +31,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; public class FirstValueTest { @Test - public void testNumberSumTest() throws ParseException { + public void test() throws ParseException { List arr = List.of("192.168.1.1", "192.168.1.2", "192.168.1.3", "192.168.1.4","192.168.1.4"); excute(arr); @@ -47,8 +47,8 @@ public class FirstValueTest { Map metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); accumulator.setMetricsFields(metricsFields); - Accumulator agg = firstValue.open(udfContext,accumulator); - + firstValue.open(udfContext); + Accumulator agg = firstValue.initAccumulator(accumulator); for (String o : arr) { Event event = new Event(); Map extractedFields = new HashMap<>(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java index e9609f7..f8306cd 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java @@ -34,7 +34,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; public class LastValueTest { @Test - public void testNumberSumTest() throws ParseException { + public void test() throws ParseException { List arr = List.of("192.168.1.1", "192.168.1.2", "192.168.1.3", "192.168.1.4","192.168.1.4"); excute(arr); @@ -50,8 +50,8 @@ public class LastValueTest { Map metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); accumulator.setMetricsFields(metricsFields); - Accumulator agg = lastValue.open(udfContext,accumulator); - + lastValue.open(udfContext); + Accumulator agg = lastValue.initAccumulator(accumulator); for (String o : arr) { Event event = new Event(); Map extractedFields = new HashMap<>(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java index 3bde558..3c02499 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java @@ -35,7 +35,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; public class LongCountTest { @Test - public void testNumberSumTest() throws ParseException { + public void test() throws ParseException { Long[] longArr = new Long[]{1L, 2L, 3L, 4L}; excute(longArr); @@ -49,8 +49,8 @@ public class LongCountTest { Map metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); accumulator.setMetricsFields(metricsFields); - Accumulator agg = longCount.open(udfContext,accumulator); - + longCount.open(udfContext); + Accumulator agg = longCount.initAccumulator(accumulator); for (Number o : arr) { Event event = new Event(); Map extractedFields = new HashMap<>(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java index 807b7db..6deed0f 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java @@ -35,7 +35,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; public class MeanTest { @Test - public void testNumberSumTest() throws ParseException { + public void test() throws ParseException { Integer[] intArr1 = new Integer[]{1, 2, 3, 4}; Integer[] intArr2 = new Integer[]{1, 6, 3}; @@ -55,8 +55,8 @@ public class MeanTest { Map metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); accumulator.setMetricsFields(metricsFields); - Accumulator agg = mean.open(udfContext,accumulator); - + mean.open(udfContext); + Accumulator agg = mean.initAccumulator(accumulator); for (Number o : arr) { Event event = new Event(); Map extractedFields = new HashMap<>(); @@ -80,8 +80,8 @@ public class MeanTest { Map metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); accumulator.setMetricsFields(metricsFields); - Accumulator agg = mean.open(udfContext,accumulator); - + mean.open(udfContext); + Accumulator agg = mean.initAccumulator(accumulator); for (Number o : arr) { Event event = new Event(); Map extractedFields = new HashMap<>(); @@ -103,8 +103,8 @@ public class MeanTest { Map metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); accumulator.setMetricsFields(metricsFields); - Accumulator agg = mean.open(udfContext,accumulator); - + mean.open(udfContext); + Accumulator agg = mean.initAccumulator(accumulator); for (Number o : arr) { Event event = new Event(); Map extractedFields = new HashMap<>(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java index a1cd54e..d0d3d2c 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java @@ -31,7 +31,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; public class NumberSumTest { @Test - public void testNumberSumTest() throws ParseException { + public void test() throws ParseException { Integer[] intArr = new Integer[]{1, 2, 3, 4}; Long[] longArr = new Long[]{1L, 2L, 3L, 4L}; @@ -52,8 +52,8 @@ public class NumberSumTest { Map metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); accumulator.setMetricsFields(metricsFields); - Accumulator agg = numberSum.open(udfContext,accumulator); - + numberSum.open(udfContext); + Accumulator agg = numberSum.initAccumulator(accumulator); for (Number o : arr) { Event event = new Event(); Map extractedFields = new HashMap<>(); -- cgit v1.2.3 From eddd8cb4ae9d0c2bad4c4dfe892ab73947d269ef Mon Sep 17 00:00:00 2001 From: wangkuan Date: Wed, 14 Aug 2024 15:13:52 +0800 Subject: [improve][core]修改UnRoll函数名称为Unroll MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- config/udf.plugins | 2 +- groot-common/src/main/resources/udf.plugins | 2 +- .../com/geedgenetworks/core/udf/udtf/UnRoll.java | 132 --------------------- .../com/geedgenetworks/core/udf/udtf/Unroll.java | 132 +++++++++++++++++++++ .../core/udf/test/table/UnRollFunctionTest.java | 132 --------------------- .../core/udf/test/table/UnrollFunctionTest.java | 132 +++++++++++++++++++++ 6 files changed, 266 insertions(+), 266 deletions(-) delete mode 100644 groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/UnRoll.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java delete mode 100644 groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnRollFunctionTest.java create mode 100644 groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java diff --git a/config/udf.plugins b/config/udf.plugins index 9eb32c4..0d6c0fe 100644 --- a/config/udf.plugins +++ b/config/udf.plugins @@ -22,4 +22,4 @@ com.geedgenetworks.core.udf.udaf.LongCount com.geedgenetworks.core.udf.udaf.Mean com.geedgenetworks.core.udf.udaf.LastValue com.geedgenetworks.core.udf.udaf.FirstValue -com.geedgenetworks.core.udf.udtf.UnRoll \ No newline at end of file +com.geedgenetworks.core.udf.udtf.Unroll \ No newline at end of file diff --git a/groot-common/src/main/resources/udf.plugins b/groot-common/src/main/resources/udf.plugins index f5a4c3f..ca0a7ac 100644 --- a/groot-common/src/main/resources/udf.plugins +++ b/groot-common/src/main/resources/udf.plugins @@ -21,4 +21,4 @@ com.geedgenetworks.core.udf.udaf.LongCount com.geedgenetworks.core.udf.udaf.Mean com.geedgenetworks.core.udf.udaf.LastValue com.geedgenetworks.core.udf.udaf.FirstValue -com.geedgenetworks.core.udf.udtf.UnRoll \ No newline at end of file +com.geedgenetworks.core.udf.udtf.Unroll \ No newline at end of file diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/UnRoll.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/UnRoll.java deleted file mode 100644 index 82bea7b..0000000 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/UnRoll.java +++ /dev/null @@ -1,132 +0,0 @@ -package com.geedgenetworks.core.udf.udtf; - -import com.alibaba.fastjson2.JSON; -import com.alibaba.fastjson2.JSONObject; -import com.geedgenetworks.common.Event; -import com.geedgenetworks.common.exception.CommonErrorCode; -import com.geedgenetworks.common.exception.GrootStreamRuntimeException; -import com.geedgenetworks.common.udf.TableFunction; -import com.geedgenetworks.common.udf.UDFContext; -import com.googlecode.aviator.AviatorEvaluator; -import com.googlecode.aviator.AviatorEvaluatorInstance; -import com.googlecode.aviator.Expression; -import com.googlecode.aviator.Options; -import lombok.extern.slf4j.Slf4j; -import org.apache.flink.api.common.functions.RuntimeContext; - -import java.util.*; - - -@Slf4j -public class UnRoll implements TableFunction { - - private String lookupFieldName; - private String outputFieldName; - private Expression compiledExp; - private String expression; - private String outputFieldType; - - - @Override - public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null){ - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); - } - this.lookupFieldName = udfContext.getLookup_fields().get(0); - this.outputFieldName = udfContext.getOutput_fields().get(0); - this.outputFieldType="object"; - if(udfContext.getParameters()==null ){ - expression=""; - } - else { - this.outputFieldType=udfContext.getParameters().getOrDefault("output_field_type", "object").toString().trim(); - expression=udfContext.getParameters().getOrDefault("path", "").toString().trim(); - } - if(!expression.isEmpty()){ - AviatorEvaluatorInstance instance = AviatorEvaluator.getInstance(); - instance.setCachedExpressionByDefault(true); - instance.setOption(Options.OPTIMIZE_LEVEL, AviatorEvaluator.EVAL); - instance.setFunctionMissing(null); - compiledExp = instance.compile("event."+expression, true); - } - - } - - @Override - public List evaluate(Event event) { - try { - if(event.getExtractedFields().containsKey(lookupFieldName)){ - Object object; - if(event.getExtractedFields().get(lookupFieldName) instanceof String){ - object = JSONObject.parseObject((String) event.getExtractedFields().get(lookupFieldName), Object.class); - } - else { - object = event.getExtractedFields().get(lookupFieldName); - } - try { - if(compiledExp!=null){ - Object obj = compiledExp.execute(compiledExp.newEnv("event", object)); - if(obj instanceof List) { - return parseList(obj,event); - } - else if(obj instanceof String){ - object = JSONObject.parseObject((String)obj, Object.class); - if(object instanceof List){ - return parseList(object,event); - } - else { - log.error("Invalid unroll ! Object is not instance of list. expression=" +expression); - } - }else { - log.error("Invalid unroll ! Object is not instance of String or List. expression=" +expression); - } - } - else { - if(object instanceof List){ - return parseList(object,event); - } - else { - log.error("Invalid unroll ! Object is not instance of list. "); - } - } - }catch (Exception e) { - log.error("Invalid unroll ! expression=" +expression + " Exception :" + e.getMessage()); - } - } - }catch (Exception e) { - log.error("Invalid parseObject ! expression=" +expression + " Exception :" + e.getMessage()); - } - return Collections.singletonList(event); - } - - private List parseList(Object object,Event event) { - List list = (List) object; - List eventList = new ArrayList<>(); - for (Object obj : list) { - Event newEvent = new Event(); - newEvent.setExtractedFields(new HashMap<>()); - newEvent.getExtractedFields().putAll(event.getExtractedFields()); - newEvent.getExtractedFields().remove(lookupFieldName); - if("string".equals(outputFieldType)) { - String jsonString = JSON.toJSONString(obj); - newEvent.getExtractedFields().put(outputFieldName, jsonString); - } - else { - newEvent.getExtractedFields().put(outputFieldName, obj); - } - eventList.add(newEvent); - } - return eventList; - } - - @Override - public String functionName() { - return "UNROLL"; - } - - @Override - public void close() { - - } - -} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java new file mode 100644 index 0000000..3931743 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java @@ -0,0 +1,132 @@ +package com.geedgenetworks.core.udf.udtf; + +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.TableFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.googlecode.aviator.AviatorEvaluator; +import com.googlecode.aviator.AviatorEvaluatorInstance; +import com.googlecode.aviator.Expression; +import com.googlecode.aviator.Options; +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.api.common.functions.RuntimeContext; + +import java.util.*; + + +@Slf4j +public class Unroll implements TableFunction { + + private String lookupFieldName; + private String outputFieldName; + private Expression compiledExp; + private String expression; + private String outputFieldType; + + + @Override + public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } + this.lookupFieldName = udfContext.getLookup_fields().get(0); + this.outputFieldName = udfContext.getOutput_fields().get(0); + this.outputFieldType="object"; + if(udfContext.getParameters()==null ){ + expression=""; + } + else { + this.outputFieldType=udfContext.getParameters().getOrDefault("output_field_type", "object").toString().trim(); + expression=udfContext.getParameters().getOrDefault("path", "").toString().trim(); + } + if(!expression.isEmpty()){ + AviatorEvaluatorInstance instance = AviatorEvaluator.getInstance(); + instance.setCachedExpressionByDefault(true); + instance.setOption(Options.OPTIMIZE_LEVEL, AviatorEvaluator.EVAL); + instance.setFunctionMissing(null); + compiledExp = instance.compile("event."+expression, true); + } + + } + + @Override + public List evaluate(Event event) { + try { + if(event.getExtractedFields().containsKey(lookupFieldName)){ + Object object; + if(event.getExtractedFields().get(lookupFieldName) instanceof String){ + object = JSONObject.parseObject((String) event.getExtractedFields().get(lookupFieldName), Object.class); + } + else { + object = event.getExtractedFields().get(lookupFieldName); + } + try { + if(compiledExp!=null){ + Object obj = compiledExp.execute(compiledExp.newEnv("event", object)); + if(obj instanceof List) { + return parseList(obj,event); + } + else if(obj instanceof String){ + object = JSONObject.parseObject((String)obj, Object.class); + if(object instanceof List){ + return parseList(object,event); + } + else { + log.error("Invalid unroll ! Object is not instance of list. expression=" +expression); + } + }else { + log.error("Invalid unroll ! Object is not instance of String or List. expression=" +expression); + } + } + else { + if(object instanceof List){ + return parseList(object,event); + } + else { + log.error("Invalid unroll ! Object is not instance of list. "); + } + } + }catch (Exception e) { + log.error("Invalid unroll ! expression=" +expression + " Exception :" + e.getMessage()); + } + } + }catch (Exception e) { + log.error("Invalid parseObject ! expression=" +expression + " Exception :" + e.getMessage()); + } + return Collections.singletonList(event); + } + + private List parseList(Object object,Event event) { + List list = (List) object; + List eventList = new ArrayList<>(); + for (Object obj : list) { + Event newEvent = new Event(); + newEvent.setExtractedFields(new HashMap<>()); + newEvent.getExtractedFields().putAll(event.getExtractedFields()); + newEvent.getExtractedFields().remove(lookupFieldName); + if("string".equals(outputFieldType)) { + String jsonString = JSON.toJSONString(obj); + newEvent.getExtractedFields().put(outputFieldName, jsonString); + } + else { + newEvent.getExtractedFields().put(outputFieldName, obj); + } + eventList.add(newEvent); + } + return eventList; + } + + @Override + public String functionName() { + return "UNROLL"; + } + + @Override + public void close() { + + } + +} diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnRollFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnRollFunctionTest.java deleted file mode 100644 index 5686a42..0000000 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnRollFunctionTest.java +++ /dev/null @@ -1,132 +0,0 @@ -package com.geedgenetworks.core.udf.test.table; - -import com.geedgenetworks.common.Event; -import com.geedgenetworks.common.udf.UDFContext; -import com.geedgenetworks.core.udf.udtf.UnRoll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.junit.jupiter.api.Assertions.assertEquals; - -public class UnRollFunctionTest { - - private static Map nestedMap; - @BeforeAll - public static void setUp() { - nestedMap = Map.of( - "k1", List.of( - Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "52:d4:18:c7:e5:11"), - Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") - ), - "k2", Map.of("name", "[{\"tunnels_schema_type\":\"MULTIPATH_ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"c2s_destination_mac\":\"10:70:fd:03:c2:6d\",\"s2c_source_mac\":\"10:70:fd:03:c2:6d\",\"s2c_destination_mac\":\"10:70:fd:03:c2:6c\"},{\"tunnels_schema_type\":\"MULTIPATH_ETHERNET\",\"source_mac\":\"ff:ff:ff:ff:ff:ff\",\"c2s_destination_mac\":\"10:70:fd:03:c2:6d\",\"s2c_source_mac\":\"10:70:fd:03:c2:6d\",\"s2c_destination_mac\":\"10:70:fd:03:c2:6c\"}]", "lastName", "{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}"), - "k3","[{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"},{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"ff:ff:ff:ff:ff:ff\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}]", - "k4","{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}", - "k5", Map.of("name",List.of( - Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "52:d4:18:c7:e5:11"), - Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") - ) - ), "k6", Map.of("name",List.of( - Map.of("name2", List.of( - Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "52:d4:18:c7:e5:11"), - Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") - ), "source_mac", "52:d4:18:c7:e5:11"), - Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") - ) - ) - ); - } - - // 测试方法 - @Test - public void testUnrollFunction1() { - - - UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("k1")); - udfContext.setOutput_fields(List.of("newk1")); - Map params = new HashMap<>(); - params.put("path", ""); - udfContext.setParameters(params); - UnRoll unroll = new UnRoll(); - unroll.open(null, udfContext); - Event event = new Event(); - event.setExtractedFields(nestedMap); - List result = unroll.evaluate(event); - assertEquals(2, result.size()); - Map map1 = (Map) result.get(0).getExtractedFields().get("newk1"); - assertEquals("52:d4:18:c7:e5:11", map1.get("source_mac")); - Map map2 = (Map) result.get(1).getExtractedFields().get("newk1"); - assertEquals("ff:ff:ff:ff:ff:ff", map2.get("source_mac")); - - - - } - - @Test - public void testUnrollFunction2() { - UDFContext udfContext = new UDFContext(); - UnRoll unroll = new UnRoll(); - Event event = new Event(); - event.setExtractedFields(nestedMap); - udfContext.setLookup_fields(List.of("k3")); - udfContext.setOutput_fields(List.of("newk3")); - unroll.open(null, udfContext); - List result3 = unroll.evaluate(event); - assertEquals(2, result3.size()); - Map map3 = (Map) result3.get(0).getExtractedFields().get("newk3"); - assertEquals("52:d4:18:c7:e5:11", map3.get("source_mac")); - Map map4 = (Map) result3.get(1).getExtractedFields().get("newk3"); - assertEquals("ff:ff:ff:ff:ff:ff", map4.get("source_mac")); - - - Map params = new HashMap<>(); - params.put("path", "name"); - udfContext.setParameters(params); - event.setExtractedFields(nestedMap); - udfContext.setLookup_fields(List.of("k2")); - udfContext.setOutput_fields(List.of("newk2")); - unroll.open(null, udfContext); - List result2 = unroll.evaluate(event); - assertEquals(2, result2.size()); - Map map5 = (Map) result2.get(0).getExtractedFields().get("newk2"); - assertEquals("52:d4:18:c7:e5:11", map5.get("source_mac")); - Map map6 = (Map) result2.get(1).getExtractedFields().get("newk2"); - assertEquals("ff:ff:ff:ff:ff:ff", map6.get("source_mac")); - - - Map params1 = new HashMap<>(); - params.put("path", "name.0.name2"); - udfContext.setParameters(params1); - event.setExtractedFields(nestedMap); - udfContext.setLookup_fields(List.of("k6")); - udfContext.setOutput_fields(List.of("newk6")); - unroll.open(null, udfContext); - List result6 = unroll.evaluate(event); - assertEquals(2, result6.size()); - Map map9 = (Map) result6.get(0).getExtractedFields().get("newk6"); - assertEquals("52:d4:18:c7:e5:11", map9.get("source_mac")); - Map map10 = (Map) result6.get(1).getExtractedFields().get("newk6"); - assertEquals("ff:ff:ff:ff:ff:ff", map10.get("source_mac")); - } - @Test - public void testUnrollFunction4() { - - - UDFContext udfContext = new UDFContext(); - UnRoll unroll = new UnRoll(); - Event event = new Event(); - Map params = new HashMap<>(); - udfContext.setParameters(params); - event.setExtractedFields(nestedMap); - udfContext.setLookup_fields(List.of("k4")); - udfContext.setOutput_fields(List.of("newk4")); - unroll.open(null, udfContext); - List result2 = unroll.evaluate(event); - assertEquals(1, result2.size()); - - } -} diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java new file mode 100644 index 0000000..68b3683 --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java @@ -0,0 +1,132 @@ +package com.geedgenetworks.core.udf.test.table; + +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.core.udf.udtf.Unroll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class UnrollFunctionTest { + + private static Map nestedMap; + @BeforeAll + public static void setUp() { + nestedMap = Map.of( + "k1", List.of( + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "52:d4:18:c7:e5:11"), + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") + ), + "k2", Map.of("name", "[{\"tunnels_schema_type\":\"MULTIPATH_ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"c2s_destination_mac\":\"10:70:fd:03:c2:6d\",\"s2c_source_mac\":\"10:70:fd:03:c2:6d\",\"s2c_destination_mac\":\"10:70:fd:03:c2:6c\"},{\"tunnels_schema_type\":\"MULTIPATH_ETHERNET\",\"source_mac\":\"ff:ff:ff:ff:ff:ff\",\"c2s_destination_mac\":\"10:70:fd:03:c2:6d\",\"s2c_source_mac\":\"10:70:fd:03:c2:6d\",\"s2c_destination_mac\":\"10:70:fd:03:c2:6c\"}]", "lastName", "{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}"), + "k3","[{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"},{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"ff:ff:ff:ff:ff:ff\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}]", + "k4","{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}", + "k5", Map.of("name",List.of( + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "52:d4:18:c7:e5:11"), + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") + ) + ), "k6", Map.of("name",List.of( + Map.of("name2", List.of( + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "52:d4:18:c7:e5:11"), + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") + ), "source_mac", "52:d4:18:c7:e5:11"), + Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") + ) + ) + ); + } + + // 测试方法 + @Test + public void testUnrollFunction1() { + + + UDFContext udfContext = new UDFContext(); + udfContext.setLookup_fields(List.of("k1")); + udfContext.setOutput_fields(List.of("newk1")); + Map params = new HashMap<>(); + params.put("path", ""); + udfContext.setParameters(params); + Unroll unroll = new Unroll(); + unroll.open(null, udfContext); + Event event = new Event(); + event.setExtractedFields(nestedMap); + List result = unroll.evaluate(event); + assertEquals(2, result.size()); + Map map1 = (Map) result.get(0).getExtractedFields().get("newk1"); + assertEquals("52:d4:18:c7:e5:11", map1.get("source_mac")); + Map map2 = (Map) result.get(1).getExtractedFields().get("newk1"); + assertEquals("ff:ff:ff:ff:ff:ff", map2.get("source_mac")); + + + + } + + @Test + public void testUnrollFunction2() { + UDFContext udfContext = new UDFContext(); + Unroll unroll = new Unroll(); + Event event = new Event(); + event.setExtractedFields(nestedMap); + udfContext.setLookup_fields(List.of("k3")); + udfContext.setOutput_fields(List.of("newk3")); + unroll.open(null, udfContext); + List result3 = unroll.evaluate(event); + assertEquals(2, result3.size()); + Map map3 = (Map) result3.get(0).getExtractedFields().get("newk3"); + assertEquals("52:d4:18:c7:e5:11", map3.get("source_mac")); + Map map4 = (Map) result3.get(1).getExtractedFields().get("newk3"); + assertEquals("ff:ff:ff:ff:ff:ff", map4.get("source_mac")); + + + Map params = new HashMap<>(); + params.put("path", "name"); + udfContext.setParameters(params); + event.setExtractedFields(nestedMap); + udfContext.setLookup_fields(List.of("k2")); + udfContext.setOutput_fields(List.of("newk2")); + unroll.open(null, udfContext); + List result2 = unroll.evaluate(event); + assertEquals(2, result2.size()); + Map map5 = (Map) result2.get(0).getExtractedFields().get("newk2"); + assertEquals("52:d4:18:c7:e5:11", map5.get("source_mac")); + Map map6 = (Map) result2.get(1).getExtractedFields().get("newk2"); + assertEquals("ff:ff:ff:ff:ff:ff", map6.get("source_mac")); + + + Map params1 = new HashMap<>(); + params.put("path", "name.0.name2"); + udfContext.setParameters(params1); + event.setExtractedFields(nestedMap); + udfContext.setLookup_fields(List.of("k6")); + udfContext.setOutput_fields(List.of("newk6")); + unroll.open(null, udfContext); + List result6 = unroll.evaluate(event); + assertEquals(2, result6.size()); + Map map9 = (Map) result6.get(0).getExtractedFields().get("newk6"); + assertEquals("52:d4:18:c7:e5:11", map9.get("source_mac")); + Map map10 = (Map) result6.get(1).getExtractedFields().get("newk6"); + assertEquals("ff:ff:ff:ff:ff:ff", map10.get("source_mac")); + } + @Test + public void testUnrollFunction4() { + + + UDFContext udfContext = new UDFContext(); + Unroll unroll = new Unroll(); + Event event = new Event(); + Map params = new HashMap<>(); + udfContext.setParameters(params); + event.setExtractedFields(nestedMap); + udfContext.setLookup_fields(List.of("k4")); + udfContext.setOutput_fields(List.of("newk4")); + unroll.open(null, udfContext); + List result2 = unroll.evaluate(event); + assertEquals(1, result2.size()); + + } +} -- cgit v1.2.3 From f11ba2bab662a629642cc230a0355cfca069de75 Mon Sep 17 00:00:00 2001 From: wangkuan Date: Thu, 15 Aug 2024 11:50:18 +0800 Subject: [feature][core]新增JsonUnroll函数,优化Unroll函数 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- config/udf.plugins | 1 + .../geedgenetworks/common/utils/JsonPathUtil.java | 33 ++++++ groot-common/src/main/resources/udf.plugins | 1 + .../geedgenetworks/core/udf/udtf/JsonUnroll.java | 125 +++++++++++++++++++++ .../com/geedgenetworks/core/udf/udtf/Unroll.java | 107 +++++++----------- .../udf/test/table/JsonUnrollFunctionTest.java | 105 +++++++++++++++++ .../core/udf/test/table/UnrollFunctionTest.java | 72 +++--------- 7 files changed, 319 insertions(+), 125 deletions(-) create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java create mode 100644 groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java diff --git a/config/udf.plugins b/config/udf.plugins index 0d6c0fe..31d1b21 100644 --- a/config/udf.plugins +++ b/config/udf.plugins @@ -22,4 +22,5 @@ com.geedgenetworks.core.udf.udaf.LongCount com.geedgenetworks.core.udf.udaf.Mean com.geedgenetworks.core.udf.udaf.LastValue com.geedgenetworks.core.udf.udaf.FirstValue +com.geedgenetworks.core.udf.udtf.JsonUnroll com.geedgenetworks.core.udf.udtf.Unroll \ No newline at end of file diff --git a/groot-common/src/main/java/com/geedgenetworks/common/utils/JsonPathUtil.java b/groot-common/src/main/java/com/geedgenetworks/common/utils/JsonPathUtil.java index dcba58c..0823ddc 100644 --- a/groot-common/src/main/java/com/geedgenetworks/common/utils/JsonPathUtil.java +++ b/groot-common/src/main/java/com/geedgenetworks/common/utils/JsonPathUtil.java @@ -2,6 +2,8 @@ package com.geedgenetworks.common.utils; import cn.hutool.log.Log; import cn.hutool.log.LogFactory; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; import com.alibaba.fastjson2.JSONPath; import com.alibaba.fastjson2.JSONReader; @@ -42,4 +44,35 @@ public class JsonPathUtil { } return flattenResult; } + public static Object get(JSONObject jsonObject, String expr) { + Object Result = ""; + try { + Result = jsonObject.getByPath(expr); + } catch (Exception e) { + logger.error( + "The label resolution exception or [expr] analytic expression error" + + e.getMessage()); + } + return Result; + } + public static JSONObject set(JSONObject jsonObject, String expr,Object value) { + try { + JSONPath.set(jsonObject, expr, value); + } catch (Exception e) { + logger.error( + "JSONObject set value exception or [expr] expression error" + + e.getMessage()); + } + return jsonObject; + } + public static JSONObject remove(JSONObject jsonObject, String expr) { + try { + JSONPath.remove(jsonObject, expr); + } catch (Exception e) { + logger.error( + "JSONObject remove value exception or [expr] expression error" + + e.getMessage()); + } + return jsonObject; + } } diff --git a/groot-common/src/main/resources/udf.plugins b/groot-common/src/main/resources/udf.plugins index ca0a7ac..18446c9 100644 --- a/groot-common/src/main/resources/udf.plugins +++ b/groot-common/src/main/resources/udf.plugins @@ -21,4 +21,5 @@ com.geedgenetworks.core.udf.udaf.LongCount com.geedgenetworks.core.udf.udaf.Mean com.geedgenetworks.core.udf.udaf.LastValue com.geedgenetworks.core.udf.udaf.FirstValue +com.geedgenetworks.core.udf.udtf.JsonUnroll com.geedgenetworks.core.udf.udtf.Unroll \ No newline at end of file diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java new file mode 100644 index 0000000..2e8eb7e --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java @@ -0,0 +1,125 @@ +package com.geedgenetworks.core.udf.udtf; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.TableFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.common.utils.JsonPathUtil; +import com.googlecode.aviator.AviatorEvaluator; +import com.googlecode.aviator.AviatorEvaluatorInstance; +import com.googlecode.aviator.Expression; +import com.googlecode.aviator.Options; +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.api.common.functions.RuntimeContext; + +import java.lang.reflect.Array; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; + + +@Slf4j +public class JsonUnroll implements TableFunction { + + private String lookupFieldName; + private String outputFieldName; + private String path; + private String new_Path; + + + @Override + public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + if(udfContext.getLookup_fields()==null ){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } + this.lookupFieldName = udfContext.getLookup_fields().get(0); + if(udfContext.getOutput_fields()!=null && !udfContext.getOutput_fields().isEmpty()) { + this.outputFieldName = udfContext.getOutput_fields().get(0); + } + else { + outputFieldName = lookupFieldName; + } + if(udfContext.getParameters()==null ){ + path=""; + new_Path=""; + } + else { + path=udfContext.getParameters().getOrDefault("path", "").toString().trim(); + new_Path=udfContext.getParameters().getOrDefault("new_path", path).toString().trim(); + } + } + + @Override + public List evaluate(Event event) { + try { + if(event.getExtractedFields().containsKey(lookupFieldName) ){ + try { + if(path.isEmpty()){ + JSONArray jsonArray = JSONArray.parseArray(event.getExtractedFields().get(lookupFieldName).toString()); + return parseList(jsonArray,event); + }else { + JSONObject jsonObject = JSONObject.parseObject(event.getExtractedFields().get(lookupFieldName).toString()); + Object obj = JsonPathUtil.get(jsonObject,path); + if(obj instanceof List || obj instanceof Array) { + List list = (List) obj; + List eventList = new ArrayList<>(); + for (Object o : list) { + JSONObject newJsonObject = new JSONObject(); + newJsonObject.putAll(jsonObject); + JsonPathUtil.remove(newJsonObject,path); + JsonPathUtil.set(newJsonObject,new_Path,o); + String jsonString = JSON.toJSONString(newJsonObject); + Event newEvent = new Event(); + newEvent.setExtractedFields(new HashMap<>()); + newEvent.getExtractedFields().putAll(event.getExtractedFields()); + newEvent.getExtractedFields().remove(lookupFieldName); + newEvent.getExtractedFields().put(outputFieldName, jsonString); + eventList.add(newEvent); + } + return eventList; + } + else { + log.error("Invalid unroll ! expression=" +path + " Exception :" + " expression should return a list or array"); + } + } + + }catch (Exception e) { + log.error("Invalid unroll ! expression=" +path + " Exception :" + e.getMessage()); + } + } + }catch (Exception e) { + log.error("Invalid parseObject ! expression=" +path + " Exception :" + e.getMessage()); + } + return Collections.singletonList(event); + } + + private List parseList(Object object,Event event) { + List list = (List) object; + List eventList = new ArrayList<>(); + for (Object obj : list) { + Event newEvent = new Event(); + newEvent.setExtractedFields(new HashMap<>()); + newEvent.getExtractedFields().putAll(event.getExtractedFields()); + newEvent.getExtractedFields().remove(lookupFieldName); + newEvent.getExtractedFields().put(outputFieldName, JSON.toJSONString(obj)); + eventList.add(newEvent); + } + return eventList; + } + + @Override + public String functionName() { + return "JSON_UNROLL"; + } + + @Override + public void close() { + + } + +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java index 3931743..5becb8e 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java @@ -1,16 +1,10 @@ package com.geedgenetworks.core.udf.udtf; -import com.alibaba.fastjson2.JSON; -import com.alibaba.fastjson2.JSONObject; import com.geedgenetworks.common.Event; import com.geedgenetworks.common.exception.CommonErrorCode; import com.geedgenetworks.common.exception.GrootStreamRuntimeException; import com.geedgenetworks.common.udf.TableFunction; import com.geedgenetworks.common.udf.UDFContext; -import com.googlecode.aviator.AviatorEvaluator; -import com.googlecode.aviator.AviatorEvaluatorInstance; -import com.googlecode.aviator.Expression; -import com.googlecode.aviator.Options; import lombok.extern.slf4j.Slf4j; import org.apache.flink.api.common.functions.RuntimeContext; @@ -22,79 +16,54 @@ public class Unroll implements TableFunction { private String lookupFieldName; private String outputFieldName; - private Expression compiledExp; - private String expression; - private String outputFieldType; + private String regex; @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null){ + if(udfContext.getLookup_fields()==null ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - this.lookupFieldName = udfContext.getLookup_fields().get(0); - this.outputFieldName = udfContext.getOutput_fields().get(0); - this.outputFieldType="object"; - if(udfContext.getParameters()==null ){ - expression=""; + this.lookupFieldName = udfContext.getLookup_fields().get(0); + if(udfContext.getOutput_fields()!=null && !udfContext.getOutput_fields().isEmpty()) { + this.outputFieldName = udfContext.getOutput_fields().get(0); } else { - this.outputFieldType=udfContext.getParameters().getOrDefault("output_field_type", "object").toString().trim(); - expression=udfContext.getParameters().getOrDefault("path", "").toString().trim(); + outputFieldName = lookupFieldName; } - if(!expression.isEmpty()){ - AviatorEvaluatorInstance instance = AviatorEvaluator.getInstance(); - instance.setCachedExpressionByDefault(true); - instance.setOption(Options.OPTIMIZE_LEVEL, AviatorEvaluator.EVAL); - instance.setFunctionMissing(null); - compiledExp = instance.compile("event."+expression, true); + if(udfContext.getParameters()==null ){ + regex=""; + } + else { + this.regex=udfContext.getParameters().getOrDefault("regex", "").toString().trim(); } - } @Override public List evaluate(Event event) { try { - if(event.getExtractedFields().containsKey(lookupFieldName)){ - Object object; - if(event.getExtractedFields().get(lookupFieldName) instanceof String){ - object = JSONObject.parseObject((String) event.getExtractedFields().get(lookupFieldName), Object.class); + if(event.getExtractedFields().containsKey(lookupFieldName)) { + + if(regex.isEmpty()){ + if (event.getExtractedFields().get(lookupFieldName) instanceof List ) { + return parseList(event.getExtractedFields().get(lookupFieldName), event); + } else if(event.getExtractedFields().get(lookupFieldName) instanceof Object[]){ + return parseArray(event.getExtractedFields().get(lookupFieldName), event); + }else { + log.error("Invalid unroll ! Object is not instance of list or array. expression=" + regex); + } } else { - object = event.getExtractedFields().get(lookupFieldName); - } - try { - if(compiledExp!=null){ - Object obj = compiledExp.execute(compiledExp.newEnv("event", object)); - if(obj instanceof List) { - return parseList(obj,event); - } - else if(obj instanceof String){ - object = JSONObject.parseObject((String)obj, Object.class); - if(object instanceof List){ - return parseList(object,event); - } - else { - log.error("Invalid unroll ! Object is not instance of list. expression=" +expression); - } - }else { - log.error("Invalid unroll ! Object is not instance of String or List. expression=" +expression); - } - } - else { - if(object instanceof List){ - return parseList(object,event); - } - else { - log.error("Invalid unroll ! Object is not instance of list. "); - } + if (event.getExtractedFields().get(lookupFieldName) instanceof String) { + String[] array =((String) event.getExtractedFields().get(lookupFieldName)).split(regex); + return parseArray(array, event); + }else { + log.error("Invalid unroll ! Object is not instance of String. expression=" + regex); } - }catch (Exception e) { - log.error("Invalid unroll ! expression=" +expression + " Exception :" + e.getMessage()); } } }catch (Exception e) { - log.error("Invalid parseObject ! expression=" +expression + " Exception :" + e.getMessage()); + log.error("Invalid parseObject ! expression=" +regex + " Exception :" + e.getMessage()); } return Collections.singletonList(event); } @@ -107,18 +76,24 @@ public class Unroll implements TableFunction { newEvent.setExtractedFields(new HashMap<>()); newEvent.getExtractedFields().putAll(event.getExtractedFields()); newEvent.getExtractedFields().remove(lookupFieldName); - if("string".equals(outputFieldType)) { - String jsonString = JSON.toJSONString(obj); - newEvent.getExtractedFields().put(outputFieldName, jsonString); - } - else { - newEvent.getExtractedFields().put(outputFieldName, obj); - } + newEvent.getExtractedFields().put(outputFieldName, obj); + eventList.add(newEvent); + } + return eventList; + } + private List parseArray(Object object, Event event) { + List eventList = new ArrayList<>(); + Object[] objects = (Object[]) object; + for (Object obj : objects) { + Event newEvent = new Event(); + newEvent.setExtractedFields(new HashMap<>()); + newEvent.getExtractedFields().putAll(event.getExtractedFields()); + newEvent.getExtractedFields().remove(lookupFieldName); + newEvent.getExtractedFields().put(outputFieldName, obj); eventList.add(newEvent); } return eventList; } - @Override public String functionName() { return "UNROLL"; diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java new file mode 100644 index 0000000..02f0b66 --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java @@ -0,0 +1,105 @@ +package com.geedgenetworks.core.udf.test.table; + +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.core.udf.udtf.JsonUnroll; +import com.geedgenetworks.core.udf.udtf.Unroll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class JsonUnrollFunctionTest { + + private static Map nestedMap; + @BeforeAll + public static void setUp() { + nestedMap = Map.of( + "k1","[{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"},{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"ff:ff:ff:ff:ff:ff\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}]", + "k2","{\"k2_1\":\"[{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"},{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"ff:ff:ff:ff:ff:ff\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}]\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}", + "k3","{\n" + + " \"k3_1\": {\n" + + " \"k3_1_1\": [\n" + + " {\n" + + " \"tunnels_schema_type\": \"ETHERNET\",\n" + + " \"source_mac\": \"52:d4:18:c7:e5:11\"\n" + + " },\n" + + " {\n" + + " \"tunnels_schema_type\": \"ETHERNET\",\n" + + " \"source_mac\": \"ff:ff:ff:ff:ff:ff\"\n" + + " }\n" + + " ],\n" + + " \"k3_1_2\": {\n" + + " \"tunnels_schema_type\": \"ETHERNET\",\n" + + " \"source_mac\": 19.95\n" + + " }\n" + + " }\n" + + "}", + "k4","" + + ); + } + + // 测试方法 + + @Test + public void testJsonUnrollFunction1() { + UDFContext udfContext = new UDFContext(); + JsonUnroll unroll = new JsonUnroll(); + Event event = new Event(); + event.setExtractedFields(nestedMap); + udfContext.setLookup_fields(List.of("k1")); + udfContext.setOutput_fields(List.of("newk1")); + unroll.open(null, udfContext); + List result3 = unroll.evaluate(event); + assertEquals(2, result3.size()); + assertEquals("{\"destination_mac\":\"ff:ff:ff:ff:ff:ff\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"tunnels_schema_type\":\"ETHERNET\"}", result3.get(0).getExtractedFields().get("newk1").toString()); + assertEquals("{\"destination_mac\":\"ff:ff:ff:ff:ff:ff\",\"source_mac\":\"ff:ff:ff:ff:ff:ff\",\"tunnels_schema_type\":\"ETHERNET\"}", result3.get(1).getExtractedFields().get("newk1").toString()); + + + + + } + @Test + public void testJsonUnrollFunction2() { + + + UDFContext udfContext = new UDFContext(); + JsonUnroll unroll = new JsonUnroll(); + Event event = new Event(); + Map params = new HashMap<>(); + udfContext.setParameters(params); + params.put("path", "$.k3_1.k3_1_1"); + event.setExtractedFields(nestedMap); + udfContext.setLookup_fields(List.of("k3")); + udfContext.setOutput_fields(List.of("newk3")); + unroll.open(null, udfContext); + List result2 = unroll.evaluate(event); + assertEquals(2, result2.size()); + assertEquals("{\"k3_1\":{\"k3_1_2\":{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":19.95},\"k3_1_1\":{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\"}}}",result2.get(0).getExtractedFields().get("newk3").toString()); + assertEquals("{\"k3_1\":{\"k3_1_2\":{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":19.95},\"k3_1_1\":{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"ff:ff:ff:ff:ff:ff\"}}}",result2.get(1).getExtractedFields().get("newk3").toString()); + + } + @Test + public void testJsonUnrollFunction3() { + + + UDFContext udfContext = new UDFContext(); + JsonUnroll unroll = new JsonUnroll(); + Event event = new Event(); + Map params = new HashMap<>(); + udfContext.setParameters(params); + params.put("path", "$.k4_1.k4_1_1"); + event.setExtractedFields(nestedMap); + udfContext.setLookup_fields(List.of("k4")); + udfContext.setOutput_fields(List.of("newk4")); + unroll.open(null, udfContext); + List result2 = unroll.evaluate(event); + assertEquals(1, result2.size()); + + } +} diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java index 68b3683..2f4da76 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java @@ -22,21 +22,9 @@ public class UnrollFunctionTest { Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "52:d4:18:c7:e5:11"), Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") ), - "k2", Map.of("name", "[{\"tunnels_schema_type\":\"MULTIPATH_ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"c2s_destination_mac\":\"10:70:fd:03:c2:6d\",\"s2c_source_mac\":\"10:70:fd:03:c2:6d\",\"s2c_destination_mac\":\"10:70:fd:03:c2:6c\"},{\"tunnels_schema_type\":\"MULTIPATH_ETHERNET\",\"source_mac\":\"ff:ff:ff:ff:ff:ff\",\"c2s_destination_mac\":\"10:70:fd:03:c2:6d\",\"s2c_source_mac\":\"10:70:fd:03:c2:6d\",\"s2c_destination_mac\":\"10:70:fd:03:c2:6c\"}]", "lastName", "{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}"), - "k3","[{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"},{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"ff:ff:ff:ff:ff:ff\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}]", - "k4","{\"tunnels_schema_type\":\"ETHERNET\",\"source_mac\":\"52:d4:18:c7:e5:11\",\"destination_mac\":\"ff:ff:ff:ff:ff:ff\"}", - "k5", Map.of("name",List.of( - Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "52:d4:18:c7:e5:11"), - Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") - ) - ), "k6", Map.of("name",List.of( - Map.of("name2", List.of( - Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "52:d4:18:c7:e5:11"), - Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") - ), "source_mac", "52:d4:18:c7:e5:11"), - Map.of("tunnels_schema_type", "ETHERNET", "source_mac", "ff:ff:ff:ff:ff:ff") - ) - ) + "k2","{\"source_mac\":\"52:d4:18:c7:e5:10\"},{\"source_mac\":\"ff:ff:ff:ff:ff:ff\"},{\"source_mac\":\"52:d4:18:c7:e5:11\"}", + "k3","" + ); } @@ -48,9 +36,6 @@ public class UnrollFunctionTest { UDFContext udfContext = new UDFContext(); udfContext.setLookup_fields(List.of("k1")); udfContext.setOutput_fields(List.of("newk1")); - Map params = new HashMap<>(); - params.put("path", ""); - udfContext.setParameters(params); Unroll unroll = new Unroll(); unroll.open(null, udfContext); Event event = new Event(); @@ -62,68 +47,37 @@ public class UnrollFunctionTest { Map map2 = (Map) result.get(1).getExtractedFields().get("newk1"); assertEquals("ff:ff:ff:ff:ff:ff", map2.get("source_mac")); - - } - @Test public void testUnrollFunction2() { + + UDFContext udfContext = new UDFContext(); Unroll unroll = new Unroll(); Event event = new Event(); - event.setExtractedFields(nestedMap); - udfContext.setLookup_fields(List.of("k3")); - udfContext.setOutput_fields(List.of("newk3")); - unroll.open(null, udfContext); - List result3 = unroll.evaluate(event); - assertEquals(2, result3.size()); - Map map3 = (Map) result3.get(0).getExtractedFields().get("newk3"); - assertEquals("52:d4:18:c7:e5:11", map3.get("source_mac")); - Map map4 = (Map) result3.get(1).getExtractedFields().get("newk3"); - assertEquals("ff:ff:ff:ff:ff:ff", map4.get("source_mac")); - - Map params = new HashMap<>(); - params.put("path", "name"); + params.put("regex", ","); + udfContext.setParameters(params); udfContext.setParameters(params); event.setExtractedFields(nestedMap); udfContext.setLookup_fields(List.of("k2")); - udfContext.setOutput_fields(List.of("newk2")); + udfContext.setOutput_fields(List.of("k2")); unroll.open(null, udfContext); List result2 = unroll.evaluate(event); - assertEquals(2, result2.size()); - Map map5 = (Map) result2.get(0).getExtractedFields().get("newk2"); - assertEquals("52:d4:18:c7:e5:11", map5.get("source_mac")); - Map map6 = (Map) result2.get(1).getExtractedFields().get("newk2"); - assertEquals("ff:ff:ff:ff:ff:ff", map6.get("source_mac")); - + assertEquals(3, result2.size()); + assertEquals("{\"source_mac\":\"52:d4:18:c7:e5:10\"}", result2.get(0).getExtractedFields().get("k2")); - Map params1 = new HashMap<>(); - params.put("path", "name.0.name2"); - udfContext.setParameters(params1); - event.setExtractedFields(nestedMap); - udfContext.setLookup_fields(List.of("k6")); - udfContext.setOutput_fields(List.of("newk6")); - unroll.open(null, udfContext); - List result6 = unroll.evaluate(event); - assertEquals(2, result6.size()); - Map map9 = (Map) result6.get(0).getExtractedFields().get("newk6"); - assertEquals("52:d4:18:c7:e5:11", map9.get("source_mac")); - Map map10 = (Map) result6.get(1).getExtractedFields().get("newk6"); - assertEquals("ff:ff:ff:ff:ff:ff", map10.get("source_mac")); } @Test - public void testUnrollFunction4() { + public void testUnrollFunction3() { UDFContext udfContext = new UDFContext(); Unroll unroll = new Unroll(); Event event = new Event(); - Map params = new HashMap<>(); - udfContext.setParameters(params); event.setExtractedFields(nestedMap); - udfContext.setLookup_fields(List.of("k4")); - udfContext.setOutput_fields(List.of("newk4")); + udfContext.setLookup_fields(List.of("k3")); + udfContext.setOutput_fields(List.of("newk3")); unroll.open(null, udfContext); List result2 = unroll.evaluate(event); assertEquals(1, result2.size()); -- cgit v1.2.3 From be33d6527324e041e8f61ef3c7bf98158c97cde7 Mon Sep 17 00:00:00 2001 From: lifengchao Date: Thu, 15 Aug 2024 15:28:24 +0800 Subject: [feature][core]添加sketche聚合函数和sketche mock类型 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- config/udf.plugins | 7 +- groot-common/src/main/resources/udf.plugins | 7 +- groot-connectors/connector-kafka/pom.xml | 11 +++ .../connectors/mock/faker/FakerUtils.java | 18 ++++ .../connectors/mock/faker/HdrHistogramFaker.java | 35 +++++++ .../connectors/mock/faker/HlldFaker.java | 39 ++++++++ groot-core/pom.xml | 5 + .../core/udf/udaf/HdrHistogram/HdrHistogram.java | 42 +++++++++ .../HdrHistogram/HdrHistogramBaseAggregate.java | 101 ++++++++++++++++++++ .../udaf/HdrHistogram/HdrHistogramQuantile.java | 36 ++++++++ .../udaf/HdrHistogram/HdrHistogramQuantiles.java | 51 +++++++++++ .../geedgenetworks/core/udf/udaf/hlld/Hlld.java | 40 ++++++++ .../udf/udaf/hlld/HlldApproxCountDistinct.java | 25 +++++ .../core/udf/udaf/hlld/HlldBaseAggregate.java | 98 ++++++++++++++++++++ .../HdrHistogram/HdrHistogramQuantileTest.java | 89 ++++++++++++++++++ .../HdrHistogram/HdrHistogramQuantilesTest.java | 98 ++++++++++++++++++++ .../udf/udaf/HdrHistogram/HdrHistogramTest.java | 102 +++++++++++++++++++++ .../udf/udaf/hlld/HlldApproxCountDistinctTest.java | 87 ++++++++++++++++++ .../core/udf/udaf/hlld/HlldTest.java | 86 +++++++++++++++++ pom.xml | 5 + 20 files changed, 980 insertions(+), 2 deletions(-) create mode 100644 groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/HdrHistogramFaker.java create mode 100644 groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/HlldFaker.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogram.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantile.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantiles.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/Hlld.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinct.java create mode 100644 groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java create mode 100644 groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java create mode 100644 groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java create mode 100644 groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java create mode 100644 groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java create mode 100644 groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java diff --git a/config/udf.plugins b/config/udf.plugins index 31d1b21..772d2bc 100644 --- a/config/udf.plugins +++ b/config/udf.plugins @@ -23,4 +23,9 @@ com.geedgenetworks.core.udf.udaf.Mean com.geedgenetworks.core.udf.udaf.LastValue com.geedgenetworks.core.udf.udaf.FirstValue com.geedgenetworks.core.udf.udtf.JsonUnroll -com.geedgenetworks.core.udf.udtf.Unroll \ No newline at end of file +com.geedgenetworks.core.udf.udtf.Unroll +com.geedgenetworks.core.udf.udaf.hlld.Hlld +com.geedgenetworks.core.udf.udaf.hlld.HlldApproxCountDistinct +com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogram +com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantile +com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantiles \ No newline at end of file diff --git a/groot-common/src/main/resources/udf.plugins b/groot-common/src/main/resources/udf.plugins index 18446c9..0062c1a 100644 --- a/groot-common/src/main/resources/udf.plugins +++ b/groot-common/src/main/resources/udf.plugins @@ -22,4 +22,9 @@ com.geedgenetworks.core.udf.udaf.Mean com.geedgenetworks.core.udf.udaf.LastValue com.geedgenetworks.core.udf.udaf.FirstValue com.geedgenetworks.core.udf.udtf.JsonUnroll -com.geedgenetworks.core.udf.udtf.Unroll \ No newline at end of file +com.geedgenetworks.core.udf.udtf.Unroll +com.geedgenetworks.core.udf.udaf.hlld.Hlld +com.geedgenetworks.core.udf.udaf.hlld.HlldApproxCountDistinct +com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogram +com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantile +com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantiles \ No newline at end of file diff --git a/groot-connectors/connector-kafka/pom.xml b/groot-connectors/connector-kafka/pom.xml index 7ec7d86..448383b 100644 --- a/groot-connectors/connector-kafka/pom.xml +++ b/groot-connectors/connector-kafka/pom.xml @@ -15,6 +15,17 @@ org.apache.flink flink-connector-kafka_${scala.version} ${flink.version} + + + org.xerial.snappy + snappy-java + + + + + org.xerial.snappy + snappy-java + 1.1.8.3 diff --git a/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/FakerUtils.java b/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/FakerUtils.java index 09cc8f8..5101fa1 100644 --- a/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/FakerUtils.java +++ b/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/FakerUtils.java @@ -46,6 +46,10 @@ public class FakerUtils { return wrapFaker(parseIPv4Faker(obj), obj); } else if ("Expression".equalsIgnoreCase(type)) { return wrapFaker(parseExpressionFaker(obj), obj); + } else if ("Hlld".equalsIgnoreCase(type)) { + return wrapFaker(parseHlldFaker(obj), obj); + } else if ("HdrHistogram".equalsIgnoreCase(type)) { + return wrapFaker(parseHdrHistogramFaker(obj), obj); } else if ("Object".equalsIgnoreCase(type)) { return wrapFaker(parseObjectFaker(obj.getJSONArray("fields")), obj); } else if ("Union".equalsIgnoreCase(type)) { @@ -109,6 +113,20 @@ public class FakerUtils { return new ExpressionFaker(expression); } + private static Faker parseHlldFaker(JSONObject obj) { + long itemCount = obj.getLongValue("itemCount", 1000000L); + int batchCount = obj.getIntValue("batchCount", 10000); + int precision = obj.getIntValue("precision", 12); + return new HlldFaker(itemCount, batchCount, precision); + } + + private static Faker parseHdrHistogramFaker(JSONObject obj) { + int max = obj.getIntValue("max", 100000); + int batchCount = obj.getIntValue("batchCount", 1000); + int numberOfSignificantValueDigits = obj.getIntValue("numberOfSignificantValueDigits", 1); + return new HdrHistogramFaker(max, batchCount, numberOfSignificantValueDigits); + } + private static Faker parseIPv4Faker(JSONObject obj) { String start = obj.getString("start"); String end = obj.getString("end"); diff --git a/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/HdrHistogramFaker.java b/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/HdrHistogramFaker.java new file mode 100644 index 0000000..393bf8e --- /dev/null +++ b/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/HdrHistogramFaker.java @@ -0,0 +1,35 @@ +package com.geedgenetworks.connectors.mock.faker; + +import com.geedgenetworks.sketch.util.StringUtils; +import org.HdrHistogram.HistogramSketch; + +import java.util.concurrent.ThreadLocalRandom; + +public class HdrHistogramFaker extends Faker { + private final int max; + private final int batchCount; + private final int numberOfSignificantValueDigits; + private HistogramSketch his; + + public HdrHistogramFaker(int max, int batchCount, int numberOfSignificantValueDigits) { + this.max = max; + this.batchCount = batchCount; + this.numberOfSignificantValueDigits = numberOfSignificantValueDigits; + } + + @Override + public void init(int instanceCount, int instanceIndex) throws Exception { + his = new HistogramSketch(1L, max, numberOfSignificantValueDigits, false); + } + + @Override + public Object geneValue() throws Exception { + his.reset(); + ThreadLocalRandom random = ThreadLocalRandom.current(); + for (int i = 0; i < batchCount; i++) { + his.recordValue(random.nextInt(max)); + } + return StringUtils.encodeBase64String(his.toBytes()); + } + +} diff --git a/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/HlldFaker.java b/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/HlldFaker.java new file mode 100644 index 0000000..5af2b35 --- /dev/null +++ b/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/HlldFaker.java @@ -0,0 +1,39 @@ +package com.geedgenetworks.connectors.mock.faker; + +import com.geedgenetworks.sketch.hlld.Hll; +import com.geedgenetworks.sketch.util.StringUtils; + +import java.util.concurrent.ThreadLocalRandom; + +public class HlldFaker extends Faker { + private final long itemCount; + private final int batchCount; + private final int precision; + private Hll hll; + + public HlldFaker(long itemCount, int batchCount, int precision) { + this.itemCount = itemCount; + this.batchCount = batchCount; + this.precision = precision; + } + + public HlldFaker(long itemCount, int batchCount) { + this(itemCount, batchCount, 12); + } + + @Override + public void init(int instanceCount, int instanceIndex) throws Exception { + hll = new Hll(precision); + } + + @Override + public Object geneValue() throws Exception { + hll.reset(); + ThreadLocalRandom random = ThreadLocalRandom.current(); + for (int i = 0; i < batchCount; i++) { + hll.add(random.nextLong(itemCount)); + } + return StringUtils.encodeBase64String(hll.toBytes()); + } + +} diff --git a/groot-core/pom.xml b/groot-core/pom.xml index 18ae33b..e723fa5 100644 --- a/groot-core/pom.xml +++ b/groot-core/pom.xml @@ -65,6 +65,11 @@ provided + + com.geedgenetworks + sketches + + com.alibaba.nacos nacos-client diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogram.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogram.java new file mode 100644 index 0000000..368e8c1 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogram.java @@ -0,0 +1,42 @@ +package com.geedgenetworks.core.udf.udaf.HdrHistogram; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.util.StringUtils; +import org.HdrHistogram.Histogramer; + +import java.util.Map; + +public class HdrHistogram extends HdrHistogramBaseAggregate { + boolean outputBase64; + + @Override + public void open(UDFContext c) { + super.open(c); + Map params = c.getParameters(); + outputBase64 = "base64".equalsIgnoreCase(params.getOrDefault("output_format", "base64").toString()); + } + + @Override + public Accumulator getResult(Accumulator acc) { + Object agg = acc.getMetricsFields().get(outputField); + if (agg == null) { + return acc; + } + + byte[] bytes = ((Histogramer) agg).toBytes(); + if (outputBase64) { + acc.getMetricsFields().put(outputField, StringUtils.encodeBase64String(bytes)); + } else { + acc.getMetricsFields().put(outputField, bytes); + } + + return acc; + } + + @Override + public String functionName() { + return "HDR_HISTOGRAM"; + } + +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java new file mode 100644 index 0000000..1648fa5 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java @@ -0,0 +1,101 @@ +package com.geedgenetworks.core.udf.udaf.HdrHistogram; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.util.StringUtils; +import org.HdrHistogram.ArrayHistogram; +import org.HdrHistogram.DirectMapHistogram; +import org.HdrHistogram.Histogramer; +import org.apache.commons.collections.CollectionUtils; + +import java.nio.charset.StandardCharsets; +import java.util.Map; + +public abstract class HdrHistogramBaseAggregate implements AggregateFunction { + protected String inputField; + protected String outputField; + protected boolean inputSketch; + protected long lowestDiscernibleValue; + protected long highestTrackableValue; + protected int numberOfSignificantValueDigits; + protected boolean autoResize; + + @Override + public void open(UDFContext c) { + inputField = c.getLookup_fields().get(0); + if (CollectionUtils.isNotEmpty(c.getOutput_fields())) { + outputField = c.getOutput_fields().get(0); + } else { + outputField = inputField; + } + Map params = c.getParameters(); + lowestDiscernibleValue = Long.parseLong(params.getOrDefault("lowestDiscernibleValue", "1").toString()); + highestTrackableValue = Long.parseLong(params.getOrDefault("highestTrackableValue", "2").toString()); + numberOfSignificantValueDigits = Integer.parseInt(params.getOrDefault("numberOfSignificantValueDigits", "1").toString()); + autoResize = Boolean.valueOf(params.getOrDefault("autoResize", "true").toString()); + inputSketch = "sketch".equalsIgnoreCase(params.getOrDefault("input_type", "sketch").toString()); + } + + @Override + public Accumulator initAccumulator(Accumulator acc) { + return acc; + } + + @Override + public Accumulator add(Event event, Accumulator acc) { + Object value = event.getExtractedFields().get(inputField); + if (value == null) { + return acc; + } + + if (inputSketch) { + updateHdrMerge(acc, value); + } else { + updateHdr(acc, value); + } + + return acc; + } + + protected void updateHdr(Accumulator acc, Object value) { + Map aggs = acc.getMetricsFields(); + ArrayHistogram his = (ArrayHistogram) aggs.get(outputField); + if (his == null) { + his = new ArrayHistogram(lowestDiscernibleValue, highestTrackableValue, numberOfSignificantValueDigits); + his.setAutoResize(autoResize); + aggs.put(outputField, his); + } + + his.recordValue(((Number) value).longValue()); + } + + + protected void updateHdrMerge(Accumulator acc, Object value) { + Map aggs = acc.getMetricsFields(); + ArrayHistogram his = (ArrayHistogram) aggs.get(outputField); + if (his == null) { + his = new ArrayHistogram(lowestDiscernibleValue, highestTrackableValue, numberOfSignificantValueDigits); + his.setAutoResize(autoResize); + aggs.put(outputField, his); + } + + Histogramer h; + if (value instanceof String) { + byte[] bytes = StringUtils.decodeBase64(((String) value).getBytes(StandardCharsets.UTF_8)); + h = DirectMapHistogram.wrapBytes(bytes); + } else if (value instanceof byte[]) { + h = DirectMapHistogram.wrapBytes((byte[]) value); + } else if (value instanceof Histogramer) { + h = (Histogramer) value; + } else { + throw new IllegalArgumentException("Unsupported type " + value.getClass()); + } + + his.merge(h); + } + + @Override + public void close() {} +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantile.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantile.java new file mode 100644 index 0000000..b9f7d5b --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantile.java @@ -0,0 +1,36 @@ +package com.geedgenetworks.core.udf.udaf.HdrHistogram; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.udf.UDFContext; +import org.HdrHistogram.Histogramer; + +import java.util.Map; + +public class HdrHistogramQuantile extends HdrHistogramBaseAggregate { + Double probability; + + @Override + public void open(UDFContext c) { + super.open(c); + Map params = c.getParameters(); + probability = Double.parseDouble(params.getOrDefault("probability", "0.5").toString()); + } + + @Override + public Accumulator getResult(Accumulator acc) { + Object agg = acc.getMetricsFields().get(outputField); + if (agg == null) { + return acc; + } + + long percentile = ((Histogramer) agg).getValueAtPercentile(probability * 100); + acc.getMetricsFields().put(outputField, percentile); + return acc; + } + + @Override + public String functionName() { + return "APPROX_QUANTILE_HDR"; + } + +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantiles.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantiles.java new file mode 100644 index 0000000..ccfffd3 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantiles.java @@ -0,0 +1,51 @@ +package com.geedgenetworks.core.udf.udaf.HdrHistogram; + +import com.alibaba.fastjson2.JSON; +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.udf.UDFContext; +import org.HdrHistogram.Histogramer; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class HdrHistogramQuantiles extends HdrHistogramBaseAggregate { + double[] probabilities; + + @Override + public void open(UDFContext c) { + super.open(c); + Map params = c.getParameters(); + Object ps = params.get("probabilities"); + if(ps == null){ + throw new IllegalArgumentException("probabilities param is requested"); + } + List floats = JSON.parseArray(ps instanceof String ? ps.toString(): JSON.toJSONString(ps), Double.class); + probabilities = new double[floats.size()]; + for (int i = 0; i < floats.size(); i++) { + probabilities[i] = floats.get(i); + } + } + + @Override + public Accumulator getResult(Accumulator acc) { + Object agg = acc.getMetricsFields().get(outputField); + if (agg == null) { + return acc; + } + + Histogramer his = ((Histogramer) agg); + final List counts = new ArrayList<>(probabilities.length); + for (int i = 0; i < probabilities.length; i++) { + counts.add(his.getValueAtPercentile(probabilities[i] * 100)); + } + acc.getMetricsFields().put(outputField, counts); + return acc; + } + + @Override + public String functionName() { + return "APPROX_QUANTILES_HDR"; + } + +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/Hlld.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/Hlld.java new file mode 100644 index 0000000..e373a7a --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/Hlld.java @@ -0,0 +1,40 @@ +package com.geedgenetworks.core.udf.udaf.hlld; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.hlld.Hll; +import com.geedgenetworks.sketch.util.StringUtils; + +import java.util.Map; + +public class Hlld extends HlldBaseAggregate { + boolean outputBase64; + + @Override + public void open(UDFContext c) { + super.open(c); + Map params = c.getParameters(); + outputBase64 = "base64".equalsIgnoreCase(params.getOrDefault("output_format", "base64").toString()); + } + + @Override + public Accumulator getResult(Accumulator acc) { + Hll hll = getResultHll(acc); + if (hll == null) { + return acc; + } + + if (outputBase64) { + acc.getMetricsFields().put(outputField, StringUtils.encodeBase64String(hll.toBytes())); + } else { + acc.getMetricsFields().put(outputField, hll.toBytes()); + } + + return acc; + } + + @Override + public String functionName() { + return "HLLD"; + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinct.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinct.java new file mode 100644 index 0000000..ec003f8 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinct.java @@ -0,0 +1,25 @@ +package com.geedgenetworks.core.udf.udaf.hlld; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.sketch.hlld.Hll; + +public class HlldApproxCountDistinct extends HlldBaseAggregate { + + @Override + public Accumulator getResult(Accumulator acc) { + Hll hll = getResultHll(acc); + if (hll == null) { + return acc; + } + + acc.getMetricsFields().put(outputField, (long)hll.size()); + + return acc; + } + + @Override + public String functionName() { + return "APPROX_COUNT_DISTINCT_HLLD"; + } + +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java new file mode 100644 index 0000000..71d61dc --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java @@ -0,0 +1,98 @@ +package com.geedgenetworks.core.udf.udaf.hlld; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.hlld.Hll; +import com.geedgenetworks.sketch.hlld.HllUnion; +import com.geedgenetworks.sketch.hlld.HllUtils; +import org.apache.commons.collections.CollectionUtils; + +import java.util.Map; + +public abstract class HlldBaseAggregate implements AggregateFunction { + protected String inputField; + protected String outputField; + protected boolean inputSketch; + protected int precision = 12; + + @Override + public void open(UDFContext c) { + inputField = c.getLookup_fields().get(0); + if (CollectionUtils.isNotEmpty(c.getOutput_fields())) { + outputField = c.getOutput_fields().get(0); + } else { + outputField = inputField; + } + Map params = c.getParameters(); + precision = Integer.parseInt(params.getOrDefault("precision", "12").toString()); + inputSketch = "sketch".equalsIgnoreCase(params.getOrDefault("input_type", "sketch").toString()); + } + + @Override + public Accumulator initAccumulator(Accumulator acc) { + return acc; + } + + @Override + public Accumulator add(Event event, Accumulator acc) { + Object value = event.getExtractedFields().get(inputField); + if (value == null) { + return acc; + } + + if (inputSketch) { + updateHllUnion(acc, value); + } else { + updateHll(acc, value); + } + + return acc; + } + + protected Hll getResultHll(Accumulator acc){ + Object agg = acc.getMetricsFields().get(outputField); + if (agg == null) { + return null; + } + + return inputSketch ? ((HllUnion) agg).getResult() : (Hll) agg; + } + + protected void updateHll(Accumulator acc, Object value) { + Map aggs = acc.getMetricsFields(); + Hll hll = (Hll) aggs.get(outputField); + if (hll == null) { + hll = new Hll(precision); + aggs.put(outputField, hll); + } + + if (value instanceof Integer || value instanceof Long) { + hll.add(((Number) value).longValue()); + } else if (value instanceof Float || value instanceof Double) { + hll.add(((Number) value).doubleValue()); + } else if (value instanceof String) { + hll.add((String) value); + } else if (value instanceof byte[]) { + hll.add((byte[]) value); + } else { + throw new IllegalArgumentException("Unsupported type " + value.getClass()); + } + } + + protected void updateHllUnion(Accumulator acc, Object value) { + Map aggs = acc.getMetricsFields(); + HllUnion hllUnion = (HllUnion) aggs.get(outputField); + if (hllUnion == null) { + hllUnion = new HllUnion(precision); + aggs.put(outputField, hllUnion); + } + + Hll hll = HllUtils.deserializeHll(value); + hllUnion.update(hll); + } + + @Override + public void close() {} +} diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java new file mode 100644 index 0000000..33f7bad --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java @@ -0,0 +1,89 @@ +package com.geedgenetworks.core.udf.udaf.HdrHistogram; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.util.StringUtils; +import org.HdrHistogram.ArrayHistogram; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.*; + +public class HdrHistogramQuantileTest { + AggregateFunction agg; + Accumulator acc; + Event event; + + @Test + public void inputRegular() { + double probability = 0.5; + initData( "regular", 2, probability); + long count = 100000; + Map fields = event.getExtractedFields(); + for (int i = 1; i <= count; i++) { + fields.put("ms", i); + agg.add(event, acc); + } + + long expect = (long) (count * probability); + long rst = (long)agg.getResult(acc).getMetricsFields().get("ms_his"); + double error = Math.abs(rst - expect) / (double) expect; + System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error)); + assertTrue(error <= 0.05); + } + + @Test + public void inputSketch() { + double probability = 0.5; + initData( "sketch", 2, probability); + long count = 100000; + Map fields = event.getExtractedFields(); + + ArrayHistogram his = new ArrayHistogram(2); + for (int i = 1; i <= count; i++) { + his.recordValue(i); + } + fields.put("ms", StringUtils.encodeBase64String(his.toBytes())); + agg.add(event, acc); + + his = new ArrayHistogram(2); + for (int i = 1; i <= count; i++) { + his.recordValue(i); + } + fields.put("ms", his.toBytes()); + agg.add(event, acc); + + long expect = (long) (count * probability); + long rst = (long)agg.getResult(acc).getMetricsFields().get("ms_his"); + double error = Math.abs(rst - expect) / (double) expect; + System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error)); + assertTrue(error <= 0.05); + } + + private void initData(String input_type, int numberOfSignificantValueDigits, double probability){ + agg = new HdrHistogramQuantile(); + UDFContext c = new UDFContext(); + Map parameters = new HashMap<>(); + parameters.put("input_type", input_type); + parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits); + parameters.put("probability", probability); + c.setParameters(parameters); + c.setLookup_fields(Collections.singletonList("ms")); + c.setOutput_fields(Collections.singletonList("ms_his")); + + agg.open(c); + Map map = new HashMap<>(); + acc = new Accumulator(); + acc.setMetricsFields(map); + agg.initAccumulator(acc); + + event = new Event(); + Map fields = new HashMap<>(); + event.setExtractedFields(fields); + } +} \ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java new file mode 100644 index 0000000..4eefd9a --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java @@ -0,0 +1,98 @@ +package com.geedgenetworks.core.udf.udaf.HdrHistogram; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.util.StringUtils; +import org.HdrHistogram.ArrayHistogram; +import org.junit.jupiter.api.Test; + +import java.util.*; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class HdrHistogramQuantilesTest { + AggregateFunction agg; + Accumulator acc; + Event event; + + @Test + public void inputRegular() { + double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1}; + initData( "regular", 2, probabilities); + long count = 100000; + Map fields = event.getExtractedFields(); + for (int i = 1; i <= count; i++) { + fields.put("ms", i); + agg.add(event, acc); + } + + long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray(); + + List rsts = (List)agg.getResult(acc).getMetricsFields().get("ms_his"); + for (int i = 0; i < expects.length; i++) { + long rst = rsts.get(i); + long expect = expects[i]; + double probability = probabilities[i]; + double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect; + System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error)); + assertTrue(error <= 0.05); + } + } + + @Test + public void inputSketch() { + double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1}; + initData( "sketch", 2, probabilities); + long count = 100000; + Map fields = event.getExtractedFields(); + long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray(); + + ArrayHistogram his = new ArrayHistogram(2); + for (int i = 1; i <= count; i++) { + his.recordValue(i); + } + fields.put("ms", StringUtils.encodeBase64String(his.toBytes())); + agg.add(event, acc); + + his = new ArrayHistogram(2); + for (int i = 1; i <= count; i++) { + his.recordValue(i); + } + fields.put("ms", his.toBytes()); + agg.add(event, acc); + + List rsts = (List)agg.getResult(acc).getMetricsFields().get("ms_his"); + for (int i = 0; i < expects.length; i++) { + long rst = rsts.get(i); + long expect = expects[i]; + double probability = probabilities[i]; + double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect; + System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error)); + assertTrue(error <= 0.05); + } + } + + private void initData(String input_type, int numberOfSignificantValueDigits, double[] probabilities){ + agg = new HdrHistogramQuantiles(); + UDFContext c = new UDFContext(); + Map parameters = new HashMap<>(); + parameters.put("input_type", input_type); + parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits); + parameters.put("probabilities", probabilities); + c.setParameters(parameters); + c.setLookup_fields(Collections.singletonList("ms")); + c.setOutput_fields(Collections.singletonList("ms_his")); + + agg.open(c); + Map map = new HashMap<>(); + acc = new Accumulator(); + acc.setMetricsFields(map); + agg.initAccumulator(acc); + + event = new Event(); + Map fields = new HashMap<>(); + event.setExtractedFields(fields); + } +} \ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java new file mode 100644 index 0000000..f177ca5 --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java @@ -0,0 +1,102 @@ +package com.geedgenetworks.core.udf.udaf.HdrHistogram; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.util.StringUtils; +import org.HdrHistogram.ArrayHistogram; +import org.junit.jupiter.api.Test; + +import java.nio.charset.StandardCharsets; +import java.util.*; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class HdrHistogramTest { + AggregateFunction agg; + Accumulator acc; + Event event; + + @Test + public void inputRegular() { + double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1}; + initData( "regular", 2, "base64"); + long count = 100000; + Map fields = event.getExtractedFields(); + for (int i = 1; i <= count; i++) { + fields.put("ms", i); + agg.add(event, acc); + } + + long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray(); + String str = (String) agg.getResult(acc).getMetricsFields().get("ms_his"); + ArrayHistogram his = ArrayHistogram.fromBytes(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8))); + + for (int i = 0; i < expects.length; i++) { + long rst = his.getValueAtPercentile(probabilities[i] * 100); + long expect = expects[i]; + double probability = probabilities[i]; + double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect; + System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error)); + assertTrue(error <= 0.05); + } + } + + @Test + public void inputSketch() { + double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1}; + initData( "sketch", 2, "binary"); + long count = 100000; + Map fields = event.getExtractedFields(); + long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray(); + + ArrayHistogram his = new ArrayHistogram(2); + for (int i = 1; i <= count; i++) { + his.recordValue(i); + } + fields.put("ms", StringUtils.encodeBase64String(his.toBytes())); + agg.add(event, acc); + + his = new ArrayHistogram(2); + for (int i = 1; i <= count; i++) { + his.recordValue(i); + } + fields.put("ms", his.toBytes()); + agg.add(event, acc); + + byte[] bytes = (byte[]) agg.getResult(acc).getMetricsFields().get("ms_his"); + ArrayHistogram h = ArrayHistogram.fromBytes(bytes); + + for (int i = 0; i < expects.length; i++) { + long rst = h.getValueAtPercentile(probabilities[i] * 100); + long expect = expects[i]; + double probability = probabilities[i]; + double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect; + System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error)); + assertTrue(error <= 0.05); + } + } + + private void initData(String input_type, int numberOfSignificantValueDigits, String output_format){ + agg = new HdrHistogram(); + UDFContext c = new UDFContext(); + Map parameters = new HashMap<>(); + parameters.put("input_type", input_type); + parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits); + parameters.put("output_format", output_format); + c.setParameters(parameters); + c.setLookup_fields(Collections.singletonList("ms")); + c.setOutput_fields(Collections.singletonList("ms_his")); + + agg.open(c); + Map map = new HashMap<>(); + acc = new Accumulator(); + acc.setMetricsFields(map); + agg.initAccumulator(acc); + + event = new Event(); + Map fields = new HashMap<>(); + event.setExtractedFields(fields); + } +} \ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java new file mode 100644 index 0000000..eae356d --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java @@ -0,0 +1,87 @@ +package com.geedgenetworks.core.udf.udaf.hlld; + + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.hlld.Hll; +import com.geedgenetworks.sketch.util.StringUtils; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertTrue; + + +public class HlldApproxCountDistinctTest { + AggregateFunction agg; + Accumulator acc; + Event event; + + + @Test + public void inputRegular() { + initData(14, "regular"); + long count = 100000; + Map fields = event.getExtractedFields(); + for (int i = 0; i < count; i++) { + fields.put("ip", i); + agg.add(event, acc); + } + + long rst = (long)agg.getResult(acc).getMetricsFields().get("ip_cnt"); + double error = Math.abs(rst - count) / (double) count; + System.out.println(String.format("%d,%d,%.4f", count , rst , error)); + assertTrue(error <= 0.05); + } + + @Test + public void inputSketch() { + initData(14, "sketch"); + long count = 150000; + Map fields = event.getExtractedFields(); + + Hll hll = new Hll(12); + for (int i = 0; i < 100000; i++) { + hll.add(i); + } + fields.put("ip", StringUtils.encodeBase64String(hll.toBytes())); + agg.add(event, acc); + + hll = new Hll(13); + for (int i = 50000; i < 150000; i++) { + hll.add(i); + } + fields.put("ip", hll.toBytes()); + agg.add(event, acc); + + long rst = (long)agg.getResult(acc).getMetricsFields().get("ip_cnt"); + double error = Math.abs(rst - count) / (double) count; + System.out.println(String.format("%d,%d,%.4f", count , rst , error)); + assertTrue(error <= 0.05); + } + + private void initData(int precision, String input_type){ + agg = new HlldApproxCountDistinct(); + UDFContext c = new UDFContext(); + Map parameters = new HashMap<>(); + parameters.put("precision", precision); + parameters.put("input_type", input_type); + c.setParameters(parameters); + c.setLookup_fields(Collections.singletonList("ip")); + c.setOutput_fields(Collections.singletonList("ip_cnt")); + + agg.open(c); + Map map = new HashMap<>(); + acc = new Accumulator(); + acc.setMetricsFields(map); + agg.initAccumulator(acc); + + event = new Event(); + Map fields = new HashMap<>(); + event.setExtractedFields(fields); + } +} \ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java new file mode 100644 index 0000000..f489ee4 --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java @@ -0,0 +1,86 @@ +package com.geedgenetworks.core.udf.udaf.hlld; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.hlld.Hll; +import com.geedgenetworks.sketch.util.StringUtils; +import org.junit.jupiter.api.Test; + +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class HlldTest { + AggregateFunction agg; + Accumulator acc; + Event event; + + @Test + public void inputRegular() { + initData(14, "regular", "base64"); + long count = 100000; + Map fields = event.getExtractedFields(); + for (int i = 0; i < count; i++) { + fields.put("ip", i); + agg.add(event, acc); + } + + String hllStr = (String)agg.getResult(acc).getMetricsFields().get("ip_cnt"); + long rst = (long) Hll.fromBytes(StringUtils.decodeBase64(hllStr.getBytes(StandardCharsets.UTF_8))).size(); + double error = Math.abs(rst - count) / (double) count; + System.out.println(String.format("%d,%d,%.4f", count , rst , error)); + assertTrue(error <= 0.05); + } + + @Test + public void inputSketch() { + initData(14, "sketch", "binary"); + long count = 150000; + Map fields = event.getExtractedFields(); + for (int i = 0; i < 100000; i++) { + Hll hll = new Hll(12); + hll.add(i); + fields.put("ip", StringUtils.encodeBase64String(hll.toBytes())); + agg.add(event, acc); + } + for (int i = 50000; i < 150000; i++) { + Hll hll = new Hll(13); + hll.add(i); + fields.put("ip", hll.toBytes()); + agg.add(event, acc); + } + + byte[] hllBytes = (byte[])agg.getResult(acc).getMetricsFields().get("ip_cnt"); + long rst = (long) Hll.fromBytes(hllBytes).size(); + double error = Math.abs(rst - count) / (double) count; + System.out.println(String.format("%d,%d,%.4f", count , rst , error)); + assertTrue(error <= 0.05); + } + + private void initData(int precision, String input_type, String output_format){ + agg = new Hlld(); + UDFContext c = new UDFContext(); + Map parameters = new HashMap<>(); + parameters.put("precision", precision); + parameters.put("input_type", input_type); + parameters.put("output_format", output_format); + c.setParameters(parameters); + c.setLookup_fields(Collections.singletonList("ip")); + c.setOutput_fields(Collections.singletonList("ip_cnt")); + + agg.open(c); + Map map = new HashMap<>(); + acc = new Accumulator(); + acc.setMetricsFields(map); + agg.initAccumulator(acc); + + event = new Event(); + Map fields = new HashMap<>(); + event.setExtractedFields(fields); + } +} diff --git a/pom.xml b/pom.xml index 177aa11..8378a72 100644 --- a/pom.xml +++ b/pom.xml @@ -268,6 +268,11 @@ fastjson ${fastjson2.version} + + com.geedgenetworks + sketches + 1.0.0 + com.alibaba.nacos nacos-client -- cgit v1.2.3 From 1da4d0d8092476bbbd17a5838f792215ee4d05ec Mon Sep 17 00:00:00 2001 From: doufenghu Date: Thu, 15 Aug 2024 16:18:41 +0800 Subject: [docs][udf] Update scalar UDFs, user-defined aggregate functions (UDAFs) description. --- README.md | 2 +- docs/processor/aggregate-processor.md | 4 ++-- docs/processor/projection-processor.md | 4 ++-- docs/processor/udf.md | 4 ++-- docs/user-guide.md | 2 +- .../java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java | 1 + .../main/java/com/geedgenetworks/example/GrootStreamExample.java | 4 +++- .../src/main/resources/examples/inline_to_kafka.yaml | 6 +++--- .../src/test/resources/clickhouse_data_type_sink.yaml | 3 ++- 9 files changed, 17 insertions(+), 13 deletions(-) diff --git a/README.md b/README.md index 633f803..d6485d9 100644 --- a/README.md +++ b/README.md @@ -27,7 +27,7 @@ Groot Stream Platform helps you process netflow data - logs, metrics etc. - in r Configure a job, you'll set up Sources, Filters, Processing Pipeline, and Sinks, and will assemble several built-in functions into a Processing Pipeline. The job will then be deployed to a Flink cluster for execution. - **Source**: The data source of the job, which can be a Kafka topic, a IPFIX Collector, or a file. - **Filter**: Filters data based on specified conditions. -- **Types of Pipelines**: The fundamental unit of data stream processing is the processor, categorized by functionality into stateless and stateful processors. Each processor can be assemble `UDFs`(User-defined functions) or `UDAFs`(User-defined aggregation functions) into a pipeline. There are 3 types of pipelines at different stages of the data processing process: +- **Types of Pipelines**: The fundamental unit of data stream processing is the processor, categorized by functionality into stateless and stateful processors. Each processor can be assemble `UDFs`(User-defined functions) into a pipeline. There are 3 types of pipelines at different stages of the data processing process: - **Pre-processing Pipeline**: Optional. These pipelines that are attached to a source to normalize the events before they enter the processing pipeline. - **Processing Pipeline**: Event processing pipeline. - **Post-processing Pipeline**: Optional. These pipelines that are attached to a sink to normalize the events before they're written to the sink. diff --git a/docs/processor/aggregate-processor.md b/docs/processor/aggregate-processor.md index d9bcdb0..af82d4e 100644 --- a/docs/processor/aggregate-processor.md +++ b/docs/processor/aggregate-processor.md @@ -1,11 +1,11 @@ # Aggregate Processor -> Processing pipelines for aggregate processor +> Processing pipelines for aggregate processors using UDAFs ## Description Aggregate processor is used to aggregate the data from source to sink. It is a part of the processing pipeline. It can be used in the pre-processing, processing, and post-processing pipeline. Each processor can assemble UDAFs(User-defined Aggregate functions) into a pipeline. -Within the pipeline, events are processed by each Function in order, top‑>down. The UDAF usage detail can be found in [UDAF](udaf.md). +Within the pipeline, events are processed by each Function in order, top‑>down. More details can be found in user-defined aggregate functions [(UDAFs)](udaf.md). ## Options Note:Default will output internal fields `__window_start_timestamp` and `__window_end_timestamp` if not set output_fields. diff --git a/docs/processor/projection-processor.md b/docs/processor/projection-processor.md index bc4b249..4319f36 100644 --- a/docs/processor/projection-processor.md +++ b/docs/processor/projection-processor.md @@ -1,12 +1,12 @@ # Projection Processor -> Processing pipelines for projection processor +> Processing pipelines for projection processors using scalar UDFs ## Description Projection processor is used to project the data from source to sink. It can be used to filter, remove, and transform fields. It is a part of the processing pipeline. It can be used in the pre-processing, processing, and post-processing pipeline. Each processor can assemble UDFs(User-defined functions) into a pipeline. -Within the pipeline, events are processed by each Function in order, top‑>down. The UDF usage detail can be found in [UDF](udf.md). +Within the pipeline, events are processed by each Function in order, top‑>down. More details can be found in User Defined Functions [(UDFs)](udf.md). ## Options diff --git a/docs/processor/udf.md b/docs/processor/udf.md index cf305ef..170d86f 100644 --- a/docs/processor/udf.md +++ b/docs/processor/udf.md @@ -1,6 +1,6 @@ # UDF -> The functions for projection processors. +> The functions for projection processors. ## Function of content @@ -24,7 +24,7 @@ ## Description -UDF(User Defined Function) is used to extend the functions of projection processor. The UDF is a part of the processing pipeline. It can be used in the pre-processing pipeline, processing pipeline, and post-processing pipeline. +Scalar UDF(User Defined Function) is used to extend the functions of projection processor. The UDF is a part of the processing pipeline. It can be used in the pre-processing pipeline, processing pipeline, and post-processing pipeline. ## UDF Definition diff --git a/docs/user-guide.md b/docs/user-guide.md index 9d5b1c7..e35616f 100644 --- a/docs/user-guide.md +++ b/docs/user-guide.md @@ -137,7 +137,7 @@ Based on the filter expression, the event will be passed to downstream if the ex ## Processing Pipelines Processing pipelines are used to define the event processing logic of the job. It can be categorized by functionality into stateless and stateful processors. Based processing order, it can be categorized into pre-processing pipeline, processing pipeline and post-processing pipeline. Each processor can assemble `UDFs`(User-defined functions) into a pipeline. The detail of processor is listed in [Processor](processor). - +UDF supports [scalar UDFs](processor/udf.md) , user-defined aggregate functions [(UDAFs)](processor/udaf.md), and user-defined table functions (UDTFs). ## Sinks Sink is used to define where GrootStream needs to output data. Multiple sinks can be defined in a job. The supported sinks are listed in [Sink Connectors](connector/sink). Each sink has its own specific parameters to define how to output data, and GrootStream also extracts the properties that each sink will use, such as the `topic` and `kafka.bootstrap.servers` of the `Kafka` sink. diff --git a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java index 336842f..c3746a4 100644 --- a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java +++ b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java @@ -68,5 +68,6 @@ public class ConfigShadeTest { System.out.println( ConfigShadeUtils.encryptOption("aes", "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"admin\" password=\"galaxy2019\";")); System.out.println( ConfigShadeUtils.decryptOption("aes", "454f65ea6eef1256e3067104f82730e737b68959560966b811e7ff364116b03124917eb2b0f3596f14733aa29ebad9352644ce1a5c85991c6f01ba8a5e8f177a7ff0b2d3889a424249967b3870b50993d9644f239f0de82cdb13bdb502959e16afadffa49ef1e1d2b9c9b5113e619817")); System.out.println( ConfigShadeUtils.encryptOption("aes", "testuser")); + System.out.println( ConfigShadeUtils.encryptOption("aes", "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"olap\" password=\"galaxy2019\";")); } } diff --git a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java index 690f21c..f666ee8 100644 --- a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java +++ b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java @@ -13,11 +13,13 @@ import java.nio.file.Paths; public class GrootStreamExample { public static void main(String[] args) throws FileNotFoundException, URISyntaxException { - String configPath = args.length > 0 ? args[0] : "/examples/session_record_mock_to_print_with_aggregation.yaml"; + String configPath = args.length > 0 ? args[0] : "/examples/inline_to_kafka.yaml"; String configFile = getTestConfigFile(configPath); ExecuteCommandArgs executeCommandArgs = new ExecuteCommandArgs(); executeCommandArgs.setConfigFile(configFile); executeCommandArgs.setCheckConfig(false); + executeCommandArgs.setEncrypt(true); + executeCommandArgs.setDecrypt(false); executeCommandArgs.setVersion(false); executeCommandArgs.setDeployMode(DeployMode.RUN); executeCommandArgs.setTargetType(TargetType.LOCAL); diff --git a/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_kafka.yaml b/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_kafka.yaml index a5c5ece..517d29b 100644 --- a/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_kafka.yaml +++ b/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_kafka.yaml @@ -46,7 +46,7 @@ sinks: kafka.compression.type: snappy kafka.security.protocol: SASL_PLAINTEXT kafka.sasl.mechanism: PLAIN - kafka.sasl.jaas.config: org.apache.kafka.common.security.plain.PlainLoginModule required username="admin" password="galaxy2019"; + kafka.sasl.jaas.config: 454f65ea6eef1256e3067104f82730e737b68959560966b811e7ff364116b03124917eb2b0f3596f14733aa29ebad9352644ce1a5c85991c6f01ba8a5e8f177a80bea937958aaa485c2acc2b475603495a23eb59f055e037c0b186acb22886bd0275ca91f1633441d9943e7962942252 format: json log.failures.only: true @@ -64,7 +64,7 @@ sinks: kafka.compression.type: snappy kafka.security.protocol: SASL_PLAINTEXT kafka.sasl.mechanism: PLAIN - kafka.sasl.jaas.config: org.apache.kafka.common.security.plain.PlainLoginModule required username="admin" password="galaxy2019"; + kafka.sasl.jaas.config: 454f65ea6eef1256e3067104f82730e737b68959560966b811e7ff364116b03124917eb2b0f3596f14733aa29ebad9352644ce1a5c85991c6f01ba8a5e8f177a7ff0b2d3889a424249967b3870b50993d9644f239f0de82cdb13bdb502959e16afadffa49ef1e1d2b9c9b5113e619817 format: json log.failures.only: true @@ -72,7 +72,7 @@ application: # [object] Define job configuration env: name: example-inline-to-kafka parallelism: 3 - shade.identifier: default + shade.identifier: aes pipeline: object-reuse: true topology: diff --git a/groot-tests/test-e2e-clickhouse/src/test/resources/clickhouse_data_type_sink.yaml b/groot-tests/test-e2e-clickhouse/src/test/resources/clickhouse_data_type_sink.yaml index eb020c1..fcff8a3 100644 --- a/groot-tests/test-e2e-clickhouse/src/test/resources/clickhouse_data_type_sink.yaml +++ b/groot-tests/test-e2e-clickhouse/src/test/resources/clickhouse_data_type_sink.yaml @@ -57,7 +57,8 @@ sinks: type: clickhouse properties: host: clickhouse:9000 - table: default.sink_table + table: sink_table + connection.database: default batch.size: 10 batch.byte.size: 200MB batch.interval: 1s -- cgit v1.2.3 From a31d9e1c5c7fb0313acc3d59feaf82e439af1a04 Mon Sep 17 00:00:00 2001 From: doufenghu Date: Mon, 19 Aug 2024 16:31:10 +0800 Subject: [improve][e2e-clickhouse] Close ResultSet after used. --- .../test/e2e/clickhouse/ClickHouseIT.java | 66 +++++++++++----------- .../test/resources/clickhouse_data_type_sink.yaml | 6 +- .../test/resources/init/clickhouse_test_sql.conf | 17 ------ .../com/geedgenetworks/test/e2e/kafka/KafkaIT.java | 6 +- 4 files changed, 39 insertions(+), 56 deletions(-) diff --git a/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java b/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java index 8eab377..20caace 100644 --- a/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java +++ b/groot-tests/test-e2e-clickhouse/src/test/java/com/geedgenetworks/test/e2e/clickhouse/ClickHouseIT.java @@ -143,15 +143,16 @@ public class ClickHouseIT extends TestSuiteBase implements TestResource { .untilAsserted( () -> { assertHasData(SINK_TABLE); + compareResult(); clearTable(SINK_TABLE); }); } private void assertHasData(String table) { - try (Statement statement = connection.createStatement()) { - String sql = String.format("select * from %s.%s limit 1", DATABASE, table); - ResultSet source = statement.executeQuery(sql); + String sql = String.format("select * from %s.%s limit 1", DATABASE, table); + try (Statement statement = connection.createStatement(); + ResultSet source = statement.executeQuery(sql);) { Assertions.assertTrue(source.next()); } catch (SQLException e) { throw new RuntimeException("test clickhouse server image error", e); @@ -167,43 +168,42 @@ public class ClickHouseIT extends TestSuiteBase implements TestResource { } private void compareResult() throws SQLException, IOException { - String sourceSql = "select * from " + SOURCE_TABLE + " order by id"; + String sourceSql = "select * from " + SOURCE_TABLE + " order by id "; String sinkSql = "select * from " + SINK_TABLE + " order by id"; - List columnList = - Arrays.stream(default_columns).collect(Collectors.toList()); - Statement sourceStatement = connection.createStatement(); - Statement sinkStatement = connection.createStatement(); - ResultSet sourceResultSet = sourceStatement.executeQuery(sourceSql); - ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql); - Assertions.assertEquals( - sourceResultSet.getMetaData().getColumnCount(), - sinkResultSet.getMetaData().getColumnCount()); - while (sourceResultSet.next()) { - if (sinkResultSet.next()) { - for (String column : columnList) { - Object source = sourceResultSet.getObject(column); - Object sink = sinkResultSet.getObject(column); - if (!Objects.deepEquals(source, sink)) { - InputStream sourceAsciiStream = sourceResultSet.getBinaryStream(column); - InputStream sinkAsciiStream = sinkResultSet.getBinaryStream(column); - String sourceValue = - IOUtils.toString(sourceAsciiStream, StandardCharsets.UTF_8); - String sinkValue = - IOUtils.toString(sinkAsciiStream, StandardCharsets.UTF_8); - Assertions.assertEquals(sourceValue, sinkValue); - } - Assertions.assertTrue(true); + try (Statement sourceStatement = connection.createStatement(); + Statement sinkStatement = connection.createStatement(); + ResultSet sourceResultSet = sourceStatement.executeQuery(sourceSql); + ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql)) { + Assertions.assertEquals( + sourceResultSet.getMetaData().getColumnCount(), + sinkResultSet.getMetaData().getColumnCount()); + String columns = String.join(",", default_columns); + Assertions.assertTrue( + compare(String.format(CONFIG.getString(COMPARE_SQL), columns, columns))); + } + + sourceSql = "select count(distinct id) as count from " + SOURCE_TABLE; + sinkSql = "select count(distinct id) as count from " + SINK_TABLE; + + try (Statement sourceStatement = connection.createStatement(); + Statement sinkStatement = connection.createStatement(); + ResultSet sourceResultSet = sourceStatement.executeQuery(sourceSql); + ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql)) { + while (sourceResultSet.next()) { + if (sinkResultSet.next()) { + int sinkUniqueIds = sinkResultSet.getInt("count"); + int sourceUniqueIds = sourceResultSet.getInt("count"); + Assertions.assertEquals(sinkUniqueIds, sourceUniqueIds); } } } - String columns = String.join(",", default_columns); - Assertions.assertTrue( - compare(String.format(CONFIG.getString(COMPARE_SQL), columns, columns))); + + } private Boolean compare(String sql) { - try (Statement statement = connection.createStatement()) { - ResultSet resultSet = statement.executeQuery(sql); + try (Statement statement = connection.createStatement(); + ResultSet resultSet = statement.executeQuery(sql);) { return !resultSet.next(); } catch (SQLException e) { throw new RuntimeException("result compare error", e); diff --git a/groot-tests/test-e2e-clickhouse/src/test/resources/clickhouse_data_type_sink.yaml b/groot-tests/test-e2e-clickhouse/src/test/resources/clickhouse_data_type_sink.yaml index fcff8a3..3406a67 100644 --- a/groot-tests/test-e2e-clickhouse/src/test/resources/clickhouse_data_type_sink.yaml +++ b/groot-tests/test-e2e-clickhouse/src/test/resources/clickhouse_data_type_sink.yaml @@ -47,8 +47,8 @@ sources: # data: '[{"id":0,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":0,"c_lowcardinality":"string"},{"id":1,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":1,"c_lowcardinality":"string"},{"id":2,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":2,"c_lowcardinality":"string"},{"id":3,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":3,"c_lowcardinality":"string"},{"id":4,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":4,"c_lowcardinality":"string"},{"id":5,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":5,"c_lowcardinality":"string"},{"id":6,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":6,"c_lowcardinality":"string"},{"id":7,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":7,"c_lowcardinality":"string"},{"id":8,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":8,"c_lowcardinality":"string"},{"id":9,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":9,"c_lowcardinality":"string"},{"id":10,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":10,"c_lowcardinality":"string"},{"id":11,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":11,"c_lowcardinality":"string"},{"id":12,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":12,"c_lowcardinality":"string"},{"id":13,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":13,"c_lowcardinality":"string"},{"id":14,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":14,"c_lowcardinality":"string"},{"id":15,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":15,"c_lowcardinality":"string"},{"id":16,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":16,"c_lowcardinality":"string"},{"id":17,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":17,"c_lowcardinality":"string"},{"id":18,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":18,"c_lowcardinality":"string"},{"id":19,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":19,"c_lowcardinality":"string"},{"id":20,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":20,"c_lowcardinality":"string"},{"id":21,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":21,"c_lowcardinality":"string"},{"id":22,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":22,"c_lowcardinality":"string"},{"id":23,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":23,"c_lowcardinality":"string"},{"id":24,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":24,"c_lowcardinality":"string"},{"id":25,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":25,"c_lowcardinality":"string"},{"id":26,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":26,"c_lowcardinality":"string"},{"id":27,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":27,"c_lowcardinality":"string"},{"id":28,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":28,"c_lowcardinality":"string"},{"id":29,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":29,"c_lowcardinality":"string"},{"id":30,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":30,"c_lowcardinality":"string"},{"id":31,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":31,"c_lowcardinality":"string"},{"id":32,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":32,"c_lowcardinality":"string"},{"id":33,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":33,"c_lowcardinality":"string"},{"id":34,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":34,"c_lowcardinality":"string"},{"id":35,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":35,"c_lowcardinality":"string"},{"id":36,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":36,"c_lowcardinality":"string"},{"id":37,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":37,"c_lowcardinality":"string"},{"id":38,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":38,"c_lowcardinality":"string"},{"id":39,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":39,"c_lowcardinality":"string"},{"id":40,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":40,"c_lowcardinality":"string"},{"id":41,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":41,"c_lowcardinality":"string"},{"id":42,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":42,"c_lowcardinality":"string"},{"id":43,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":43,"c_lowcardinality":"string"},{"id":44,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":44,"c_lowcardinality":"string"},{"id":45,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":45,"c_lowcardinality":"string"},{"id":46,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":46,"c_lowcardinality":"string"},{"id":47,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":47,"c_lowcardinality":"string"},{"id":48,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":48,"c_lowcardinality":"string"},{"id":49,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":49,"c_lowcardinality":"string"},{"id":50,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":50,"c_lowcardinality":"string"},{"id":51,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":51,"c_lowcardinality":"string"},{"id":52,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":52,"c_lowcardinality":"string"},{"id":53,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":53,"c_lowcardinality":"string"},{"id":54,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":54,"c_lowcardinality":"string"},{"id":55,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":55,"c_lowcardinality":"string"},{"id":56,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":56,"c_lowcardinality":"string"},{"id":57,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":57,"c_lowcardinality":"string"},{"id":58,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":58,"c_lowcardinality":"string"},{"id":59,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":59,"c_lowcardinality":"string"},{"id":60,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":60,"c_lowcardinality":"string"},{"id":61,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":61,"c_lowcardinality":"string"},{"id":62,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":62,"c_lowcardinality":"string"},{"id":63,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":63,"c_lowcardinality":"string"},{"id":64,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":64,"c_lowcardinality":"string"},{"id":65,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":65,"c_lowcardinality":"string"},{"id":66,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":66,"c_lowcardinality":"string"},{"id":67,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":67,"c_lowcardinality":"string"},{"id":68,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":68,"c_lowcardinality":"string"},{"id":69,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":69,"c_lowcardinality":"string"},{"id":70,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":70,"c_lowcardinality":"string"},{"id":71,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":71,"c_lowcardinality":"string"},{"id":72,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":72,"c_lowcardinality":"string"},{"id":73,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":73,"c_lowcardinality":"string"},{"id":74,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":74,"c_lowcardinality":"string"},{"id":75,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":75,"c_lowcardinality":"string"},{"id":76,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":76,"c_lowcardinality":"string"},{"id":77,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":77,"c_lowcardinality":"string"},{"id":78,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":78,"c_lowcardinality":"string"},{"id":79,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":79,"c_lowcardinality":"string"},{"id":80,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":80,"c_lowcardinality":"string"},{"id":81,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":81,"c_lowcardinality":"string"},{"id":82,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":82,"c_lowcardinality":"string"},{"id":83,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":83,"c_lowcardinality":"string"},{"id":84,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":84,"c_lowcardinality":"string"},{"id":85,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":85,"c_lowcardinality":"string"},{"id":86,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":86,"c_lowcardinality":"string"},{"id":87,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":87,"c_lowcardinality":"string"},{"id":88,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":88,"c_lowcardinality":"string"},{"id":89,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":89,"c_lowcardinality":"string"},{"id":90,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":90,"c_lowcardinality":"string"},{"id":91,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":91,"c_lowcardinality":"string"},{"id":92,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":92,"c_lowcardinality":"string"},{"id":93,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":93,"c_lowcardinality":"string"},{"id":94,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":94,"c_lowcardinality":"string"},{"id":95,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":95,"c_lowcardinality":"string"},{"id":96,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":96,"c_lowcardinality":"string"},{"id":97,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":97,"c_lowcardinality":"string"},{"id":98,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":98,"c_lowcardinality":"string"},{"id":99,"c_array_string":["string"],"c_array_short":[1],"c_array_int":[1],"c_array_long":[1],"c_array_float":[1.1],"c_array_double":[1.1],"c_string":"string","c_int8":1,"c_int16":1,"c_int32":1,"c_int64":1,"c_float32":1.1,"c_float64":1.1,"c_decimal":1.1,"c_nullable":99,"c_lowcardinality":"string"}]' format: json - interval.per.row: 1s - repeat.count: 100 + interval.per.row: 10ms + repeat.count: -1 json.ignore.parse.errors: false @@ -59,7 +59,7 @@ sinks: host: clickhouse:9000 table: sink_table connection.database: default - batch.size: 10 + batch.size: 100 batch.byte.size: 200MB batch.interval: 1s connection.user: ee9b0016824d59c8c191aa9633e4b61e diff --git a/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_test_sql.conf b/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_test_sql.conf index c24e632..f132795 100644 --- a/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_test_sql.conf +++ b/groot-tests/test-e2e-clickhouse/src/test/resources/init/clickhouse_test_sql.conf @@ -1,20 +1,3 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - source_table = """ set allow_experimental_geo_types = 1; create table if not exists `default`.source_table( diff --git a/groot-tests/test-e2e-kafka/src/test/java/com/geedgenetworks/test/e2e/kafka/KafkaIT.java b/groot-tests/test-e2e-kafka/src/test/java/com/geedgenetworks/test/e2e/kafka/KafkaIT.java index 1c40e01..50ff9d8 100644 --- a/groot-tests/test-e2e-kafka/src/test/java/com/geedgenetworks/test/e2e/kafka/KafkaIT.java +++ b/groot-tests/test-e2e-kafka/src/test/java/com/geedgenetworks/test/e2e/kafka/KafkaIT.java @@ -163,8 +163,8 @@ public class KafkaIT extends TestSuiteBase implements TestResource { public void testKafkaAsSinkProducerQuota(TestContainer container) throws IOException, InterruptedException { //Create topic with 3 partitions executeShell("kafka-topics --create --topic SESSION-RECORD-QUOTA-TEST --bootstrap-server kafkaCluster:9092 --partitions 3 --replication-factor 1 --command-config /etc/kafka/kafka_client_jass_cli.properties"); - //Set producer quota to 5KB/s - executeShell("kafka-configs --bootstrap-server kafkaCluster:9092 --alter --add-config 'producer_byte_rate=5120' --entity-type users --entity-name admin --entity-type clients --entity-name SESSION-RECORD-QUOTA-TEST --command-config /etc/kafka/kafka_client_jass_cli.properties "); + //Set producer quota to 2KB/s + executeShell("kafka-configs --bootstrap-server kafkaCluster:9092 --alter --add-config 'producer_byte_rate=2048' --entity-type users --entity-name admin --entity-type clients --entity-name SESSION-RECORD-QUOTA-TEST --command-config /etc/kafka/kafka_client_jass_cli.properties "); CompletableFuture.supplyAsync( () -> { @@ -183,7 +183,7 @@ public class KafkaIT extends TestSuiteBase implements TestResource { .untilAsserted( () -> { data.addAll(getKafkaConsumerListData("SESSION-RECORD-QUOTA-TEST")); - Assertions.assertTrue(StringUtils.contains(container.getServerLogs(), "TimeoutException") && data.size()>1000); + Assertions.assertTrue(StringUtils.contains(container.getServerLogs(), "TimeoutException") && data.size()>100); }); } -- cgit v1.2.3 From 515d42ce04e5fd211eec76a83deb38ba13752ef8 Mon Sep 17 00:00:00 2001 From: doufenghu Date: Mon, 19 Aug 2024 16:33:30 +0800 Subject: [docs][mock-connector] add HLLD and HDRHistogram mock data type description. --- config/template/grootstream_job_template.yaml | 36 ++- .../mock_schema/object_statistics_mock_desc.json | 186 +++++++----- .../mock_schema/statistics_rule_mock_desc.json | 320 +++++++++++++-------- docs/connector/connector.md | 94 +++--- docs/images/groot_stream_architecture.jpg | Bin 5054004 -> 5263679 bytes .../geedgenetworks/example/GrootStreamExample.java | 4 +- 6 files changed, 414 insertions(+), 226 deletions(-) diff --git a/config/template/grootstream_job_template.yaml b/config/template/grootstream_job_template.yaml index 9f64abe..7cf50c8 100644 --- a/config/template/grootstream_job_template.yaml +++ b/config/template/grootstream_job_template.yaml @@ -151,7 +151,7 @@ preprocessing_pipelines: # [object] Define Processors for preprocessing pipeline # It will be accomplished the common processing for the event by the user-defined functions. # processing_pipelines: # [object] Define Processors for processing pipelines. - processor: # [object] Define projection processor name, must be unique. + projection_processor: # [object] Define projection processor name, must be unique. type: projection # [string] Processor Type remove_fields: output_fields: @@ -276,7 +276,7 @@ processing_pipelines: # [object] Define Processors for processing pipelines. lookup_fields: [ client_asn,server_asn ] output_fields: [ asn_list ] - metrics_processor: # [object] metrics processing Pipeline + projection_metrics_processor: # [object] metrics processing Pipeline type: projection output_fields: properties: @@ -323,6 +323,38 @@ processing_pipelines: # [object] Define Processors for processing pipelines. parameters: data_center_id_num: 1 + aggregate_processor: # [object] Define aggregate processor name, must be unique. + type: aggregate + group_by_fields: [ recv_time, sled_ip ] # [array of string] Group By Fields + window_type: tumbling_processing_time # [string] Window Type, tumbling_processing_time, tumbling_event_time, sliding_processing_time, sliding_event_time + window_size: 60 + functions: + - function: NUMBER_SUM + lookup_fields: [ received_bytes, sent_bytes ] + output_fields: [ received_bytes_sum ] + + - function: LONG_COUNT + lookup_fields: [ received_bytes ] + output_fields: [ sessions ] + + - function: MEAN + lookup_fields: [ received_bytes ] + output_fields: [ received_bytes_mean ] + parameters: + precision: 2 + + - function: FIRST_VALUE + lookup_fields: [ received_bytes ] + output_fields: [ received_bytes_first ] + + - function: LAST_VALUE + lookup_fields: [ received_bytes ] + output_fields: [ received_bytes_last ] + + - function: COLLECT_LIST + lookup_fields: [ received_bytes ] + output_fields: [ received_bytes_set ] + postprocessing_pipelines: # [object] Define Processors for postprocessing pipelines. postprocessor: # [object] Define projection processor name, must be unique. type: projection diff --git a/config/template/mock_schema/object_statistics_mock_desc.json b/config/template/mock_schema/object_statistics_mock_desc.json index 8542767..fbe5eb5 100644 --- a/config/template/mock_schema/object_statistics_mock_desc.json +++ b/config/template/mock_schema/object_statistics_mock_desc.json @@ -1,77 +1,113 @@ [ - {"name": "name", "type": "String", "options":["object_statistics"]}, - {"name": "timestamp_ms", "type": "Timestamp", "unit":"millis"}, - {"name": "tags", "type": "Object", "fields": [ - {"name": "object_id", "type":"Number","min":1,"max":100}, - {"name": "item_id", "type":"Number","min":1,"max":100000}, - {"name": "object_type", "type":"String","options":["ip","asn","port","url","fqdn","account","subscriberid","keywords","application","fqdn_category"]}, - { - "name": "device_id", - "type": "String", - "options": [ - "9800165603191151", - "9800165603247024", - "9800165802621377", - "9800165603191148" - ] - }, - { - "name": "device_group", - "type": "String", - "options": [ - "OLAP-MOCK-DG-1", - "OLAP-MOCK-DG-2", - "OLAP-MOCK-DG-3", - "OLAP-MOCK-DG-4" - ] - },{ - "name": "data_center", - "type": "String", - "options": [ - "OLAP-MOCK-DC-1", - "OLAP-MOCK-DC-2", - "OLAP-MOCK-DC-3", - "OLAP-MOCK-DC-4" - ] - },{ - "name": "vsys_id", - "type": "Number", - "options": [ - 2048 - ] - } - ] }, - {"name": "fields", "type": "Object", "fields": [ - { - "name": "out_bytes", - "type": "Number", - "min": 1, - "max": 15000 - }, - { - "name": "in_bytes", - "type": "Number", - "min": 1, - "max": 60000 - },{ - "name": "bytes", - "type": "Eval", - "expression": "in_bytes+out_bytes" - },{ - "name": "new_in_sessions", - "type": "Number", - "min": 1, - "max": 10 - },{ - "name": "new_out_sessions", - "type": "Number", - "min": 1, - "max": 100 - },{ - "name": "sessions", - "type": "Eval", - "expression": "new_in_sessions+new_out_sessions" - } - ] } - + { + "name": "name", + "type": "String", + "options": [ + "object_statistics" + ] + }, + { + "name": "timestamp_ms", + "type": "Timestamp", + "unit": "millis" + }, + { + "name": "object_id", + "type": "Number", + "min": 1, + "max": 100 + }, + { + "name": "item_id", + "type": "Number", + "min": 1, + "max": 100000 + }, + { + "name": "object_type", + "type": "String", + "options": [ + "ip", + "asn", + "port", + "url", + "fqdn", + "account", + "subscriberid", + "keywords", + "application", + "fqdn_category" + ] + }, + { + "name": "device_id", + "type": "String", + "options": [ + "9800165603191151", + "9800165603247024", + "9800165802621377", + "9800165603191148" + ] + }, + { + "name": "device_group", + "type": "String", + "options": [ + "OLAP-MOCK-DG-1", + "OLAP-MOCK-DG-2", + "OLAP-MOCK-DG-3", + "OLAP-MOCK-DG-4" + ] + }, + { + "name": "data_center", + "type": "String", + "options": [ + "OLAP-MOCK-DC-1", + "OLAP-MOCK-DC-2", + "OLAP-MOCK-DC-3", + "OLAP-MOCK-DC-4" + ] + }, + { + "name": "vsys_id", + "type": "Number", + "options": [ + 2048 + ] + }, + { + "name": "out_bytes", + "type": "Number", + "min": 1, + "max": 15000 + }, + { + "name": "in_bytes", + "type": "Number", + "min": 1, + "max": 60000 + }, + { + "name": "bytes", + "type": "Eval", + "expression": "in_bytes+out_bytes" + }, + { + "name": "new_in_sessions", + "type": "Number", + "min": 1, + "max": 10 + }, + { + "name": "new_out_sessions", + "type": "Number", + "min": 1, + "max": 100 + }, + { + "name": "sessions", + "type": "Eval", + "expression": "new_in_sessions+new_out_sessions" + } ] \ No newline at end of file diff --git a/config/template/mock_schema/statistics_rule_mock_desc.json b/config/template/mock_schema/statistics_rule_mock_desc.json index c3207ec..91f54e9 100644 --- a/config/template/mock_schema/statistics_rule_mock_desc.json +++ b/config/template/mock_schema/statistics_rule_mock_desc.json @@ -1,122 +1,218 @@ [ - {"name": "name", "type": "String", "options":["statistics_rule"]}, - {"name": "timestamp_ms", "type": "Timestamp", "unit":"millis"}, - {"name": "tags", "type": "Object", "fields": [ - { "name": "unionFields", "type": "Union", "random": false, "unionFields": [ - { "weight": 2, "fields": [ - {"name": "rule_id", "type":"Number", "random": false, "options": [ - 1 - ]}, - {"name": "chart_id", "type":"Number", "random": false,"options": [ - 1 - ]}, - {"name": "template_id", "type":"Number", "random": false, "options": [ - 1 - ]} - ] + { + "name": "name", + "type": "String", + "options": [ + "statistics_rule" + ] + }, + { + "name": "timestamp_ms", + "type": "Timestamp", + "unit": "millis" + }, + { + "name": "unionFields", + "type": "Union", + "random": false, + "unionFields": [ + { + "weight": 2, + "fields": [ + { + "name": "rule_id", + "type": "Number", + "random": false, + "options": [ + 1 + ] + }, + { + "name": "chart_id", + "type": "Number", + "random": false, + "options": [ + 1 + ] + }, + { + "name": "template_id", + "type": "Number", + "random": false, + "options": [ + 1 + ] + } + ] }, - { "weight": 2, "fields": [ - {"name": "rule_id", "type":"Number", "random": false,"options": [ - 4 - ]}, - {"name": "chart_id", "type":"Number", "random": false,"options": [ - 4 - ]}, - {"name": "template_id", "type":"Number", "random": false,"options": [ - 4 - ]}, - { "name": "server_ip", - "type": "IPv4", - "start": "1.0.0.0", - "end": "162.105.10.255"} - ] + { + "weight": 2, + "fields": [ + { + "name": "rule_id", + "type": "Number", + "random": false, + "options": [ + 4 + ] + }, + { + "name": "chart_id", + "type": "Number", + "random": false, + "options": [ + 4 + ] + }, + { + "name": "template_id", + "type": "Number", + "random": false, + "options": [ + 4 + ] + }, + { + "name": "server_ip", + "type": "IPv4", + "start": "1.0.0.0", + "end": "162.105.10.255" + } + ] }, - { "weight": 2, "fields": [ - {"name": "rule_id", "type":"Number", "random": false,"options": [ - 7 - ]}, - {"name": "chart_id", "type":"Number", "random": false,"options": [ - 7 - ]}, - {"name": "template_id", "type":"Number", "random": false,"options": [ - 7 - ]}, - { - "name": "application", - "type": "String", - "options": [ - "ntp", - "stun", - "unknown", - "teredo", - "qq_web_qq", - "kugou", - "quic" - ] - } - ] + { + "weight": 2, + "fields": [ + { + "name": "rule_id", + "type": "Number", + "random": false, + "options": [ + 7 + ] + }, + { + "name": "chart_id", + "type": "Number", + "random": false, + "options": [ + 7 + ] + }, + { + "name": "template_id", + "type": "Number", + "random": false, + "options": [ + 7 + ] + }, + { + "name": "application", + "type": "String", + "options": [ + "ntp", + "stun", + "unknown", + "teredo", + "qq_web_qq", + "kugou", + "quic" + ] + } + ] } ] - }, - { "name": "version", "type": "Number", "options": [1] }, - { - "name": "device_group", - "type": "String", - "options": [ - "OLAP-MOCK-DG-1", - "OLAP-MOCK-DG-2", - "OLAP-MOCK-DG-3", - "OLAP-MOCK-DG-4" - ] - },{ - "name": "data_center", - "type": "String", - "options": [ - "OLAP-MOCK-DC-1", - "OLAP-MOCK-DC-2", - "OLAP-MOCK-DC-3", - "OLAP-MOCK-DC-4" - ] - },{ - "name": "vsys_id", - "type": "Number", - "options": [ - 2048 - ] - } - ] }, - {"name": "fields", "type": "Object", "fields": [ - { "name": "unionFields", "type": "Union", "random": false, "unionFields": [ - { "weight": 2, "fields": [ - { "name": "client_ip_sketch", "type": "String", "options": [ - "AQwAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAGAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAUAAACAAAAAAAAAAAAAAAACAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAIAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAwAAAIAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAYAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAA==" - ,"AQwAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAABAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAUAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAADAAAAQBQAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA==" - ,"AQwAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAABAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAUAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAADAAAAQBQAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA==" - ,"AQwAABAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAABAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAUAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAQBQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA==" - ,"AQwAABAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAABAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAUAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAQBQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA==" - ] } - ] + }, + { + "name": "version", + "type": "Number", + "options": [ + 1 + ] + }, + { + "name": "device_group", + "type": "String", + "options": [ + "OLAP-MOCK-DG-1", + "OLAP-MOCK-DG-2", + "OLAP-MOCK-DG-3", + "OLAP-MOCK-DG-4" + ] + }, + { + "name": "data_center", + "type": "String", + "options": [ + "OLAP-MOCK-DC-1", + "OLAP-MOCK-DC-2", + "OLAP-MOCK-DC-3", + "OLAP-MOCK-DC-4" + ] + }, + { + "name": "vsys_id", + "type": "Number", + "options": [ + 2048 + ] + }, + { + "name": "unionFields", + "type": "Union", + "random": false, + "unionFields": [ + { + "weight": 2, + "fields": [ + { + "name": "client_ip_sketch", + "type": "String", + "options": [ + "AQwAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAGAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAUAAACAAAAAAAAAAAAAAAACAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAIAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAwAAAIAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAYAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAA==", + "AQwAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAABAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAUAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAADAAAAQBQAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA==", + "AQwAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAABAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAUAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAADAAAAQBQAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA==", + "AQwAABAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAABAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAUAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAQBQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA==", + "AQwAABAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAABAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAMAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAUAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAQBQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQAAAAACAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA==" + ] + }, + { + "name": "unique_count_1", + "type": "Hlld", + "itemCount": 100000, + "batchCount": 1000 + }, + { + "name": "distribution_1", + "type": "HdrHistogram", + "max": 100000, + "batchCount": 1000 + } + ] }, - { "weight": 2, "fields": [ - { - "name": "bytes", - "type": "Number", - "min": 1, - "max": 15000 - } - ] + { + "weight": 2, + "fields": [ + { + "name": "bytes", + "type": "Number", + "min": 1, + "max": 15000 + } + ] }, - { "weight": 2, "fields": [ - { - "name": "sessions", - "type": "Number", - "min": 1, - "max": 200 - } - ] + { + "weight": 2, + "fields": [ + { + "name": "sessions", + "type": "Number", + "min": 1, + "max": 200 + } + ] } ] - } - ] } - + } ] \ No newline at end of file diff --git a/docs/connector/connector.md b/docs/connector/connector.md index 1123385..766b73e 100644 --- a/docs/connector/connector.md +++ b/docs/connector/connector.md @@ -85,41 +85,49 @@ schema: The mock data type is used to define the template of the mock data. -| Mock Type | Parameter | Result Type | Default | Description | -|-----------------------------------------|-------------|-----------------------|---------------------|---------------------------------------------------------------------------------------------------------------------------------------------| -| **[Number](#Number)** | - | **int/bigint/double** | - | **Randomly generate a number.** | -| - | min | number | 0 | The minimum value (include). | -| - | max | number | int32.max | The maximum value (exclusive). | -| - | options | array of number | (none) | The optional values. If set, the random value will be selected from the options and `start` and `end` will be ignored. | -| - | random | boolean | true | Default is random mode. If set to false, the value will be generated in order. | -| **[Sequence](#Sequence)** | - | **bigint** | - | **Generate a sequence number based on a specific step value .** | -| - | start | bigint | 0 | The first number in the sequence (include). | -| - | step | bigint | 1 | The number to add to each subsequent value. | -| **[UniqueSequence](#UniqueSequence)** | - | **bigint** | - | **Generate a global unique sequence number.** | -| - | start | bigint | 0 | The first number in the sequence (include). | -| **[String](#String)** | - | string | - | **Randomly generate a string.** | -| - | regex | string | [a-zA-Z]{0,5} | The regular expression. | -| - | options | array of string | (none) | The optional values. If set, the random value will be selected from the options and `regex` will be ignored. | -| - | random | boolean | true | Default is random mode. If set to false, the options value will be generated in order. | -| **[Timestamp](#Timestamp)** | - | **bigint** | - | **Generate a unix timestamp in milliseconds or seconds.** | -| - | unit | string | second | The unit of the timestamp. The optional values are `second`, `millis`. | -| **[FormatTimestamp](#FormatTimestamp)** | - | **string** | - | **Generate a formatted timestamp.** | -| - | format | string | yyyy-MM-dd HH:mm:ss | The format to output. | -| - | utc | boolean | false | Default is local time. If set to true, the time will be converted to UTC time. | -| **[IPv4](#IPv4)** | - | **string** | - | **Randomly generate a IPv4 address.** | -| - | start | string | 0.0.0.0 | The minimum value of the IPv4 address(include). | -| - | end | string | 255.255.255.255 | The maximum value of the IPv4 address(include). | -| **[Expression](#Expression)** | - | string | - | **Use library [Datafaker](https://www.datafaker.net/documentation/expressions/) expressions to generate fake data.** | -| - | expression | string | (none) | The datafaker expression used #{expression}. | -| **[Eval](#Eval)** | - | **string** | - | **Use AviatorScript value expression to generate data.** | -| - | expression | string | (none) | Support basic arithmetic operations and function calls. More details sess [AviatorScript](https://www.yuque.com/boyan-avfmj/aviatorscript). | -| **[Object](#Object)** | - | **struct/object** | - | **Generate a object data structure. It used to define the nested structure of the mock data.** | -| - | fields | array of object | (none) | The fields of the object. | -| **[Union](#Union)** | - | - | - | **Generate a union data structure with multiple mock data type fields.** | -| - | unionFields | array of object | (none) | The fields of the object. | -| - | - fields | - array of object | (none) | | -| - | - weight | - int | 0 | The weight of the generated object. | -| | random | boolean | true | Default is random mode. If set to false, the options value will be generated in order. | +| Mock Type | Parameter | Result Type | Default | Description | +|-----------------------------------------|---------------------------------|-----------------------|---------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------| +| **[Number](#Number)** | - | **int/bigint/double** | - | **Randomly generate a number.** | +| | min | number | 0 | The minimum value (inclusive). | +| | max | number | int32.max | The maximum value (exclusive). | +| | options | array of number | (none) | The optional values. If set, the random value will be selected from the options and `min` and `max` will be ignored. | +| | random | boolean | true | Default is random mode. If set to false, the value will be generated in order. | +| **[Sequence](#Sequence)** | - | **bigint** | - | **Generate a sequence number based on a specific step value.** | +| | start | bigint | 0 | The first number in the sequence (inclusive). | +| | step | bigint | 1 | The number to add to each subsequent value. | +| **[UniqueSequence](#UniqueSequence)** | - | **bigint** | - | **Generate a globally unique sequence number.** | +| | start | bigint | 0 | The first number in the sequence (inclusive). | +| **[String](#String)** | - | string | - | **Randomly generate a string.** | +| | regex | string | [a-zA-Z]{0,5} | The regular expression used to generate the string. | +| | options | array of string | (none) | The optional values. If set, the random value will be selected from the options and `regex` will be ignored. | +| | random | boolean | true | Default is random mode. If set to false, the options value will be generated in order. | +| **[Timestamp](#Timestamp)** | - | **bigint** | - | **Generate a Unix timestamp in milliseconds or seconds.** | +| | unit | string | second | The unit of the timestamp. Options are `second` or `millis`. | +| **[FormatTimestamp](#FormatTimestamp)** | - | **string** | - | **Generate a formatted timestamp.** | +| | format | string | yyyy-MM-dd HH:mm:ss | The format to output the timestamp in. | +| | utc | boolean | false | Default is local time. If set to true, the time will be converted to UTC time. | +| **[IPv4](#IPv4)** | - | **string** | - | **Randomly generate an IPv4 address.** | +| | start | string | 0.0.0.0 | The minimum value of the IPv4 address (inclusive). | +| | end | string | 255.255.255.255 | The maximum value of the IPv4 address (inclusive). | +| **[Expression](#Expression)** | - | string | - | **Use library [Datafaker](https://www.datafaker.net/documentation/expressions/) expressions to generate fake data.** | +| | expression | string | (none) | The Datafaker expression to use, in the format `#{expression}`. | +| **[Hlld](#HLLD)** | - | **string** | - | **Generate a IP Address HyperLogLog data structure and store it as a base64 string. Use library [HLLD](https://github.com/armon/hlld).** | +| | itemCount | bigint | 1000000 | The total number of items. | +| | batchCount | int | 10000 | The number of items in each batch. | +| | precision | int | 12 | The precision of the HyperLogLog data structure. Allowed range is [4, 18]. | +| **[HdrHistogram](#HdrHistogram)** | - | **string** | - | **Generate a Latency HdrHistogram data structure and store it as a base64 string. Use library [HdrHistogram](https://github.com/HdrHistogram/HdrHistogram).** | +| | max | bigint | 100000 | The maximum value of the histogram. | +| | batchCount | int | 1000 | The random number of items in each batch. | +| | numberOfSignificantValueDigits | int | 1 | The precision of the histogram data structure. Allowed range is [1, 5]. | +| **[Eval](#Eval)** | - | **string** | - | **Use AviatorScript value expression to generate data.** | +| | expression | string | (none) | Support basic arithmetic operations and function calls. More details in [AviatorScript](https://www.yuque.com/boyan-avfmj/aviatorscript). | +| **[Object](#Object)** | - | **struct/object** | - | **Generate an object data structure. Used to define the nested structure of the mock data.** | +| | fields | array of object | (none) | The fields of the object. | +| **[Union](#Union)** | - | - | - | **Generate a union data structure with multiple mock data type fields.** | +| | unionFields | array of object | (none) | The fields of the union. | +| | weight | int | 0 | The weight of the generated object. | +| | random | boolean | true | Default is random mode. If set to false, the options value will be generated in order. | + ### Common Parameters @@ -250,6 +258,22 @@ Mock data type supports some common parameters. {"name":"phoneNumber","type":"Expression","expression":"#{phoneNumber.phoneNumber}"} ``` +### HLLD + +- Generate a IP Address HyperLogLog data structure, stored as a base64 string. At most 1000 IP addresses are generated in each batch. + +```json +{"name":"hll","type":"Hlld","itemCount":1000000,"batchCount":1000,"precision":12} +``` + +### HdrHistogram + +- Generate a Latency HdrHistogram data structure, stored as a base64 string. The maximum value of the histogram is 100000, and at most 1000 items are generated in each batch. + +```json +{"name":"distribution","type":"HdrHistogram","max":100000,"batchCount":1000,"numberOfSignificantValueDigits":1} +``` + ### Eval - Generate a value by using AviatorScript expression. Commonly used for arithmetic operations. diff --git a/docs/images/groot_stream_architecture.jpg b/docs/images/groot_stream_architecture.jpg index 1fff0e5..d8f1d4b 100644 Binary files a/docs/images/groot_stream_architecture.jpg and b/docs/images/groot_stream_architecture.jpg differ diff --git a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java index f666ee8..c637e36 100644 --- a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java +++ b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java @@ -13,12 +13,12 @@ import java.nio.file.Paths; public class GrootStreamExample { public static void main(String[] args) throws FileNotFoundException, URISyntaxException { - String configPath = args.length > 0 ? args[0] : "/examples/inline_to_kafka.yaml"; + String configPath = args.length > 0 ? args[0] : "/examples/object_statistics_mock_to_print.yaml"; String configFile = getTestConfigFile(configPath); ExecuteCommandArgs executeCommandArgs = new ExecuteCommandArgs(); executeCommandArgs.setConfigFile(configFile); executeCommandArgs.setCheckConfig(false); - executeCommandArgs.setEncrypt(true); + executeCommandArgs.setEncrypt(false); executeCommandArgs.setDecrypt(false); executeCommandArgs.setVersion(false); executeCommandArgs.setDeployMode(DeployMode.RUN); -- cgit v1.2.3 From fdcccac13d29f553093c420a7d05e60e28e6db0d Mon Sep 17 00:00:00 2001 From: doufenghu Date: Mon, 19 Aug 2024 18:54:33 +0800 Subject: [docs][table processor] add table-processor and udtf description. --- config/udf.plugins | 6 +- docs/processor/aggregate-processor.md | 2 +- docs/processor/table-processor.md | 61 +++++++ docs/processor/udaf.md | 180 ++++++++++++++++++++- docs/processor/udtf.md | 66 ++++++++ groot-common/src/main/resources/udf.plugins | 6 +- .../geedgenetworks/example/GrootStreamExample.java | 2 +- .../examples/inline_to_print_use_udtf.yaml | 36 +++++ 8 files changed, 349 insertions(+), 10 deletions(-) create mode 100644 docs/processor/table-processor.md create mode 100644 docs/processor/udtf.md create mode 100644 groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_use_udtf.yaml diff --git a/config/udf.plugins b/config/udf.plugins index 772d2bc..e4f940f 100644 --- a/config/udf.plugins +++ b/config/udf.plugins @@ -22,10 +22,10 @@ com.geedgenetworks.core.udf.udaf.LongCount com.geedgenetworks.core.udf.udaf.Mean com.geedgenetworks.core.udf.udaf.LastValue com.geedgenetworks.core.udf.udaf.FirstValue -com.geedgenetworks.core.udf.udtf.JsonUnroll -com.geedgenetworks.core.udf.udtf.Unroll com.geedgenetworks.core.udf.udaf.hlld.Hlld com.geedgenetworks.core.udf.udaf.hlld.HlldApproxCountDistinct com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogram com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantile -com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantiles \ No newline at end of file +com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantiles +com.geedgenetworks.core.udf.udtf.JsonUnroll +com.geedgenetworks.core.udf.udtf.Unroll \ No newline at end of file diff --git a/docs/processor/aggregate-processor.md b/docs/processor/aggregate-processor.md index af82d4e..5ab0ae0 100644 --- a/docs/processor/aggregate-processor.md +++ b/docs/processor/aggregate-processor.md @@ -12,7 +12,7 @@ Note:Default will output internal fields `__window_start_timestamp` and `__win | name | type | required | default value | |--------------------------|--------|----------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| type | String | Yes | The type of the processor, now only support `com.geedgenetworks.core.processor.projection.AggregateProcessor` | +| type | String | Yes | The type of the processor, now only support `com.geedgenetworks.core.processor.aggregate.AggregateProcessor` | | output_fields | Array | No | Array of String. The list of fields that need to be kept. Fields not in the list will be removed. | | remove_fields | Array | No | Array of String. The list of fields that need to be removed. | | group_by_fields | Array | yes | Array of String. The list of fields that need to be grouped. | diff --git a/docs/processor/table-processor.md b/docs/processor/table-processor.md new file mode 100644 index 0000000..7b3066c --- /dev/null +++ b/docs/processor/table-processor.md @@ -0,0 +1,61 @@ +# Table Processor + +> Processing pipelines for table processors using UDTFs + +## Description + +Table processor is used to process the data from source to sink. It is a part of the processing pipeline. It can be used in the pre-processing, processing, and post-processing pipeline. Each processor can assemble UDTFs(User-defined Table functions) into a pipeline. Within the pipeline, events are processed by each Function in order, top‑>down. More details can be found in user-defined table functions [(UDTFs)](udtf.md). + +## Options + +| name | type | required | default value | +|-----------------|--------|----------|------------------------------------------------------------------------------------------------------| +| type | String | Yes | The type of the processor, now only support `com.geedgenetworks.core.processor.table.TableProcessor` | +| output_fields | Array | No | Array of String. The list of fields that ne ed to be kept. Fields not in the list will be removed. | +| remove_fields | Array | No | Array of String. The list of fields that need to be removed. | +| functions | Array | No | Array of Object. The list of functions that need to be applied to the data. | + +## Usage Example +This example uses a table processor to unroll the encapsulation field, converting one row into multiple rows. + +```yaml +sources: + inline_source: + type: inline + properties: + data: '[{"tcp_rtt_ms":128,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.ct.cn","http_url":"www.ct.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.11.22.22","server_ip":"8.8.8.8","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":5575,"sent_pkts":97,"sent_bytes":5892,"received_pkts":250,"received_bytes":333931,"encapsulation":"[{\"tunnels_schema_type\":\"MULTIPATH_ETHERNET\",\"c2s_source_mac\":\"48:73:97:96:38:27\",\"c2s_destination_mac\":\"58:b3:8f:fa:3b:11\",\"s2c_source_mac\":\"58:b3:8f:fa:3b:11\",\"s2c_destination_mac\":\"48:73:97:96:38:27\"}]"},{"tcp_rtt_ms":256,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.abc.cn","http_url":"www.cabc.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.168.10.198","server_ip":"4.4.4.4","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":2575,"sent_pkts":197,"sent_bytes":5892,"received_pkts":350,"received_bytes":533931,"device_tag":"{\"tags\":[{\"tag\":\"data_center\",\"value\":\"center-xxg-tsgx\"},{\"tag\":\"device_group\",\"value\":\"group-xxg-tsgx\"}]}"}]' + format: json + json.ignore.parse.errors: false + +processing_pipelines: + table_processor: + type: table + functions: + - function: JSON_UNROLL + lookup_fields: [ encapsulation] + output_fields: [ encapsulation ] + +sinks: + print_sink: + type: print + properties: + format: json + mode: log_warn + +application: + env: + name: example-inline-to-print-use-udtf + parallelism: 3 + pipeline: + object-reuse: true + topology: + - name: inline_source + downstream: [table_processor] + - name: table_processor + downstream: [ print_sink ] + - name: print_sink + downstream: [] + +``` + + diff --git a/docs/processor/udaf.md b/docs/processor/udaf.md index e22846f..dd1dd70 100644 --- a/docs/processor/udaf.md +++ b/docs/processor/udaf.md @@ -11,7 +11,11 @@ - [Long Count](#Long-Count) - [MEAN](#Mean) - [Number SUM](#Number-SUM) - +- [HLLD](#HLLD) +- [Approx Count Distinct HLLD](#Approx-Count-Distinct-HLLD) +- [HDR Histogram](#HDR-Histogram) +- [Approx Quantile HDR](#APPROX_QUANTILE_HDR) +- [Approx Quantiles HDR](#APPROX_QUANTILES_HDR) ## Description @@ -146,4 +150,176 @@ NUMBER_SUM is used to sum the value of the field in the group of events. The loo - function: NUMBER_SUM lookup_fields: [received_bytes] output_fields: [received_bytes_sum] -``` \ No newline at end of file +``` + +### HLLD +hlld is a high-performance C server which is used to expose HyperLogLog sets and operations over them to networked clients. More details can be found in [hlld](https://github.com/armon/hlld). + +```HLLD(filter, lookup_fields, output_fields[, parameters])``` +- filter: optional +- lookup_fields: required. +- output_fields: required. +- parameters: optional. + - input_type: `` optional. input field type can be `regular` or `sketch`. Default is `sketch`. regular field data type includes `string`, `int`, `long`, `float`, `double` etc. + - precision: `` optional. The precision of the hlld value. Default is 12. + - output_format: `` optional. The output format can be either `base64(encoded string)` or `binary(byte[])`. The default is `base64`. + +### Example + Merge multiple string field into a HyperLogLog data structure. +```yaml + - function: HLLD + lookup_fields: [client_ip] + output_fields: [client_ip_hlld] + parameters: + input_type: regular + +``` + Merge multiple `unique_count ` metric type fields into a HyperLogLog data structure +```yaml + - function: HLLD + lookup_fields: [client_ip_hlld] + output_fields: [client_ip_hlld] + parameters: + input_type: sketch +``` + +### Approx Count Distinct HLLD +Approx Count Distinct HLLD is used to count the approximate number of distinct values in the group of events. + +```APPROX_COUNT_DISTINCT_HLLD(filter, lookup_fields, output_fields[, parameters])``` +- filter: optional +- lookup_fields: required. +- output_fields: required. +- parameters: optional. + - input_type: `` optional. Refer to `HLLD` function. + - precision: `` optional. Refer to `HLLD` function. + +### Example + +```yaml +- function: APPROX_COUNT_DISTINCT_HLLD + lookup_fields: [client_ip] + output_fields: [unique_client_ip] + parameters: + input_type: regular +``` + +```yaml +- function: APPROX_COUNT_DISTINCT_HLLD + lookup_fields: [client_ip_hlld] + output_fields: [unique_client_ip] + parameters: + input_type: sketch +``` + +### HDR Histogram + +A High Dynamic Range (HDR) Histogram. More details can be found in [HDR Histogram](https://github.com/HdrHistogram/HdrHistogram). + +```HDR_HISTOGRAM(filter, lookup_fields, output_fields[, parameters])``` +- filter: optional +- lookup_fields: required. +- output_fields: required. +- parameters: optional. + - input_type: `` optional. input field type can be `regular` or `sketch`. Default is `sketch`. regular field is a number. + - lowestDiscernibleValue: `` optional. The lowest trackable value. Default is 1. + - highestTrackableValue: `` optional. The highest trackable value. Default is 2. + - numberOfSignificantValueDigits: `` optional. The number of significant value digits. Default is 1. The range is 1 to 5. + - autoResize: `` optional. If true, the highestTrackableValue will auto-resize. Default is true. + - output_format: `` optional. The output format can be either `base64(encoded string)` or `binary(byte[])`. The default is `base64`. + +### Example + + ```yaml + - function: HDR_HISTOGRAM + lookup_fields: [latency_ms] + output_fields: [latency_ms_histogram] + parameters: + input_type: regular + lowestDiscernibleValue: 1 + highestTrackableValue: 3600000 + numberOfSignificantValueDigits: 3 + ``` + ```yaml + - function: HDR_HISTOGRAM + lookup_fields: [latency_ms_histogram] + output_fields: [latency_ms_histogram] + parameters: + input_type: sketch + ``` + +### Approx Quantile HDR + +Approx Quantile HDR is used to calculate the approximate quantile value of the field in the group of events. + +```APPROX_QUANTILE_HDR(filter, lookup_fields, output_fields, quantile[, parameters])``` +- filter: optional +- lookup_fields: required. +- output_fields: required. +- parameters: optional. + - input_type: `` optional. Refer to `HDR_HISTOGRAM` function. + - lowestDiscernibleValue: `` optional. Refer to `HDR_HISTOGRAM` function. + - highestTrackableValue: `` required. Refer to `HDR_HISTOGRAM` function. + - numberOfSignificantValueDigits: `` optional. Refer to `HDR_HISTOGRAM` function. + - autoResize: `` optional. Refer to `HDR_HISTOGRAM` function. + - probability: `` optional. The probability of the quantile. Default is 0.5. + +### Example + + ```yaml + - function: APPROX_QUANTILE_HDR + lookup_fields: [latency_ms] + output_fields: [latency_ms_p95] + parameters: + input_type: regular + probability: 0.95 + ``` + + ```yaml + - function: APPROX_QUANTILE_HDR + lookup_fields: [latency_ms_HDR] + output_fields: [latency_ms_p95] + parameters: + input_type: sketch + probability: 0.95 + + ``` + +### Approx Quantiles HDR + +Approx Quantiles HDR is used to calculate the approximate quantile values of the field in the group of events. + +```APPROX_QUANTILES_HDR(filter, lookup_fields, output_fields, quantiles[, parameters])``` +- filter: optional +- lookup_fields: required. +- output_fields: required. +- parameters: optional. + - input_type: `` optional. Refer to `HDR_HISTOGRAM` function. + - lowestDiscernibleValue: `` optional. Refer to `HDR_HISTOGRAM` function. + - highestTrackableValue: `` required. Refer to `HDR_HISTOGRAM` function. + - numberOfSignificantValueDigits: `` optional. Refer to `HDR_HISTOGRAM` function. + - autoResize: `` optional. Refer to `HDR_HISTOGRAM` function. + - probabilities: `>` required. The list of probabilities of the quantiles. Range is 0 to 1. + +### Example + +```yaml +- function: APPROX_QUANTILES_HDR + lookup_fields: [latency_ms] + output_fields: [latency_ms_quantiles] + parameters: + input_type: regular + probabilities: [0.5, 0.95, 0.99] +``` + +```yaml +- function: APPROX_QUANTILES_HDR + lookup_fields: [latency_ms_HDR] + output_fields: [latency_ms_quantiles] + parameters: + input_type: sketch + probabilities: [0.5, 0.95, 0.99] +``` + + + diff --git a/docs/processor/udtf.md b/docs/processor/udtf.md new file mode 100644 index 0000000..a6e8444 --- /dev/null +++ b/docs/processor/udtf.md @@ -0,0 +1,66 @@ +# UDTF + +> The functions for table processors. + +## Function of content + +- [UNROLL](#unroll) +- [JSON_UNROLL](#json_unroll) + +## Description + +The UDTFs(user-defined table functions) are used to process the data from source to sink. It is a part of the processing pipeline. It can be used in the pre-processing, processing, and post-processing pipeline. Each processor can assemble UDTFs into a pipeline. Within the pipeline, events are processed by each Function in order, top‑>down. +Unlike scalar functions, which return a single value, UDTFs are particularly useful when you need to explode or unroll data, transforming a single input row into multiple output rows. + +## UDTF Definition + + The UDTFs and UDFs share similar input and context structures, please refer to [UDF](udf.md). + +## Functions + +### UNROLL + +The Unroll Function handles an array field—or an expression evaluating to an array—and unrolls it into individual events. + +```UNROLL(filter, lookup_fields, output_fields[, parameters])``` +- filter: optional +- lookup_fields: required +- output_fields: required +- parameters: optional + - regex: `` optional. If lookup_fields is a string, the regex parameter is used to split the string into an array. The default value is a comma. + +#### Example + +```yaml +functions: + - function: UNROLL + lookup_fields: [ monitor_rule_list ] + output_fields: [ monitor_rule ] +``` + +### JSON_UNROLL + +The JSON Unroll Function handles a JSON object, unrolls/explodes an array of objects therein into individual events, while also inheriting top level fields. + +```JSON_UNROLL(filter, lookup_fields, output_fields[, parameters])``` +- filter: optional +- lookup_fields: required +- output_fields: required +- parameters: optional + - path: `` optional. Path to array to unroll, default is the root of the JSON object. + - new_path: `` optional. Rename path to new_path, default is the same as path. + +#### Example + +```yaml +functions: + - function: JSON_UNROLL + lookup_fields: [ device_tag ] + output_fields: [ device_tag ] + parameters: + - path: tags + - new_path: tag +``` + + + diff --git a/groot-common/src/main/resources/udf.plugins b/groot-common/src/main/resources/udf.plugins index 0062c1a..7544cc7 100644 --- a/groot-common/src/main/resources/udf.plugins +++ b/groot-common/src/main/resources/udf.plugins @@ -21,10 +21,10 @@ com.geedgenetworks.core.udf.udaf.LongCount com.geedgenetworks.core.udf.udaf.Mean com.geedgenetworks.core.udf.udaf.LastValue com.geedgenetworks.core.udf.udaf.FirstValue -com.geedgenetworks.core.udf.udtf.JsonUnroll -com.geedgenetworks.core.udf.udtf.Unroll com.geedgenetworks.core.udf.udaf.hlld.Hlld com.geedgenetworks.core.udf.udaf.hlld.HlldApproxCountDistinct com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogram com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantile -com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantiles \ No newline at end of file +com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantiles +com.geedgenetworks.core.udf.udtf.JsonUnroll +com.geedgenetworks.core.udf.udtf.Unroll \ No newline at end of file diff --git a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java index c637e36..d6f736f 100644 --- a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java +++ b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java @@ -13,7 +13,7 @@ import java.nio.file.Paths; public class GrootStreamExample { public static void main(String[] args) throws FileNotFoundException, URISyntaxException { - String configPath = args.length > 0 ? args[0] : "/examples/object_statistics_mock_to_print.yaml"; + String configPath = args.length > 0 ? args[0] : "/examples/inline_to_print_use_udtf.yaml"; String configFile = getTestConfigFile(configPath); ExecuteCommandArgs executeCommandArgs = new ExecuteCommandArgs(); executeCommandArgs.setConfigFile(configFile); diff --git a/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_use_udtf.yaml b/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_use_udtf.yaml new file mode 100644 index 0000000..edde893 --- /dev/null +++ b/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_use_udtf.yaml @@ -0,0 +1,36 @@ +sources: + inline_source: + type: inline + properties: + data: '[{"tcp_rtt_ms":128,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.ct.cn","http_url":"www.ct.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.11.22.22","server_ip":"8.8.8.8","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":5575,"sent_pkts":97,"sent_bytes":5892,"received_pkts":250,"received_bytes":333931,"encapsulation":"[{\"tunnels_schema_type\":\"MULTIPATH_ETHERNET\",\"c2s_source_mac\":\"48:73:97:96:38:27\",\"c2s_destination_mac\":\"58:b3:8f:fa:3b:11\",\"s2c_source_mac\":\"58:b3:8f:fa:3b:11\",\"s2c_destination_mac\":\"48:73:97:96:38:27\"}]"},{"tcp_rtt_ms":256,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.abc.cn","http_url":"www.cabc.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.168.10.198","server_ip":"4.4.4.4","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":2575,"sent_pkts":197,"sent_bytes":5892,"received_pkts":350,"received_bytes":533931,"device_tag":"{\"tags\":[{\"tag\":\"data_center\",\"value\":\"center-xxg-tsgx\"},{\"tag\":\"device_group\",\"value\":\"group-xxg-tsgx\"}]}"}]' + format: json + json.ignore.parse.errors: false + +processing_pipelines: + table_processor: + type: table + functions: + - function: JSON_UNROLL + lookup_fields: [ encapsulation] + output_fields: [ encapsulation ] + +sinks: + print_sink: + type: print + properties: + format: json + mode: log_warn + +application: + env: + name: example-inline-to-print-with-aggregation + parallelism: 3 + pipeline: + object-reuse: true + topology: + - name: inline_source + downstream: [table_processor] + - name: table_processor + downstream: [ print_sink ] + - name: print_sink + downstream: [] \ No newline at end of file -- cgit v1.2.3 From cf51b22932c6b1d78667a049a23ec472d1c42e58 Mon Sep 17 00:00:00 2001 From: doufenghu Date: Mon, 19 Aug 2024 19:03:59 +0800 Subject: add APPROX_COUNT_DISTINCT_HLLD test case --- .../main/java/com/geedgenetworks/example/GrootStreamExample.java | 2 +- .../main/resources/examples/inline_to_print_with_aggregation.yaml | 8 +++++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java index d6f736f..0eba408 100644 --- a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java +++ b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java @@ -13,7 +13,7 @@ import java.nio.file.Paths; public class GrootStreamExample { public static void main(String[] args) throws FileNotFoundException, URISyntaxException { - String configPath = args.length > 0 ? args[0] : "/examples/inline_to_print_use_udtf.yaml"; + String configPath = args.length > 0 ? args[0] : "/examples/inline_to_print_with_aggregation.yaml"; String configFile = getTestConfigFile(configPath); ExecuteCommandArgs executeCommandArgs = new ExecuteCommandArgs(); executeCommandArgs.setConfigFile(configFile); diff --git a/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_with_aggregation.yaml b/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_with_aggregation.yaml index 6f08be2..04dddf8 100644 --- a/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_with_aggregation.yaml +++ b/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_with_aggregation.yaml @@ -2,7 +2,7 @@ sources: inline_source: type: inline properties: - data: '[{"tcp_rtt_ms":128,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.ct.cn","http_url":"www.ct.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.11.22.22","server_ip":"8.8.8.8","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":5575,"sent_pkts":97,"sent_bytes":5892,"received_pkts":250,"received_bytes":333931},{"tcp_rtt_ms":256,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.abc.cn","http_url":"www.cabc.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.168.10.198","server_ip":"4.4.4.4","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":2575,"sent_pkts":197,"sent_bytes":5892,"received_pkts":350,"received_bytes":533931}]' + data: '[{"tcp_rtt_ms":128,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.ct.cn","http_url":"www.ct.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.11.22.22","server_ip":"8.8.8.8","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":5575,"sent_pkts":97,"sent_bytes":5892,"received_pkts":250,"received_bytes":333931},{"tcp_rtt_ms":256,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.abc.cn","http_url":"www.cabc.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.11.22.22","server_ip":"4.4.4.4","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":2575,"sent_pkts":197,"sent_bytes":5892,"received_pkts":350,"received_bytes":533931}]' format: json json.ignore.parse.errors: false @@ -18,6 +18,12 @@ processing_pipelines: - function: NUMBER_SUM lookup_fields: [ received_bytes] output_fields: [ received_bytes_sum ] + - function: APPROX_COUNT_DISTINCT_HLLD + lookup_fields: [ server_ip ] + output_fields: [ server_ip_count ] + parameters: + input_type: regular + sinks: print_sink: -- cgit v1.2.3