From 2ca86bc94a84f34cd5bc06e011a236e3b4bd8022 Mon Sep 17 00:00:00 2001 From: henryzhx8 Date: Thu, 7 Nov 2024 11:49:57 +0800 Subject: [PATCH 1/5] do not separate event group when group size excceeds min batch size to improve performance (#1850) --- core/models/EventPool.cpp | 30 ++-- core/models/PipelineEventGroup.cpp | 16 +- .../metric_constants/ComponentMetrics.cpp | 4 +- .../metric_constants/MetricConstants.h | 4 +- core/pipeline/batch/BatchItem.h | 8 +- core/pipeline/batch/BatchedEvents.cpp | 16 +- core/pipeline/batch/Batcher.h | 159 +++++++++++------- core/pipeline/batch/FlushStrategy.h | 34 ++-- core/pipeline/queue/BoundedProcessQueue.cpp | 5 + core/pipeline/queue/CircularProcessQueue.cpp | 5 + core/pipeline/queue/ProcessQueueInterface.cpp | 4 +- core/pipeline/queue/ProcessQueueInterface.h | 3 + core/pipeline/queue/SenderQueue.cpp | 8 +- core/pipeline/queue/SenderQueue.h | 4 +- core/pipeline/serializer/SLSSerializer.cpp | 2 +- core/plugin/flusher/sls/FlusherSLS.cpp | 29 ++-- core/plugin/flusher/sls/FlusherSLS.h | 1 - core/runner/FlusherRunner.cpp | 91 ++++------ core/runner/FlusherRunner.h | 6 +- core/runner/ProcessorRunner.cpp | 19 ++- core/runner/sink/http/HttpSink.cpp | 21 ++- core/unittest/batch/BatchItemUnittest.cpp | 10 +- core/unittest/batch/BatcherUnittest.cpp | 128 +++++++++----- core/unittest/batch/FlushStrategyUnittest.cpp | 17 +- core/unittest/flusher/FlusherSLSUnittest.cpp | 22 +-- .../InstanceConfigManagerUnittest.cpp | 9 +- 26 files changed, 374 insertions(+), 281 deletions(-) diff --git a/core/models/EventPool.cpp b/core/models/EventPool.cpp index 8941137c86..0a9035af71 100644 --- a/core/models/EventPool.cpp +++ b/core/models/EventPool.cpp @@ -41,9 +41,8 @@ EventPool::~EventPool() { } LogEvent* EventPool::AcquireLogEvent(PipelineEventGroup* ptr) { - TransferPoolIfEmpty(mLogEventPool, mLogEventPoolBak); - if (mEnableLock) { + TransferPoolIfEmpty(mLogEventPool, mLogEventPoolBak); lock_guard lock(mPoolMux); return AcquireEventNoLock(ptr, mLogEventPool, mMinUnusedLogEventsCnt); } @@ -51,9 +50,8 @@ LogEvent* EventPool::AcquireLogEvent(PipelineEventGroup* ptr) { } MetricEvent* EventPool::AcquireMetricEvent(PipelineEventGroup* ptr) { - TransferPoolIfEmpty(mMetricEventPool, mMetricEventPoolBak); - if (mEnableLock) { + TransferPoolIfEmpty(mMetricEventPool, mMetricEventPoolBak); lock_guard lock(mPoolMux); return AcquireEventNoLock(ptr, mMetricEventPool, mMinUnusedMetricEventsCnt); } @@ -61,9 +59,8 @@ MetricEvent* EventPool::AcquireMetricEvent(PipelineEventGroup* ptr) { } SpanEvent* EventPool::AcquireSpanEvent(PipelineEventGroup* ptr) { - TransferPoolIfEmpty(mSpanEventPool, mSpanEventPoolBak); - if (mEnableLock) { + TransferPoolIfEmpty(mSpanEventPool, mSpanEventPoolBak); lock_guard lock(mPoolMux); return AcquireEventNoLock(ptr, mSpanEventPool, mMinUnusedSpanEventsCnt); } @@ -98,20 +95,27 @@ void EventPool::Release(vector&& obj) { } template -void DoGC(vector& pool, vector& poolBak, size_t& minUnusedCnt, mutex* mux) { +void DoGC(vector& pool, vector& poolBak, size_t& minUnusedCnt, mutex* mux, const string& type) { if (minUnusedCnt <= pool.size() || minUnusedCnt == numeric_limits::max()) { auto sz = minUnusedCnt == numeric_limits::max() ? pool.size() : minUnusedCnt; for (size_t i = 0; i < sz; ++i) { delete pool.back(); pool.pop_back(); } + size_t bakSZ = 0; if (mux) { lock_guard lock(*mux); + bakSZ = poolBak.size(); for (auto& item : poolBak) { delete item; } poolBak.clear(); } + if (sz != 0 || bakSZ != 0) { + LOG_INFO( + sLogger, + ("event pool gc", "done")("event type", type)("gc event cnt", sz + bakSZ)("pool size", pool.size())); + } } else { LOG_ERROR(sLogger, ("unexpected error", "min unused event cnt is greater than pool size")( @@ -124,13 +128,13 @@ void EventPool::CheckGC() { if (time(nullptr) - mLastGCTime > INT32_FLAG(event_pool_gc_interval_secs)) { if (mEnableLock) { lock_guard lock(mPoolMux); - DoGC(mLogEventPool, mLogEventPoolBak, mMinUnusedLogEventsCnt, &mPoolBakMux); - DoGC(mMetricEventPool, mMetricEventPoolBak, mMinUnusedMetricEventsCnt, &mPoolBakMux); - DoGC(mSpanEventPool, mSpanEventPoolBak, mMinUnusedSpanEventsCnt, &mPoolBakMux); + DoGC(mLogEventPool, mLogEventPoolBak, mMinUnusedLogEventsCnt, &mPoolBakMux, "log"); + DoGC(mMetricEventPool, mMetricEventPoolBak, mMinUnusedMetricEventsCnt, &mPoolBakMux, "metric"); + DoGC(mSpanEventPool, mSpanEventPoolBak, mMinUnusedSpanEventsCnt, &mPoolBakMux, "span"); } else { - DoGC(mLogEventPool, mLogEventPoolBak, mMinUnusedLogEventsCnt, nullptr); - DoGC(mMetricEventPool, mMetricEventPoolBak, mMinUnusedMetricEventsCnt, nullptr); - DoGC(mSpanEventPool, mSpanEventPoolBak, mMinUnusedSpanEventsCnt, nullptr); + DoGC(mLogEventPool, mLogEventPoolBak, mMinUnusedLogEventsCnt, nullptr, "log"); + DoGC(mMetricEventPool, mMetricEventPoolBak, mMinUnusedMetricEventsCnt, nullptr, "metric"); + DoGC(mSpanEventPool, mSpanEventPoolBak, mMinUnusedSpanEventsCnt, nullptr, "span"); } mLastGCTime = time(nullptr); } diff --git a/core/models/PipelineEventGroup.cpp b/core/models/PipelineEventGroup.cpp index 16f194fd03..35ebd35a0d 100644 --- a/core/models/PipelineEventGroup.cpp +++ b/core/models/PipelineEventGroup.cpp @@ -34,10 +34,24 @@ namespace logtail { template void DestroyEvents(vector&& events) { unordered_map> eventsPoolMap; + // for most cases, all events have the same origin. So we cache the pool pointer and iterator for efficiency + EventPool* cachedPoolPtr = nullptr; + typename unordered_map>::iterator cachedIt; + bool firstEvent = true; for (auto& item : events) { if (item && item.IsFromEventPool()) { item->Reset(); - eventsPoolMap[item.GetEventPool()].emplace_back(static_cast(item.Release())); + if (firstEvent || item.GetEventPool() != cachedPoolPtr) { + cachedPoolPtr = item.GetEventPool(); + cachedIt = eventsPoolMap.find(cachedPoolPtr); + if (cachedIt == eventsPoolMap.end()) { + eventsPoolMap.emplace(cachedPoolPtr, vector()); + cachedIt = eventsPoolMap.find(cachedPoolPtr); + cachedIt->second.reserve(events.size()); + } + firstEvent = false; + } + cachedIt->second.emplace_back(static_cast(item.Release())); } } for (auto& item : eventsPoolMap) { diff --git a/core/monitor/metric_constants/ComponentMetrics.cpp b/core/monitor/metric_constants/ComponentMetrics.cpp index 006602e751..cbbb8ed40b 100644 --- a/core/monitor/metric_constants/ComponentMetrics.cpp +++ b/core/monitor/metric_constants/ComponentMetrics.cpp @@ -74,8 +74,8 @@ const string METRIC_COMPONENT_QUEUE_EXTRA_BUFFER_SIZE_BYTES = "component_extra_b const string METRIC_COMPONENT_QUEUE_DISCARDED_EVENTS_TOTAL = "component_discarded_events_total"; const string METRIC_COMPONENT_QUEUE_FETCHED_ITEMS_TOTAL = "component_fetched_items_total"; -const string METRIC_COMPONENT_QUEUE_FETCH_ATTEMPTS_TOTAL = "component_fetch_attempts_total"; -const string METRIC_COMPONENT_QUEUE_SUCCESSFUL_FETCH_TIMES_TOTAL = "component_successful_fetch_times_total"; +const string METRIC_COMPONENT_QUEUE_FETCH_TIMES_TOTAL = "component_fetch_times_total"; +const string METRIC_COMPONENT_QUEUE_VALID_FETCH_TIMES_TOTAL = "component_valid_fetch_times_total"; const string METRIC_COMPONENT_QUEUE_FETCH_REJECTED_BY_REGION_LIMITER_TIMES_TOTAL = "component_fetch_rejected_by_region_limiter_times_total"; const string METRIC_COMPONENT_QUEUE_FETCH_REJECTED_BY_PROJECT_LIMITER_TIMES_TOTAL diff --git a/core/monitor/metric_constants/MetricConstants.h b/core/monitor/metric_constants/MetricConstants.h index 988c8b4afa..84740a09b9 100644 --- a/core/monitor/metric_constants/MetricConstants.h +++ b/core/monitor/metric_constants/MetricConstants.h @@ -261,8 +261,8 @@ extern const std::string METRIC_COMPONENT_QUEUE_EXTRA_BUFFER_SIZE_BYTES; extern const std::string METRIC_COMPONENT_QUEUE_DISCARDED_EVENTS_TOTAL; extern const std::string METRIC_COMPONENT_QUEUE_FETCHED_ITEMS_TOTAL; -extern const std::string METRIC_COMPONENT_QUEUE_FETCH_ATTEMPTS_TOTAL; -extern const std::string METRIC_COMPONENT_QUEUE_SUCCESSFUL_FETCH_TIMES_TOTAL; +extern const std::string METRIC_COMPONENT_QUEUE_FETCH_TIMES_TOTAL; +extern const std::string METRIC_COMPONENT_QUEUE_VALID_FETCH_TIMES_TOTAL; extern const std::string METRIC_COMPONENT_QUEUE_FETCH_REJECTED_BY_REGION_LIMITER_TIMES_TOTAL; extern const std::string METRIC_COMPONENT_QUEUE_FETCH_REJECTED_BY_PROJECT_LIMITER_TIMES_TOTAL; extern const std::string METRIC_COMPONENT_QUEUE_FETCH_REJECTED_BY_LOGSTORE_LIMITER_TIMES_TOTAL; diff --git a/core/pipeline/batch/BatchItem.h b/core/pipeline/batch/BatchItem.h index 9fa956dbef..d76dda00c1 100644 --- a/core/pipeline/batch/BatchItem.h +++ b/core/pipeline/batch/BatchItem.h @@ -34,7 +34,7 @@ class GroupBatchItem { void Add(BatchedEvents&& g, int64_t totalEnqueTimeMs) { mEventsCnt += g.mEvents.size(); - mTotalEnqueTimeMs += totalEnqueTimeMs; + // mTotalEnqueTimeMs += totalEnqueTimeMs; mGroups.emplace_back(std::move(g)); mStatus.Update(mGroups.back()); } @@ -94,9 +94,9 @@ class EventBatchItem { void Add(PipelineEventPtr&& e) { mBatch.mEvents.emplace_back(std::move(e)); mStatus.Update(mBatch.mEvents.back()); - mTotalEnqueTimeMs += std::chrono::time_point_cast(std::chrono::system_clock::now()) - .time_since_epoch() - .count(); + // mTotalEnqueTimeMs += std::chrono::time_point_cast(std::chrono::system_clock::now()) + // .time_since_epoch() + // .count(); } void Flush(GroupBatchItem& res) { diff --git a/core/pipeline/batch/BatchedEvents.cpp b/core/pipeline/batch/BatchedEvents.cpp index 02ccc25905..43a63ea9e4 100644 --- a/core/pipeline/batch/BatchedEvents.cpp +++ b/core/pipeline/batch/BatchedEvents.cpp @@ -23,10 +23,24 @@ namespace logtail { template void DestroyEvents(vector&& events) { unordered_map> eventsPoolMap; + // for most cases, all events have the same origin. So we cache the pool pointer and iterator for efficiency + EventPool* cachedPoolPtr = nullptr; + typename unordered_map>::iterator cachedIt; + bool firstEvent = true; for (auto& item : events) { if (item && item.IsFromEventPool()) { item->Reset(); - eventsPoolMap[item.GetEventPool()].emplace_back(static_cast(item.Release())); + if (firstEvent || item.GetEventPool() != cachedPoolPtr) { + cachedPoolPtr = item.GetEventPool(); + cachedIt = eventsPoolMap.find(cachedPoolPtr); + if (cachedIt == eventsPoolMap.end()) { + eventsPoolMap.emplace(cachedPoolPtr, vector()); + cachedIt = eventsPoolMap.find(cachedPoolPtr); + cachedIt->second.reserve(events.size()); + } + firstEvent = false; + } + cachedIt->second.emplace_back(static_cast(item.Release())); } } for (auto& item : eventsPoolMap) { diff --git a/core/pipeline/batch/Batcher.h b/core/pipeline/batch/Batcher.h index 523e390d1f..aeca80b600 100644 --- a/core/pipeline/batch/Batcher.h +++ b/core/pipeline/batch/Batcher.h @@ -47,12 +47,12 @@ class Batcher { std::string errorMsg; PipelineContext& ctx = flusher->GetContext(); - uint32_t maxSizeBytes = strategy.mMaxSizeBytes; - if (!GetOptionalUIntParam(config, "MaxSizeBytes", maxSizeBytes, errorMsg)) { + uint32_t minSizeBytes = strategy.mMinSizeBytes; + if (!GetOptionalUIntParam(config, "MinSizeBytes", minSizeBytes, errorMsg)) { PARAM_WARNING_DEFAULT(ctx.GetLogger(), ctx.GetAlarm(), errorMsg, - maxSizeBytes, + minSizeBytes, flusher->Name(), ctx.GetConfigName(), ctx.GetProjectName(), @@ -60,12 +60,12 @@ class Batcher { ctx.GetRegion()); } - uint32_t maxCnt = strategy.mMaxCnt; - if (!GetOptionalUIntParam(config, "MaxCnt", maxCnt, errorMsg)) { + uint32_t minCnt = strategy.mMinCnt; + if (!GetOptionalUIntParam(config, "MinCnt", minCnt, errorMsg)) { PARAM_WARNING_DEFAULT(ctx.GetLogger(), ctx.GetAlarm(), errorMsg, - maxCnt, + minCnt, flusher->Name(), ctx.GetConfigName(), ctx.GetProjectName(), @@ -88,14 +88,15 @@ class Batcher { if (enableGroupBatch) { uint32_t groupTimeout = timeoutSecs / 2; - mGroupFlushStrategy = GroupFlushStrategy(maxSizeBytes, groupTimeout); + mGroupFlushStrategy = GroupFlushStrategy(minSizeBytes, groupTimeout); mGroupQueue = GroupBatchItem(); mEventFlushStrategy.SetTimeoutSecs(timeoutSecs - groupTimeout); } else { mEventFlushStrategy.SetTimeoutSecs(timeoutSecs); } - mEventFlushStrategy.SetMaxSizeBytes(maxSizeBytes); - mEventFlushStrategy.SetMaxCnt(maxCnt); + mEventFlushStrategy.SetMaxSizeBytes(strategy.mMaxSizeBytes); + mEventFlushStrategy.SetMinSizeBytes(minSizeBytes); + mEventFlushStrategy.SetMinCnt(minCnt); mFlusher = flusher; @@ -114,7 +115,7 @@ class Batcher { mInEventsTotal = mMetricsRecordRef.CreateCounter(METRIC_COMPONENT_IN_EVENTS_TOTAL); mInGroupDataSizeBytes = mMetricsRecordRef.CreateCounter(METRIC_COMPONENT_IN_SIZE_BYTES); mOutEventsTotal = mMetricsRecordRef.CreateCounter(METRIC_COMPONENT_OUT_EVENTS_TOTAL); - mTotalDelayMs = mMetricsRecordRef.CreateCounter(METRIC_COMPONENT_TOTAL_DELAY_MS); + // mTotalDelayMs = mMetricsRecordRef.CreateCounter(METRIC_COMPONENT_TOTAL_DELAY_MS); mEventBatchItemsTotal = mMetricsRecordRef.CreateIntGauge(METRIC_COMPONENT_BATCHER_EVENT_BATCHES_TOTAL); mBufferedGroupsTotal = mMetricsRecordRef.CreateIntGauge(METRIC_COMPONENT_BATCHER_BUFFERED_GROUPS_TOTAL); mBufferedEventsTotal = mMetricsRecordRef.CreateIntGauge(METRIC_COMPONENT_BATCHER_BUFFERED_EVENTS_TOTAL); @@ -134,51 +135,79 @@ class Batcher { mInGroupDataSizeBytes->Add(g.DataSize()); mEventBatchItemsTotal->Set(mEventQueueMap.size()); - size_t eventsSize = g.GetEvents().size(); - for (size_t i = 0; i < eventsSize; ++i) { - PipelineEventPtr& e = g.MutableEvents()[i]; - if (!item.IsEmpty() && mEventFlushStrategy.NeedFlushByTime(item.GetStatus(), e)) { - if (!mGroupQueue) { - UpdateMetricsOnFlushingEventQueue(item); + if (g.DataSize() > mEventFlushStrategy.GetMinSizeBytes()) { + // for group size larger than min batch size, separate group only if size is larger than max batch size + if (!item.IsEmpty()) { + UpdateMetricsOnFlushingEventQueue(item); + item.Flush(res); + } + for (auto& e : g.MutableEvents()) { + // should consider time condition here because sls require this + if (!item.IsEmpty() && mEventFlushStrategy.NeedFlushByTime(item.GetStatus(), e)) { + mOutEventsTotal->Add(item.EventSize()); + item.Flush(res); + } + if (item.IsEmpty()) { + item.Reset(g.GetSizedTags(), + g.GetSourceBuffer(), + g.GetExactlyOnceCheckpoint(), + g.GetMetadata(EventGroupMetaKey::SOURCE_ID)); + } + item.Add(std::move(e)); + if (mEventFlushStrategy.SizeReachingUpperLimit(item.GetStatus())) { + mOutEventsTotal->Add(item.EventSize()); item.Flush(res); - } else { - if (!mGroupQueue->IsEmpty() && mGroupFlushStrategy->NeedFlushByTime(mGroupQueue->GetStatus())) { - UpdateMetricsOnFlushingGroupQueue(); - mGroupQueue->Flush(res); - } - if (mGroupQueue->IsEmpty()) { - TimeoutFlushManager::GetInstance()->UpdateRecord(mFlusher->GetContext().GetConfigName(), - 0, - 0, - mGroupFlushStrategy->GetTimeoutSecs(), - mFlusher); - } - item.Flush(mGroupQueue.value()); - if (mGroupFlushStrategy->NeedFlushBySize(mGroupQueue->GetStatus())) { - UpdateMetricsOnFlushingGroupQueue(); - mGroupQueue->Flush(res); - } } } - if (item.IsEmpty()) { - item.Reset(g.GetSizedTags(), - g.GetSourceBuffer(), - g.GetExactlyOnceCheckpoint(), - g.GetMetadata(EventGroupMetaKey::SOURCE_ID)); - TimeoutFlushManager::GetInstance()->UpdateRecord( - mFlusher->GetContext().GetConfigName(), 0, key, mEventFlushStrategy.GetTimeoutSecs(), mFlusher); - mBufferedGroupsTotal->Add(1); - mBufferedDataSizeByte->Add(item.DataSize()); - } else if (i == 0) { - item.AddSourceBuffer(g.GetSourceBuffer()); - } - mBufferedEventsTotal->Add(1); - mBufferedDataSizeByte->Add(e->DataSize()); - item.Add(std::move(e)); - if (mEventFlushStrategy.NeedFlushBySize(item.GetStatus()) - || mEventFlushStrategy.NeedFlushByCnt(item.GetStatus())) { - UpdateMetricsOnFlushingEventQueue(item); - item.Flush(res); + mOutEventsTotal->Add(item.EventSize()); + item.Flush(res); + } else { + size_t eventsSize = g.GetEvents().size(); + for (size_t i = 0; i < eventsSize; ++i) { + PipelineEventPtr& e = g.MutableEvents()[i]; + if (!item.IsEmpty() && mEventFlushStrategy.NeedFlushByTime(item.GetStatus(), e)) { + if (!mGroupQueue) { + UpdateMetricsOnFlushingEventQueue(item); + item.Flush(res); + } else { + if (!mGroupQueue->IsEmpty() && mGroupFlushStrategy->NeedFlushByTime(mGroupQueue->GetStatus())) { + UpdateMetricsOnFlushingGroupQueue(); + mGroupQueue->Flush(res); + } + if (mGroupQueue->IsEmpty()) { + TimeoutFlushManager::GetInstance()->UpdateRecord(mFlusher->GetContext().GetConfigName(), + 0, + 0, + mGroupFlushStrategy->GetTimeoutSecs(), + mFlusher); + } + item.Flush(mGroupQueue.value()); + if (mGroupFlushStrategy->NeedFlushBySize(mGroupQueue->GetStatus())) { + UpdateMetricsOnFlushingGroupQueue(); + mGroupQueue->Flush(res); + } + } + } + if (item.IsEmpty()) { + item.Reset(g.GetSizedTags(), + g.GetSourceBuffer(), + g.GetExactlyOnceCheckpoint(), + g.GetMetadata(EventGroupMetaKey::SOURCE_ID)); + TimeoutFlushManager::GetInstance()->UpdateRecord( + mFlusher->GetContext().GetConfigName(), 0, key, mEventFlushStrategy.GetTimeoutSecs(), mFlusher); + mBufferedGroupsTotal->Add(1); + mBufferedDataSizeByte->Add(item.DataSize()); + } else if (i == 0) { + item.AddSourceBuffer(g.GetSourceBuffer()); + } + mBufferedEventsTotal->Add(1); + mBufferedDataSizeByte->Add(e->DataSize()); + item.Add(std::move(e)); + if (mEventFlushStrategy.NeedFlushBySize(item.GetStatus()) + || mEventFlushStrategy.NeedFlushByCnt(item.GetStatus())) { + UpdateMetricsOnFlushingEventQueue(item); + item.Flush(res); + } } } mTotalAddTimeMs->Add(std::chrono::system_clock::now() - before); @@ -260,12 +289,12 @@ class Batcher { private: void UpdateMetricsOnFlushingEventQueue(const EventBatchItem& item) { mOutEventsTotal->Add(item.EventSize()); - mTotalDelayMs->Add( - item.EventSize() - * std::chrono::time_point_cast(std::chrono::system_clock::now()) - .time_since_epoch() - .count() - - item.TotalEnqueTimeMs()); + // mTotalDelayMs->Add( + // item.EventSize() + // * std::chrono::time_point_cast(std::chrono::system_clock::now()) + // .time_since_epoch() + // .count() + // - item.TotalEnqueTimeMs()); mBufferedGroupsTotal->Sub(1); mBufferedEventsTotal->Sub(item.EventSize()); mBufferedDataSizeByte->Sub(item.DataSize()); @@ -273,12 +302,12 @@ class Batcher { void UpdateMetricsOnFlushingGroupQueue() { mOutEventsTotal->Add(mGroupQueue->EventSize()); - mTotalDelayMs->Add( - mGroupQueue->EventSize() - * std::chrono::time_point_cast(std::chrono::system_clock::now()) - .time_since_epoch() - .count() - - mGroupQueue->TotalEnqueTimeMs()); + // mTotalDelayMs->Add( + // mGroupQueue->EventSize() + // * std::chrono::time_point_cast(std::chrono::system_clock::now()) + // .time_since_epoch() + // .count() + // - mGroupQueue->TotalEnqueTimeMs()); mBufferedGroupsTotal->Sub(mGroupQueue->GroupSize()); mBufferedEventsTotal->Sub(mGroupQueue->EventSize()); mBufferedDataSizeByte->Sub(mGroupQueue->DataSize()); @@ -297,7 +326,7 @@ class Batcher { CounterPtr mInEventsTotal; CounterPtr mInGroupDataSizeBytes; CounterPtr mOutEventsTotal; - CounterPtr mTotalDelayMs; + // CounterPtr mTotalDelayMs; IntGaugePtr mEventBatchItemsTotal; IntGaugePtr mBufferedGroupsTotal; IntGaugePtr mBufferedEventsTotal; diff --git a/core/pipeline/batch/FlushStrategy.h b/core/pipeline/batch/FlushStrategy.h index d248cc7f10..e2a718244e 100644 --- a/core/pipeline/batch/FlushStrategy.h +++ b/core/pipeline/batch/FlushStrategy.h @@ -20,15 +20,17 @@ #include #include +#include -#include "pipeline/batch/BatchStatus.h" #include "models/PipelineEventPtr.h" +#include "pipeline/batch/BatchStatus.h" namespace logtail { struct DefaultFlushStrategyOptions { - uint32_t mMaxSizeBytes = 0; - uint32_t mMaxCnt = 0; + uint32_t mMaxSizeBytes = std::numeric_limits::max(); + uint32_t mMinSizeBytes = 0; + uint32_t mMinCnt = 0; uint32_t mTimeoutSecs = 0; }; @@ -36,49 +38,53 @@ template class EventFlushStrategy { public: void SetMaxSizeBytes(uint32_t size) { mMaxSizeBytes = size; } - void SetMaxCnt(uint32_t cnt) { mMaxCnt = cnt; } + void SetMinSizeBytes(uint32_t size) { mMinSizeBytes = size; } + void SetMinCnt(uint32_t cnt) { mMinCnt = cnt; } void SetTimeoutSecs(uint32_t secs) { mTimeoutSecs = secs; } uint32_t GetMaxSizeBytes() const { return mMaxSizeBytes; } - uint32_t GetMaxCnt() const { return mMaxCnt; } + uint32_t GetMinSizeBytes() const { return mMinSizeBytes; } + uint32_t GetMinCnt() const { return mMinCnt; } uint32_t GetTimeoutSecs() const { return mTimeoutSecs; } // should be called after event is added - bool NeedFlushBySize(const T& status) { return status.GetSize() >= mMaxSizeBytes; } - bool NeedFlushByCnt(const T& status) { return status.GetCnt() == mMaxCnt; } + bool NeedFlushBySize(const T& status) { return status.GetSize() >= mMinSizeBytes; } + bool NeedFlushByCnt(const T& status) { return status.GetCnt() == mMinCnt; } // should be called before event is added bool NeedFlushByTime(const T& status, const PipelineEventPtr& e) { return time(nullptr) - status.GetCreateTime() >= mTimeoutSecs; } + bool SizeReachingUpperLimit(const T& status) { return status.GetSize() >= mMaxSizeBytes; } private: uint32_t mMaxSizeBytes = 0; - uint32_t mMaxCnt = 0; + uint32_t mMinSizeBytes = 0; + uint32_t mMinCnt = 0; uint32_t mTimeoutSecs = 0; }; class GroupFlushStrategy { public: - GroupFlushStrategy(uint32_t size, uint32_t timeout) : mMaxSizeBytes(size), mTimeoutSecs(timeout) {} + GroupFlushStrategy(uint32_t size, uint32_t timeout) : mMinSizeBytes(size), mTimeoutSecs(timeout) {} - void SetMaxSizeBytes(uint32_t size) { mMaxSizeBytes = size; } + void SetMinSizeBytes(uint32_t size) { mMinSizeBytes = size; } void SetTimeoutSecs(uint32_t secs) { mTimeoutSecs = secs; } - uint32_t GetMaxSizeBytes() const { return mMaxSizeBytes; } + uint32_t GetMinSizeBytes() const { return mMinSizeBytes; } uint32_t GetTimeoutSecs() const { return mTimeoutSecs; } // should be called after event is added - bool NeedFlushBySize(const GroupBatchStatus& status) { return status.GetSize() >= mMaxSizeBytes; } + bool NeedFlushBySize(const GroupBatchStatus& status) { return status.GetSize() >= mMinSizeBytes; } // should be called before event is added bool NeedFlushByTime(const GroupBatchStatus& status) { return time(nullptr) - status.GetCreateTime() >= mTimeoutSecs; } private: - uint32_t mMaxSizeBytes = 0; + uint32_t mMinSizeBytes = 0; uint32_t mTimeoutSecs = 0; }; template <> bool EventFlushStrategy::NeedFlushByTime(const SLSEventBatchStatus& status, - const PipelineEventPtr& e); + const PipelineEventPtr& e); } // namespace logtail diff --git a/core/pipeline/queue/BoundedProcessQueue.cpp b/core/pipeline/queue/BoundedProcessQueue.cpp index 170e4444f0..ee7532f9b2 100644 --- a/core/pipeline/queue/BoundedProcessQueue.cpp +++ b/core/pipeline/queue/BoundedProcessQueue.cpp @@ -49,6 +49,11 @@ bool BoundedProcessQueue::Push(unique_ptr&& item) { } bool BoundedProcessQueue::Pop(unique_ptr& item) { + mFetchTimesCnt->Add(1); + if (Empty()) { + return false; + } + mValidFetchTimesCnt->Add(1); if (!IsValidToPop()) { return false; } diff --git a/core/pipeline/queue/CircularProcessQueue.cpp b/core/pipeline/queue/CircularProcessQueue.cpp index 992fade5f7..6c4f57ba74 100644 --- a/core/pipeline/queue/CircularProcessQueue.cpp +++ b/core/pipeline/queue/CircularProcessQueue.cpp @@ -56,6 +56,11 @@ bool CircularProcessQueue::Push(unique_ptr&& item) { } bool CircularProcessQueue::Pop(unique_ptr& item) { + mFetchTimesCnt->Add(1); + if (Empty()) { + return false; + } + mValidFetchTimesCnt->Add(1); if (!IsValidToPop()) { return false; } diff --git a/core/pipeline/queue/ProcessQueueInterface.cpp b/core/pipeline/queue/ProcessQueueInterface.cpp index 3f28864625..01acae6d69 100644 --- a/core/pipeline/queue/ProcessQueueInterface.cpp +++ b/core/pipeline/queue/ProcessQueueInterface.cpp @@ -23,6 +23,8 @@ namespace logtail { ProcessQueueInterface::ProcessQueueInterface(int64_t key, size_t cap, uint32_t priority, const PipelineContext& ctx) : QueueInterface(key, cap, ctx), mPriority(priority), mConfigName(ctx.GetConfigName()) { mMetricsRecordRef.AddLabels({{METRIC_LABEL_KEY_COMPONENT_NAME, METRIC_LABEL_VALUE_COMPONENT_NAME_PROCESS_QUEUE}}); + mFetchTimesCnt = mMetricsRecordRef.CreateCounter(METRIC_COMPONENT_QUEUE_FETCH_TIMES_TOTAL); + mValidFetchTimesCnt = mMetricsRecordRef.CreateCounter(METRIC_COMPONENT_QUEUE_VALID_FETCH_TIMES_TOTAL); } void ProcessQueueInterface::SetDownStreamQueues(vector&& ques) { @@ -37,7 +39,7 @@ void ProcessQueueInterface::SetDownStreamQueues(vectorCommitMetricsRecordRef(mMetricsRecordRef); } @@ -111,7 +111,7 @@ bool SenderQueue::Remove(SenderQueueItem* item) { } void SenderQueue::GetAvailableItems(vector& items, int32_t limit) { - mFetchAttemptsCnt->Add(1); + mFetchTimesCnt->Add(1); if (Empty()) { return; } @@ -173,7 +173,7 @@ void SenderQueue::GetAvailableItems(vector& items, int32_t lim } } if (hasAvailableItem) { - mSuccessfulFetchTimesCnt->Add(1); + mValidFetchTimesCnt->Add(1); } } diff --git a/core/pipeline/queue/SenderQueue.h b/core/pipeline/queue/SenderQueue.h index 0e18bdb700..4c3a714d09 100644 --- a/core/pipeline/queue/SenderQueue.h +++ b/core/pipeline/queue/SenderQueue.h @@ -47,8 +47,8 @@ class SenderQueue : public BoundedSenderQueueInterface { size_t mRead = 0; size_t mSize = 0; - CounterPtr mSuccessfulFetchTimesCnt; - CounterPtr mFetchAttemptsCnt; + CounterPtr mFetchTimesCnt; + CounterPtr mValidFetchTimesCnt; CounterPtr mFetchedItemsCnt; #ifdef APSARA_UNIT_TEST_MAIN diff --git a/core/pipeline/serializer/SLSSerializer.cpp b/core/pipeline/serializer/SLSSerializer.cpp index 3f48dc961a..a6b8d7b803 100644 --- a/core/pipeline/serializer/SLSSerializer.cpp +++ b/core/pipeline/serializer/SLSSerializer.cpp @@ -19,7 +19,7 @@ #include "plugin/flusher/sls/FlusherSLS.h" #include "protobuf/sls/LogGroupSerializer.h" -DEFINE_FLAG_INT32(max_send_log_group_size, "bytes", 10 * 1024 * 1024); +DECLARE_FLAG_INT32(max_send_log_group_size); using namespace std; diff --git a/core/plugin/flusher/sls/FlusherSLS.cpp b/core/plugin/flusher/sls/FlusherSLS.cpp index 9f5b4cd08b..b0fd4d629a 100644 --- a/core/plugin/flusher/sls/FlusherSLS.cpp +++ b/core/plugin/flusher/sls/FlusherSLS.cpp @@ -59,6 +59,8 @@ DEFINE_FLAG_INT32(profile_data_send_retrytimes, "how many times should retry if DEFINE_FLAG_INT32(unknow_error_try_max, "discard data when try times > this value", 5); DEFINE_FLAG_BOOL(global_network_success, "global network success flag, default false", false); DEFINE_FLAG_BOOL(enable_metricstore_channel, "only works for metrics data for enhance metrics query performance", true); +DEFINE_FLAG_INT32(max_send_log_group_size, "bytes", 10 * 1024 * 1024); +DEFINE_FLAG_INT32(sls_serialize_size_expansion_ratio, "", 1.15); DECLARE_FLAG_BOOL(send_prefer_real_ip); @@ -374,8 +376,10 @@ bool FlusherSLS::Init(const Json::Value& config, Json::Value& optionalGoPipeline #endif mEndpoint = TrimString(mEndpoint); if (!mEndpoint.empty()) { - SLSClientManager::GetInstance()->AddEndpointEntry( - mRegion, StandardizeEndpoint(mEndpoint, mEndpoint), false, SLSClientManager::EndpointSourceType::LOCAL); + SLSClientManager::GetInstance()->AddEndpointEntry(mRegion, + StandardizeEndpoint(mEndpoint, mEndpoint), + false, + SLSClientManager::EndpointSourceType::LOCAL); } } #ifdef __ENTERPRISE__ @@ -472,9 +476,11 @@ bool FlusherSLS::Init(const Json::Value& config, Json::Value& optionalGoPipeline mContext->GetRegion()); } - DefaultFlushStrategyOptions strategy{static_cast(INT32_FLAG(batch_send_metric_size)), - static_cast(INT32_FLAG(merge_log_count_limit)), - static_cast(INT32_FLAG(batch_send_interval))}; + DefaultFlushStrategyOptions strategy{ + static_cast(INT32_FLAG(max_send_log_group_size) / INT32_FLAG(sls_serialize_size_expansion_ratio)), + static_cast(INT32_FLAG(batch_send_metric_size)), + static_cast(INT32_FLAG(merge_log_count_limit)), + static_cast(INT32_FLAG(batch_send_interval))}; if (!mBatcher.Init( itr ? *itr : Json::Value(), this, strategy, !mContext->IsExactlyOnceEnabled() && mShardHashKeys.empty())) { // when either exactly once is enabled or ShardHashKeys is not empty, we don't enable group batch @@ -516,19 +522,6 @@ bool FlusherSLS::Init(const Json::Value& config, Json::Value& optionalGoPipeline mMaxSendRate); } - // (Deprecated) FlowControlExpireTime - if (!GetOptionalUIntParam(config, "FlowControlExpireTime", mFlowControlExpireTime, errorMsg)) { - PARAM_WARNING_DEFAULT(mContext->GetLogger(), - mContext->GetAlarm(), - errorMsg, - mFlowControlExpireTime, - sName, - mContext->GetConfigName(), - mContext->GetProjectName(), - mContext->GetLogstoreName(), - mContext->GetRegion()); - } - GenerateGoPlugin(config, optionalGoPipeline); mSendCnt = GetMetricsRecordRef().CreateCounter(METRIC_PLUGIN_FLUSHER_OUT_EVENT_GROUPS_TOTAL); diff --git a/core/plugin/flusher/sls/FlusherSLS.h b/core/plugin/flusher/sls/FlusherSLS.h index 9fb043d9cf..42446b3813 100644 --- a/core/plugin/flusher/sls/FlusherSLS.h +++ b/core/plugin/flusher/sls/FlusherSLS.h @@ -81,7 +81,6 @@ class FlusherSLS : public HttpFlusher { sls_logs::SlsTelemetryType mTelemetryType = sls_logs::SlsTelemetryType::SLS_TELEMETRY_TYPE_LOGS; std::vector mShardHashKeys; uint32_t mMaxSendRate = 0; // preserved only for exactly once - uint32_t mFlowControlExpireTime = 0; // TODO: temporarily public for profile std::unique_ptr mCompressor; diff --git a/core/runner/FlusherRunner.cpp b/core/runner/FlusherRunner.cpp index ae1d55dc1a..095241c6ef 100644 --- a/core/runner/FlusherRunner.cpp +++ b/core/runner/FlusherRunner.cpp @@ -31,19 +31,18 @@ #include "plugin/flusher/sls/PackIdManager.h" #include "plugin/flusher/sls/SLSClientManager.h" -using namespace std; - -DEFINE_FLAG_INT32(check_send_client_timeout_interval, "", 600); -DEFINE_FLAG_BOOL(enable_flow_control, "if enable flow control", true); -DEFINE_FLAG_BOOL(enable_send_tps_smoothing, "avoid web server load burst", true); DEFINE_FLAG_INT32(flusher_runner_exit_timeout_secs, "", 60); +DEFINE_FLAG_INT32(check_send_client_timeout_interval, "", 600); -static const int SEND_BLOCK_COST_TIME_ALARM_INTERVAL_SECOND = 3; +using namespace std; namespace logtail { bool FlusherRunner::Init() { - srand(time(nullptr)); + LoadModuleConfig(true); + mCallback = [this]() { return LoadModuleConfig(false); }; + AppConfig::GetInstance()->RegisterCallback("max_bytes_per_sec", &mCallback); + WriteMetrics::GetInstance()->PrepareMetricsRecordRef( mMetricsRecordRef, {{METRIC_LABEL_KEY_RUNNER_NAME, METRIC_LABEL_VALUE_RUNNER_NAME_FLUSHER}, @@ -58,9 +57,7 @@ bool FlusherRunner::Init() { mThreadRes = async(launch::async, &FlusherRunner::Run, this); mLastCheckSendClientTime = time(nullptr); - LoadModuleConfig(true); - mCallback = [this]() { return LoadModuleConfig(false); }; - AppConfig::GetInstance()->RegisterCallback("max_bytes_per_sec", &mCallback); + return true; } @@ -87,18 +84,11 @@ bool FlusherRunner::LoadModuleConfig(bool isInit) { void FlusherRunner::UpdateSendFlowControl() { // when inflow exceed 30MB/s, FlowControl lose precision if (AppConfig::GetInstance()->GetMaxBytePerSec() >= 30 * 1024 * 1024) { - if (mSendFlowControl) - mSendFlowControl = false; - if (mSendRandomSleep) - mSendRandomSleep = false; - } else { - mSendRandomSleep = BOOL_FLAG(enable_send_tps_smoothing); - mSendFlowControl = BOOL_FLAG(enable_flow_control); + mEnableRateLimiter = false; } LOG_INFO(sLogger, ("send byte per second limit", AppConfig::GetInstance()->GetMaxBytePerSec())( - "send flow control", mSendFlowControl ? "enable" : "disable")( - "send random sleep", mSendRandomSleep ? "enable" : "disable")); + "send flow control", mEnableRateLimiter ? "enable" : "disable")); } void FlusherRunner::Stop() { @@ -118,29 +108,10 @@ void FlusherRunner::DecreaseHttpSendingCnt() { } void FlusherRunner::PushToHttpSink(SenderQueueItem* item, bool withLimit) { - if (!BOOL_FLAG(enable_full_drain_mode) && item->mFlusher->Name() == "flusher_sls" - && Application::GetInstance()->IsExiting()) { - DiskBufferWriter::GetInstance()->PushToDiskBuffer(item, 3); - SenderQueueManager::GetInstance()->RemoveItem(item->mFlusher->GetQueueKey(), item); - return; - } - - int32_t beforeSleepTime = time(NULL); + // TODO: use semaphore instead while (withLimit && !Application::GetInstance()->IsExiting() && GetSendingBufferCount() >= AppConfig::GetInstance()->GetSendRequestConcurrency()) { - usleep(10 * 1000); - } - int32_t afterSleepTime = time(NULL); - int32_t blockCostTime = afterSleepTime - beforeSleepTime; - if (blockCostTime > SEND_BLOCK_COST_TIME_ALARM_INTERVAL_SECOND) { - LOG_WARNING(sLogger, - ("sending log group blocked too long because send concurrency reached limit. current " - "concurrency used", - GetSendingBufferCount())("max concurrency", AppConfig::GetInstance()->GetSendRequestConcurrency())( - "blocked time", blockCostTime)); - LogtailAlarm::GetInstance()->SendAlarm(SENDING_COSTS_TOO_MUCH_TIME_ALARM, - "sending log group blocked for too much time, cost " - + ToString(blockCostTime)); + this_thread::sleep_for(chrono::milliseconds(10)); } auto req = static_cast(item->mFlusher)->BuildRequest(item); @@ -173,30 +144,19 @@ void FlusherRunner::Run() { } mInItemsTotal->Add(items.size()); mWaitingItemsTotal->Add(items.size()); - - // smoothing send tps, walk around webserver load burst - uint32_t bufferPackageCount = items.size(); - if (!Application::GetInstance()->IsExiting() && mSendRandomSleep) { - int64_t sleepMicroseconds = 0; - if (bufferPackageCount < 20) - sleepMicroseconds = (rand() % 30) * 10000; // 0ms ~ 300ms - else if (bufferPackageCount < 30) - sleepMicroseconds = (rand() % 20) * 10000; // 0ms ~ 200ms - else if (bufferPackageCount < 40) - sleepMicroseconds = (rand() % 10) * 10000; // 0ms ~ 100ms - if (sleepMicroseconds > 0) - usleep(sleepMicroseconds); - } } for (auto itr = items.begin(); itr != items.end(); ++itr) { - auto waitTime = chrono::duration_cast(curTime - (*itr)->mFirstEnqueTime); - LOG_DEBUG(sLogger, - ("got item from sender queue, item address", - *itr)("config-flusher-dst", QueueKeyManager::GetInstance()->GetName((*itr)->mQueueKey))( - "wait time", ToString(waitTime.count()) + "ms")("try cnt", ToString((*itr)->mTryCnt))); - - if (!Application::GetInstance()->IsExiting() && mSendFlowControl) { + LOG_DEBUG( + sLogger, + ("got item from sender queue, item address", + *itr)("config-flusher-dst", QueueKeyManager::GetInstance()->GetName((*itr)->mQueueKey))( + "wait time", + ToString(chrono::duration_cast(curTime - (*itr)->mFirstEnqueTime).count()) + + "ms")("try cnt", ToString((*itr)->mTryCnt))); + + // TODO: use rate limiter instead + if (!Application::GetInstance()->IsExiting() && mEnableRateLimiter) { RateLimiter::FlowControl((*itr)->mRawSize, mSendLastTime, mSendLastByte, true); } @@ -222,7 +182,14 @@ void FlusherRunner::Run() { void FlusherRunner::Dispatch(SenderQueueItem* item) { switch (item->mFlusher->GetSinkType()) { case SinkType::HTTP: - PushToHttpSink(item); + // TODO: make it common for all http flushers + if (!BOOL_FLAG(enable_full_drain_mode) && Application::GetInstance()->IsExiting() + && item->mFlusher->Name() == "flusher_sls") { + DiskBufferWriter::GetInstance()->PushToDiskBuffer(item, 3); + SenderQueueManager::GetInstance()->RemoveItem(item->mFlusher->GetQueueKey(), item); + } else { + PushToHttpSink(item); + } break; default: SenderQueueManager::GetInstance()->RemoveItem(item->mFlusher->GetQueueKey(), item); diff --git a/core/runner/FlusherRunner.h b/core/runner/FlusherRunner.h index 4c25d1e350..e8fbf23e0f 100644 --- a/core/runner/FlusherRunner.h +++ b/core/runner/FlusherRunner.h @@ -47,14 +47,13 @@ class FlusherRunner { int32_t GetSendingBufferCount() { return mHttpSendingCnt.load(); } - bool LoadModuleConfig(bool isInit); - private: FlusherRunner() = default; ~FlusherRunner() = default; void Run(); void Dispatch(SenderQueueItem* item); + bool LoadModuleConfig(bool isInit); void UpdateSendFlowControl(); std::function mCallback; @@ -69,8 +68,7 @@ class FlusherRunner { int64_t mSendLastTime = 0; int32_t mSendLastByte = 0; - bool mSendRandomSleep; - bool mSendFlowControl; + bool mEnableRateLimiter = true; mutable MetricsRecordRef mMetricsRecordRef; CounterPtr mInItemsTotal; diff --git a/core/runner/ProcessorRunner.cpp b/core/runner/ProcessorRunner.cpp index 7b4177ade9..98b604747a 100644 --- a/core/runner/ProcessorRunner.cpp +++ b/core/runner/ProcessorRunner.cpp @@ -27,13 +27,13 @@ #include "queue/ProcessQueueManager.h" #include "queue/QueueKeyManager.h" +DEFINE_FLAG_INT32(default_flush_merged_buffer_interval, "default flush merged buffer, seconds", 1); +DEFINE_FLAG_INT32(processor_runner_exit_timeout_secs, "", 60); + DECLARE_FLAG_INT32(max_send_log_group_size); using namespace std; -DEFINE_FLAG_INT32(default_flush_merged_buffer_interval, "default flush merged buffer, seconds", 1); -DEFINE_FLAG_INT32(processor_runner_exit_timeout_secs, "", 60); - namespace logtail { thread_local MetricsRecordRef ProcessorRunner::sMetricsRecordRef; @@ -84,7 +84,7 @@ bool ProcessorRunner::PushQueue(QueueKey key, size_t inputIndex, PipelineEventGr } void ProcessorRunner::Run(uint32_t threadNo) { - LOG_INFO(sLogger, ("processor runner", "started")("threadNo", threadNo)); + LOG_INFO(sLogger, ("processor runner", "started")("thread no", threadNo)); // thread local metrics should be initialized in each thread WriteMetrics::GetInstance()->PrepareMetricsRecordRef( @@ -97,12 +97,12 @@ void ProcessorRunner::Run(uint32_t threadNo) { sInGroupDataSizeBytes = sMetricsRecordRef.CreateCounter(METRIC_RUNNER_IN_SIZE_BYTES); sLastRunTime = sMetricsRecordRef.CreateIntGauge(METRIC_RUNNER_LAST_RUN_TIME); - static int32_t lastMergeTime = 0; + static int32_t lastFlushBatchTime = 0; while (true) { - int32_t curTime = time(NULL); - if (threadNo == 0 && curTime - lastMergeTime >= INT32_FLAG(default_flush_merged_buffer_interval)) { + int32_t curTime = time(nullptr); + if (threadNo == 0 && curTime - lastFlushBatchTime >= INT32_FLAG(default_flush_merged_buffer_interval)) { TimeoutFlushManager::GetInstance()->FlushTimeoutBatch(); - lastMergeTime = curTime; + lastFlushBatchTime = curTime; } sLastRunTime->Set(curTime); @@ -138,6 +138,9 @@ void ProcessorRunner::Run(uint32_t threadNo) { pipeline->Process(eventGroupList, item->mInputIndex); if (pipeline->IsFlushingThroughGoPipeline()) { + // TODO: + // 1. allow all event types to be sent to Go pipelines + // 2. use event group protobuf instead if (isLog) { for (auto& group : eventGroupList) { string res, errorMsg; diff --git a/core/runner/sink/http/HttpSink.cpp b/core/runner/sink/http/HttpSink.cpp index d1670ffeb9..2969ee5bf9 100644 --- a/core/runner/sink/http/HttpSink.cpp +++ b/core/runner/sink/http/HttpSink.cpp @@ -168,7 +168,8 @@ void HttpSink::DoRun() { HandleCompletedRequests(runningHandlers); unique_ptr request; - if (mQueue.TryPop(request)) { + bool hasRequest = false; + while (mQueue.TryPop(request)) { mInItemsTotal->Add(1); LOG_DEBUG(sLogger, ("got item from flusher runner, item address", request->mItem)( @@ -180,8 +181,12 @@ void HttpSink::DoRun() { if (AddRequestToClient(std::move(request))) { ++runningHandlers; mSendingItemsTotal->Add(1); + hasRequest = true; } } + if (hasRequest) { + continue; + } struct timeval timeout { 1, 0 @@ -272,13 +277,13 @@ void HttpSink::HandleCompletedRequests(int& runningHandlers) { static_cast(request->mItem->mFlusher) ->OnSendDone(request->mResponse, request->mItem); FlusherRunner::GetInstance()->DecreaseHttpSendingCnt(); - LOG_DEBUG( - sLogger, - ("failed to send http request", "abort")("item address", request->mItem)( - "config-flusher-dst", - QueueKeyManager::GetInstance()->GetName(request->mItem->mQueueKey))( - "response time", ToString(responseTimeMs) + "ms")("try cnt", ToString(request->mTryCnt))( - "sending cnt", ToString(FlusherRunner::GetInstance()->GetSendingBufferCount()))); + LOG_DEBUG(sLogger, + ("failed to send http request", "abort")("item address", request->mItem)( + "config-flusher-dst", + QueueKeyManager::GetInstance()->GetName(request->mItem->mQueueKey))( + "response time", ToString(responseTimeMs) + "ms")("try cnt", + ToString(request->mTryCnt))( + "sending cnt", ToString(FlusherRunner::GetInstance()->GetSendingBufferCount()))); } mOutFailedItemsTotal->Add(1); mFailedItemTotalResponseTimeMs->Add(responseTime); diff --git a/core/unittest/batch/BatchItemUnittest.cpp b/core/unittest/batch/BatchItemUnittest.cpp index 0aac5d30b3..b60c0e356f 100644 --- a/core/unittest/batch/BatchItemUnittest.cpp +++ b/core/unittest/batch/BatchItemUnittest.cpp @@ -92,7 +92,7 @@ void EventBatchItemUnittest::TestAdd() { APSARA_TEST_EQUAL(1U, mItem.mBatch.mEvents.size()); APSARA_TEST_EQUAL(1U, mItem.GetStatus().GetCnt()); APSARA_TEST_EQUAL(size, mItem.GetStatus().GetSize()); - APSARA_TEST_NOT_EQUAL(0, mItem.mTotalEnqueTimeMs); + // APSARA_TEST_NOT_EQUAL(0, mItem.mTotalEnqueTimeMs); } void EventBatchItemUnittest::TestFlushEmpty() { @@ -133,7 +133,7 @@ void EventBatchItemUnittest::TestFlushGroupBatchItem() { APSARA_TEST_EQUAL(0U, mItem.GetStatus().GetCnt()); APSARA_TEST_EQUAL(0U, mItem.GetStatus().GetSize()); APSARA_TEST_EQUAL(0, mItem.GetStatus().GetCreateTime()); - APSARA_TEST_EQUAL(0, mItem.mTotalEnqueTimeMs); + // APSARA_TEST_EQUAL(0, mItem.mTotalEnqueTimeMs); } void EventBatchItemUnittest::TestFlushBatchedEvensList() { @@ -161,7 +161,7 @@ void EventBatchItemUnittest::TestFlushBatchedEvensList() { APSARA_TEST_EQUAL(0U, mItem.GetStatus().GetCnt()); APSARA_TEST_EQUAL(0U, mItem.GetStatus().GetSize()); APSARA_TEST_EQUAL(0, mItem.GetStatus().GetCreateTime()); - APSARA_TEST_EQUAL(0, mItem.mTotalEnqueTimeMs); + // APSARA_TEST_EQUAL(0, mItem.mTotalEnqueTimeMs); } void EventBatchItemUnittest::TestFlushBatchedEvensLists() { @@ -190,7 +190,7 @@ void EventBatchItemUnittest::TestFlushBatchedEvensLists() { APSARA_TEST_EQUAL(0U, mItem.GetStatus().GetCnt()); APSARA_TEST_EQUAL(0U, mItem.GetStatus().GetSize()); APSARA_TEST_EQUAL(0, mItem.GetStatus().GetCreateTime()); - APSARA_TEST_EQUAL(0, mItem.mTotalEnqueTimeMs); + // APSARA_TEST_EQUAL(0, mItem.mTotalEnqueTimeMs); } void EventBatchItemUnittest::TestExactlyOnce() { @@ -252,7 +252,7 @@ void GroupBatchItemUnittest::TestAdd() { APSARA_TEST_EQUAL(1U, mItem.mGroups.size()); APSARA_TEST_EQUAL(size, mItem.GetStatus().GetSize()); - APSARA_TEST_EQUAL(1234567890000, mItem.TotalEnqueTimeMs()); + // APSARA_TEST_EQUAL(1234567890000, mItem.TotalEnqueTimeMs()); APSARA_TEST_EQUAL(1U, mItem.EventSize()); APSARA_TEST_EQUAL(1U, mItem.GroupSize()); APSARA_TEST_EQUAL(100U, mItem.DataSize()); diff --git a/core/unittest/batch/BatcherUnittest.cpp b/core/unittest/batch/BatcherUnittest.cpp index d75b2ed9ec..5e105d6f90 100644 --- a/core/unittest/batch/BatcherUnittest.cpp +++ b/core/unittest/batch/BatcherUnittest.cpp @@ -28,6 +28,7 @@ class BatcherUnittest : public ::testing::Test { void TestInitWithGroupBatch(); void TestAddWithoutGroupBatch(); void TestAddWithGroupBatch(); + void TestAddWithOversizedGroup(); void TestFlushEventQueueWithoutGroupBatch(); void TestFlushEventQueueWithGroupBatch(); void TestFlushGroupQueue(); @@ -59,16 +60,18 @@ unique_ptr BatcherUnittest::sFlusher; void BatcherUnittest::TestParamInit() { DefaultFlushStrategyOptions strategy; - strategy.mMaxCnt = 1; - strategy.mMaxSizeBytes = 100; + strategy.mMinCnt = 1; + strategy.mMaxSizeBytes = 300; + strategy.mMinSizeBytes = 100; strategy.mTimeoutSecs = 3; { // empty config Batcher<> batch; batch.Init(Json::Value(), sFlusher.get(), strategy); - APSARA_TEST_EQUAL(1U, batch.mEventFlushStrategy.GetMaxCnt()); - APSARA_TEST_EQUAL(100U, batch.mEventFlushStrategy.GetMaxSizeBytes()); + APSARA_TEST_EQUAL(1U, batch.mEventFlushStrategy.GetMinCnt()); + APSARA_TEST_EQUAL(100U, batch.mEventFlushStrategy.GetMinSizeBytes()); APSARA_TEST_EQUAL(3U, batch.mEventFlushStrategy.GetTimeoutSecs()); + APSARA_TEST_EQUAL(300U, batch.mEventFlushStrategy.GetMaxSizeBytes()); } { // invalid param @@ -76,8 +79,8 @@ void BatcherUnittest::TestParamInit() { string configStr, errorMsg; configStr = R"( { - "MaxSizeBytes": "1000", - "MaxCnt": "10", + "MinSizeBytes": "1000", + "MinCnt": "10", "TimeoutSecs": "5" } )"; @@ -85,9 +88,10 @@ void BatcherUnittest::TestParamInit() { Batcher<> batch; batch.Init(configJson, sFlusher.get(), strategy); - APSARA_TEST_EQUAL(1U, batch.mEventFlushStrategy.GetMaxCnt()); - APSARA_TEST_EQUAL(100U, batch.mEventFlushStrategy.GetMaxSizeBytes()); + APSARA_TEST_EQUAL(1U, batch.mEventFlushStrategy.GetMinCnt()); + APSARA_TEST_EQUAL(100U, batch.mEventFlushStrategy.GetMinSizeBytes()); APSARA_TEST_EQUAL(3U, batch.mEventFlushStrategy.GetTimeoutSecs()); + APSARA_TEST_EQUAL(300U, batch.mEventFlushStrategy.GetMaxSizeBytes()); } } @@ -96,8 +100,8 @@ void BatcherUnittest::TestInitWithoutGroupBatch() { string configStr, errorMsg; configStr = R"( { - "MaxSizeBytes": 1000, - "MaxCnt": 10, + "MinSizeBytes": 1000, + "MinCnt": 10, "TimeoutSecs": 5 } )"; @@ -105,9 +109,10 @@ void BatcherUnittest::TestInitWithoutGroupBatch() { Batcher<> batch; batch.Init(configJson, sFlusher.get(), DefaultFlushStrategyOptions()); - APSARA_TEST_EQUAL(10U, batch.mEventFlushStrategy.GetMaxCnt()); - APSARA_TEST_EQUAL(1000U, batch.mEventFlushStrategy.GetMaxSizeBytes()); + APSARA_TEST_EQUAL(10U, batch.mEventFlushStrategy.GetMinCnt()); + APSARA_TEST_EQUAL(1000U, batch.mEventFlushStrategy.GetMinSizeBytes()); APSARA_TEST_EQUAL(5U, batch.mEventFlushStrategy.GetTimeoutSecs()); + APSARA_TEST_EQUAL(numeric_limits::max(), batch.mEventFlushStrategy.GetMaxSizeBytes()); APSARA_TEST_EQUAL(sFlusher.get(), batch.mFlusher); } @@ -116,8 +121,8 @@ void BatcherUnittest::TestInitWithGroupBatch() { string configStr, errorMsg; configStr = R"( { - "MaxSizeBytes": 1000, - "MaxCnt": 10, + "MinSizeBytes": 1000, + "MinCnt": 10, "TimeoutSecs": 5 } )"; @@ -125,11 +130,12 @@ void BatcherUnittest::TestInitWithGroupBatch() { Batcher<> batch; batch.Init(configJson, sFlusher.get(), DefaultFlushStrategyOptions(), true); - APSARA_TEST_EQUAL(10U, batch.mEventFlushStrategy.GetMaxCnt()); - APSARA_TEST_EQUAL(1000U, batch.mEventFlushStrategy.GetMaxSizeBytes()); + APSARA_TEST_EQUAL(10U, batch.mEventFlushStrategy.GetMinCnt()); + APSARA_TEST_EQUAL(1000U, batch.mEventFlushStrategy.GetMinSizeBytes()); APSARA_TEST_EQUAL(3U, batch.mEventFlushStrategy.GetTimeoutSecs()); + APSARA_TEST_EQUAL(numeric_limits::max(), batch.mEventFlushStrategy.GetMaxSizeBytes()); APSARA_TEST_TRUE(batch.mGroupFlushStrategy); - APSARA_TEST_EQUAL(1000U, batch.mGroupFlushStrategy->GetMaxSizeBytes()); + APSARA_TEST_EQUAL(1000U, batch.mGroupFlushStrategy->GetMinSizeBytes()); APSARA_TEST_EQUAL(2U, batch.mGroupFlushStrategy->GetTimeoutSecs()); APSARA_TEST_TRUE(batch.mGroupQueue); APSARA_TEST_EQUAL(sFlusher.get(), batch.mFlusher); @@ -137,8 +143,8 @@ void BatcherUnittest::TestInitWithGroupBatch() { void BatcherUnittest::TestAddWithoutGroupBatch() { DefaultFlushStrategyOptions strategy; - strategy.mMaxCnt = 3; - strategy.mMaxSizeBytes = 1000; + strategy.mMinCnt = 3; + strategy.mMinSizeBytes = 1000; strategy.mTimeoutSecs = 3; Batcher<> batch; @@ -186,7 +192,7 @@ void BatcherUnittest::TestAddWithoutGroupBatch() { // flush by time then by size res.clear(); batch.mEventFlushStrategy.SetTimeoutSecs(0); - batch.mEventFlushStrategy.SetMaxSizeBytes(10); + batch.mEventFlushStrategy.SetMinSizeBytes(10); PipelineEventGroup group3 = CreateEventGroup(1); SourceBuffer* buffer3 = group3.GetSourceBuffer().get(); RangeCheckpoint* eoo3 = group3.GetExactlyOnceCheckpoint().get(); @@ -218,8 +224,8 @@ void BatcherUnittest::TestAddWithoutGroupBatch() { void BatcherUnittest::TestAddWithGroupBatch() { DefaultFlushStrategyOptions strategy; - strategy.mMaxCnt = 3; - strategy.mMaxSizeBytes = 1000; + strategy.mMinCnt = 3; + strategy.mMinSizeBytes = 1000; strategy.mTimeoutSecs = 3; Batcher<> batch; @@ -298,7 +304,7 @@ void BatcherUnittest::TestAddWithGroupBatch() { // flush by time to group batch, and then group flush by size res.clear(); batch.mGroupFlushStrategy->SetTimeoutSecs(3); - batch.mGroupFlushStrategy->SetMaxSizeBytes(10); + batch.mGroupFlushStrategy->SetMinSizeBytes(10); PipelineEventGroup group5 = CreateEventGroup(1); SourceBuffer* buffer5 = group5.GetSourceBuffer().get(); RangeCheckpoint* eoo5 = group5.GetExactlyOnceCheckpoint().get(); @@ -328,31 +334,63 @@ void BatcherUnittest::TestAddWithGroupBatch() { // flush by size res.clear(); - batch.mEventFlushStrategy.SetMaxSizeBytes(10); + batch.mEventFlushStrategy.SetMinSizeBytes(159); batch.mEventFlushStrategy.SetTimeoutSecs(3); PipelineEventGroup group6 = CreateEventGroup(1); SourceBuffer* buffer6 = group6.GetSourceBuffer().get(); batch.Add(std::move(group6), res); + PipelineEventGroup group7 = CreateEventGroup(2); + SourceBuffer* buffer7 = group7.GetSourceBuffer().get(); + batch.Add(std::move(group7), res); APSARA_TEST_EQUAL(1U, batch.mEventQueueMap.size()); - APSARA_TEST_EQUAL(0U, batch.mEventQueueMap[key].mBatch.mEvents.size()); + APSARA_TEST_EQUAL(1U, batch.mEventQueueMap[key].mBatch.mEvents.size()); APSARA_TEST_EQUAL(1U, res.size()); APSARA_TEST_EQUAL(1U, res[0].size()); - APSARA_TEST_EQUAL(2U, res[0][0].mEvents.size()); + APSARA_TEST_EQUAL(3U, res[0][0].mEvents.size()); APSARA_TEST_EQUAL(1U, res[0][0].mTags.mInner.size()); APSARA_TEST_STREQ("val", res[0][0].mTags.mInner["key"].data()); - APSARA_TEST_EQUAL(2U, res[0][0].mSourceBuffers.size()); + APSARA_TEST_EQUAL(3U, res[0][0].mSourceBuffers.size()); APSARA_TEST_EQUAL(buffer5, res[0][0].mSourceBuffers[0].get()); APSARA_TEST_EQUAL(buffer6, res[0][0].mSourceBuffers[1].get()); + APSARA_TEST_EQUAL(buffer7, res[0][0].mSourceBuffers[2].get()); APSARA_TEST_EQUAL(eoo5, res[0][0].mExactlyOnceCheckpoint.get()); APSARA_TEST_STREQ("pack_id", res[0][0].mPackIdPrefix.data()); APSARA_TEST_GT(TimeoutFlushManager::GetInstance()->mTimeoutRecords["test_config"].at(make_pair(0, key)).mUpdateTime, updateTime - 1); } +void BatcherUnittest::TestAddWithOversizedGroup() { + DefaultFlushStrategyOptions strategy; + strategy.mMaxSizeBytes = 500; + strategy.mMinCnt = 3; + strategy.mMinSizeBytes = 200; + strategy.mTimeoutSecs = 3; + + Batcher<> batch; + batch.Init(Json::Value(), sFlusher.get(), strategy); + + vector res; + PipelineEventGroup group1 = CreateEventGroup(2); + size_t key = group1.GetTagsHash(); + batch.Add(std::move(group1), res); + + PipelineEventGroup group2 = CreateEventGroup(20); + batch.Add(std::move(group2), res); + APSARA_TEST_EQUAL(1U, batch.mEventQueueMap.size()); + APSARA_TEST_EQUAL(0U, batch.mEventQueueMap[key].mBatch.mEvents.size()); + APSARA_TEST_EQUAL(3U, res.size()); + APSARA_TEST_EQUAL(1U, res[0].size()); + APSARA_TEST_EQUAL(2U, res[0][0].mEvents.size()); + APSARA_TEST_EQUAL(1U, res[1].size()); + APSARA_TEST_EQUAL(13U, res[1][0].mEvents.size()); + APSARA_TEST_EQUAL(1U, res[2].size()); + APSARA_TEST_EQUAL(7U, res[2][0].mEvents.size()); +} + void BatcherUnittest::TestFlushEventQueueWithoutGroupBatch() { DefaultFlushStrategyOptions strategy; - strategy.mMaxCnt = 3; - strategy.mMaxSizeBytes = 1000; + strategy.mMinCnt = 3; + strategy.mMinSizeBytes = 1000; strategy.mTimeoutSecs = 3; Batcher<> batch; @@ -385,8 +423,8 @@ void BatcherUnittest::TestFlushEventQueueWithoutGroupBatch() { void BatcherUnittest::TestFlushEventQueueWithGroupBatch() { DefaultFlushStrategyOptions strategy; - strategy.mMaxCnt = 10; - strategy.mMaxSizeBytes = 1000; + strategy.mMinCnt = 10; + strategy.mMinSizeBytes = 1000; strategy.mTimeoutSecs = 3; Batcher<> batch; @@ -415,7 +453,7 @@ void BatcherUnittest::TestFlushEventQueueWithGroupBatch() { // flush to group item, and group is flushed by time then by size batch.mGroupFlushStrategy->SetTimeoutSecs(0); - batch.mGroupFlushStrategy->SetMaxSizeBytes(10); + batch.mGroupFlushStrategy->SetMinSizeBytes(10); PipelineEventGroup group2 = CreateEventGroup(2); SourceBuffer* buffer2 = group2.GetSourceBuffer().get(); RangeCheckpoint* eoo2 = group2.GetExactlyOnceCheckpoint().get(); @@ -441,8 +479,8 @@ void BatcherUnittest::TestFlushEventQueueWithGroupBatch() { void BatcherUnittest::TestFlushGroupQueue() { DefaultFlushStrategyOptions strategy; - strategy.mMaxCnt = 3; - strategy.mMaxSizeBytes = 1000; + strategy.mMinCnt = 3; + strategy.mMinSizeBytes = 1000; strategy.mTimeoutSecs = 3; { // no group item @@ -481,8 +519,8 @@ void BatcherUnittest::TestFlushGroupQueue() { void BatcherUnittest::TestFlushAllWithoutGroupBatch() { DefaultFlushStrategyOptions strategy; - strategy.mMaxCnt = 3; - strategy.mMaxSizeBytes = 1000; + strategy.mMinCnt = 3; + strategy.mMinSizeBytes = 1000; strategy.mTimeoutSecs = 3; Batcher<> batch; @@ -509,8 +547,8 @@ void BatcherUnittest::TestFlushAllWithoutGroupBatch() { void BatcherUnittest::TestFlushAllWithGroupBatch() { DefaultFlushStrategyOptions strategy; - strategy.mMaxCnt = 3; - strategy.mMaxSizeBytes = 1000; + strategy.mMinCnt = 3; + strategy.mMinSizeBytes = 1000; strategy.mTimeoutSecs = 3; Batcher<> batch; @@ -537,7 +575,7 @@ void BatcherUnittest::TestFlushAllWithGroupBatch() { // flush all by time then by size batch.mGroupFlushStrategy->SetTimeoutSecs(0); - batch.mGroupFlushStrategy->SetMaxSizeBytes(10); + batch.mGroupFlushStrategy->SetMinSizeBytes(10); vector res; batch.FlushAll(res); APSARA_TEST_EQUAL(0U, batch.mEventQueueMap.size()); @@ -563,8 +601,8 @@ void BatcherUnittest::TestFlushAllWithGroupBatch() { void BatcherUnittest::TestMetric() { { DefaultFlushStrategyOptions strategy; - strategy.mMaxCnt = 2; - strategy.mMaxSizeBytes = 1000; + strategy.mMinCnt = 2; + strategy.mMinSizeBytes = 1000; strategy.mTimeoutSecs = 3; Batcher<> batch; @@ -579,7 +617,8 @@ void BatcherUnittest::TestMetric() { APSARA_TEST_EQUAL(6U, batch.mMetricsRecordRef->GetLabels()->size()); APSARA_TEST_TRUE(batch.mMetricsRecordRef.HasLabel(METRIC_LABEL_KEY_PROJECT, "")); APSARA_TEST_TRUE(batch.mMetricsRecordRef.HasLabel(METRIC_LABEL_KEY_PIPELINE_NAME, "test_config")); - APSARA_TEST_TRUE(batch.mMetricsRecordRef.HasLabel(METRIC_LABEL_KEY_COMPONENT_NAME, METRIC_LABEL_VALUE_COMPONENT_NAME_BATCHER)); + APSARA_TEST_TRUE(batch.mMetricsRecordRef.HasLabel(METRIC_LABEL_KEY_COMPONENT_NAME, + METRIC_LABEL_VALUE_COMPONENT_NAME_BATCHER)); APSARA_TEST_TRUE(batch.mMetricsRecordRef.HasLabel(METRIC_LABEL_KEY_FLUSHER_PLUGIN_ID, "1")); APSARA_TEST_TRUE(batch.mMetricsRecordRef.HasLabel(METRIC_LABEL_KEY_GROUP_BATCH_ENABLED, "false")); APSARA_TEST_EQUAL(3U, batch.mInEventsTotal->GetValue()); @@ -592,8 +631,8 @@ void BatcherUnittest::TestMetric() { } { DefaultFlushStrategyOptions strategy; - strategy.mMaxCnt = 2; - strategy.mMaxSizeBytes = 1000; + strategy.mMinCnt = 2; + strategy.mMinSizeBytes = 1000; strategy.mTimeoutSecs = 3; Batcher<> batch; @@ -632,6 +671,7 @@ PipelineEventGroup BatcherUnittest::CreateEventGroup(size_t cnt) { UNIT_TEST_CASE(BatcherUnittest, TestParamInit) UNIT_TEST_CASE(BatcherUnittest, TestInitWithoutGroupBatch) UNIT_TEST_CASE(BatcherUnittest, TestInitWithGroupBatch) +UNIT_TEST_CASE(BatcherUnittest, TestAddWithOversizedGroup) UNIT_TEST_CASE(BatcherUnittest, TestAddWithoutGroupBatch) UNIT_TEST_CASE(BatcherUnittest, TestAddWithGroupBatch) UNIT_TEST_CASE(BatcherUnittest, TestFlushEventQueueWithoutGroupBatch) diff --git a/core/unittest/batch/FlushStrategyUnittest.cpp b/core/unittest/batch/FlushStrategyUnittest.cpp index 102b5fbfc2..b19e6c7cba 100644 --- a/core/unittest/batch/FlushStrategyUnittest.cpp +++ b/core/unittest/batch/FlushStrategyUnittest.cpp @@ -27,8 +27,9 @@ class EventFlushStrategyUnittest : public ::testing::Test { protected: void SetUp() override { - mStrategy.SetMaxCnt(2); - mStrategy.SetMaxSizeBytes(100); + mStrategy.SetMinCnt(2); + mStrategy.SetMaxSizeBytes(200); + mStrategy.SetMinSizeBytes(100); mStrategy.SetTimeoutSecs(3); } @@ -45,6 +46,7 @@ void EventFlushStrategyUnittest::TestNeedFlush() { APSARA_TEST_TRUE(mStrategy.NeedFlushByCnt(status)); APSARA_TEST_FALSE(mStrategy.NeedFlushBySize(status)); APSARA_TEST_FALSE(mStrategy.NeedFlushByTime(status, PipelineEventPtr())); + APSARA_TEST_FALSE(mStrategy.SizeReachingUpperLimit(status)); status.mCnt = 1; status.mSizeBytes = 100; @@ -52,6 +54,7 @@ void EventFlushStrategyUnittest::TestNeedFlush() { APSARA_TEST_FALSE(mStrategy.NeedFlushByCnt(status)); APSARA_TEST_TRUE(mStrategy.NeedFlushBySize(status)); APSARA_TEST_FALSE(mStrategy.NeedFlushByTime(status, PipelineEventPtr())); + APSARA_TEST_FALSE(mStrategy.SizeReachingUpperLimit(status)); status.mCnt = 1; status.mSizeBytes = 50; @@ -59,6 +62,12 @@ void EventFlushStrategyUnittest::TestNeedFlush() { APSARA_TEST_FALSE(mStrategy.NeedFlushByCnt(status)); APSARA_TEST_FALSE(mStrategy.NeedFlushBySize(status)); APSARA_TEST_TRUE(mStrategy.NeedFlushByTime(status, PipelineEventPtr())); + APSARA_TEST_FALSE(mStrategy.SizeReachingUpperLimit(status)); + + status.mSizeBytes = 300; + status.mCreateTime = time(nullptr) - 1; + APSARA_TEST_FALSE(mStrategy.NeedFlushByTime(status, PipelineEventPtr())); + APSARA_TEST_TRUE(mStrategy.SizeReachingUpperLimit(status)); } UNIT_TEST_CASE(EventFlushStrategyUnittest, TestNeedFlush) @@ -91,8 +100,8 @@ class SLSEventFlushStrategyUnittest : public ::testing::Test { protected: void SetUp() override { - mStrategy.SetMaxCnt(2); - mStrategy.SetMaxSizeBytes(100); + mStrategy.SetMinCnt(2); + mStrategy.SetMinSizeBytes(100); mStrategy.SetTimeoutSecs(3); } diff --git a/core/unittest/flusher/FlusherSLSUnittest.cpp b/core/unittest/flusher/FlusherSLSUnittest.cpp index a760f5b8cd..1e1251fd51 100644 --- a/core/unittest/flusher/FlusherSLSUnittest.cpp +++ b/core/unittest/flusher/FlusherSLSUnittest.cpp @@ -107,15 +107,17 @@ void FlusherSLSUnittest::OnSuccessfulInit() { APSARA_TEST_EQUAL(sls_logs::SlsTelemetryType::SLS_TELEMETRY_TYPE_LOGS, flusher->mTelemetryType); APSARA_TEST_TRUE(flusher->mShardHashKeys.empty()); APSARA_TEST_EQUAL(static_cast(INT32_FLAG(merge_log_count_limit)), - flusher->mBatcher.GetEventFlushStrategy().GetMaxCnt()); - APSARA_TEST_EQUAL(static_cast(INT32_FLAG(batch_send_metric_size)), + flusher->mBatcher.GetEventFlushStrategy().GetMinCnt()); + APSARA_TEST_EQUAL(static_cast(INT32_FLAG(max_send_log_group_size)), flusher->mBatcher.GetEventFlushStrategy().GetMaxSizeBytes()); + APSARA_TEST_EQUAL(static_cast(INT32_FLAG(batch_send_metric_size)), + flusher->mBatcher.GetEventFlushStrategy().GetMinSizeBytes()); uint32_t timeout = static_cast(INT32_FLAG(batch_send_interval)) / 2; APSARA_TEST_EQUAL(static_cast(INT32_FLAG(batch_send_interval)) - timeout, flusher->mBatcher.GetEventFlushStrategy().GetTimeoutSecs()); APSARA_TEST_TRUE(flusher->mBatcher.GetGroupFlushStrategy().has_value()); APSARA_TEST_EQUAL(static_cast(INT32_FLAG(batch_send_metric_size)), - flusher->mBatcher.GetGroupFlushStrategy()->GetMaxSizeBytes()); + flusher->mBatcher.GetGroupFlushStrategy()->GetMinSizeBytes()); APSARA_TEST_EQUAL(timeout, flusher->mBatcher.GetGroupFlushStrategy()->GetTimeoutSecs()); APSARA_TEST_TRUE(flusher->mGroupSerializer); APSARA_TEST_TRUE(flusher->mGroupListSerializer); @@ -645,7 +647,7 @@ void FlusherSLSUnittest::TestSend() { } { // non-replay group - flusher.mBatcher.GetEventFlushStrategy().SetMaxCnt(1); + flusher.mBatcher.GetEventFlushStrategy().SetMinCnt(1); PipelineEventGroup group(make_shared()); group.SetMetadata(EventGroupMetaKey::SOURCE_ID, string("source-id")); group.SetTag(LOG_RESERVED_KEY_HOSTNAME, "hostname"); @@ -730,7 +732,7 @@ void FlusherSLSUnittest::TestSend() { } { // group - flusher.mBatcher.GetEventFlushStrategy().SetMaxCnt(1); + flusher.mBatcher.GetEventFlushStrategy().SetMinCnt(1); PipelineEventGroup group(make_shared()); group.SetMetadata(EventGroupMetaKey::SOURCE_ID, string("source-id")); group.SetTag(LOG_RESERVED_KEY_HOSTNAME, "hostname"); @@ -778,19 +780,19 @@ void FlusherSLSUnittest::TestSend() { APSARA_TEST_EQUAL("content_value", logGroup.logs(0).contents(0).value()); SenderQueueManager::GetInstance()->RemoveItem(item->mQueueKey, item); - flusher.mBatcher.GetEventFlushStrategy().SetMaxCnt(4000); + flusher.mBatcher.GetEventFlushStrategy().SetMinCnt(4000); } { // oversized group INT32_FLAG(max_send_log_group_size) = 1; - flusher.mBatcher.GetEventFlushStrategy().SetMaxCnt(1); + flusher.mBatcher.GetEventFlushStrategy().SetMinCnt(1); PipelineEventGroup group(make_shared()); auto e = group.AddLogEvent(); e->SetTimestamp(1234567890); e->SetContent(string("content_key"), string("content_value")); APSARA_TEST_FALSE(flusher.Send(std::move(group))); INT32_FLAG(max_send_log_group_size) = 10 * 1024 * 1024; - flusher.mBatcher.GetEventFlushStrategy().SetMaxCnt(4000); + flusher.mBatcher.GetEventFlushStrategy().SetMinCnt(4000); } } { @@ -829,7 +831,7 @@ void FlusherSLSUnittest::TestSend() { e->SetTimestamp(1234567990); e->SetContent(string("content_key"), string("content_value")); } - flusher.mBatcher.GetGroupFlushStrategy()->SetMaxSizeBytes(group.DataSize()); + flusher.mBatcher.GetGroupFlushStrategy()->SetMinSizeBytes(group.DataSize()); // flush the above two events from group item by the following event { auto e = group.AddLogEvent(); @@ -892,7 +894,7 @@ void FlusherSLSUnittest::TestSend() { for (auto& tmp : res) { SenderQueueManager::GetInstance()->RemoveItem(tmp->mQueueKey, tmp); } - flusher.mBatcher.GetGroupFlushStrategy()->SetMaxSizeBytes(256 * 1024); + flusher.mBatcher.GetGroupFlushStrategy()->SetMinSizeBytes(256 * 1024); } } diff --git a/core/unittest/instance_config/InstanceConfigManagerUnittest.cpp b/core/unittest/instance_config/InstanceConfigManagerUnittest.cpp index 5adb906538..f3bc3c7b54 100644 --- a/core/unittest/instance_config/InstanceConfigManagerUnittest.cpp +++ b/core/unittest/instance_config/InstanceConfigManagerUnittest.cpp @@ -22,9 +22,6 @@ using namespace std; -DECLARE_FLAG_BOOL(enable_send_tps_smoothing); -DECLARE_FLAG_BOOL(enable_flow_control); - namespace logtail { class InstanceConfigManagerUnittest : public testing::Test { @@ -151,8 +148,7 @@ void InstanceConfigManagerUnittest::TestUpdateInstanceConfigs() { APSARA_TEST_EQUAL(nullptr, InstanceConfigManager::GetInstance()->FindConfigByName("test3")); } APSARA_TEST_EQUAL(kDefaultMaxSendBytePerSec, AppConfig::GetInstance()->GetMaxBytePerSec()); - APSARA_TEST_EQUAL(true, FlusherRunner::GetInstance()->mSendRandomSleep); - APSARA_TEST_EQUAL(true, FlusherRunner::GetInstance()->mSendFlowControl); + APSARA_TEST_EQUAL(true, FlusherRunner::GetInstance()->mEnableRateLimiter); // Modified status = 1; { @@ -206,8 +202,7 @@ void InstanceConfigManagerUnittest::TestUpdateInstanceConfigs() { APSARA_TEST_EQUAL(nullptr, InstanceConfigManager::GetInstance()->FindConfigByName("test3")); } APSARA_TEST_EQUAL(31457280, AppConfig::GetInstance()->GetMaxBytePerSec()); - APSARA_TEST_EQUAL(false, FlusherRunner::GetInstance()->mSendRandomSleep); - APSARA_TEST_EQUAL(false, FlusherRunner::GetInstance()->mSendFlowControl); + APSARA_TEST_EQUAL(false, FlusherRunner::GetInstance()->mEnableRateLimiter); // Removed status = 2; { From 9759b5295327b09ded1ad1c2aac16c754399e08c Mon Sep 17 00:00:00 2001 From: henryzhx8 Date: Thu, 7 Nov 2024 15:43:34 +0800 Subject: [PATCH 2/5] read blocked event as soon as process queue triggers feedback to improve file reading performance (#1863) --- core/file_server/event/BlockEventManager.cpp | 8 +++- core/file_server/event_handler/LogInput.cpp | 40 ++++++++++---------- core/file_server/event_handler/LogInput.h | 5 +++ 3 files changed, 31 insertions(+), 22 deletions(-) diff --git a/core/file_server/event/BlockEventManager.cpp b/core/file_server/event/BlockEventManager.cpp index b583d25e6a..006fbff7fd 100644 --- a/core/file_server/event/BlockEventManager.cpp +++ b/core/file_server/event/BlockEventManager.cpp @@ -17,6 +17,7 @@ #include "common/Flags.h" #include "common/HashUtil.h" #include "common/StringTools.h" +#include "file_server/event_handler/LogInput.h" #include "logger/Logger.h" #include "pipeline/queue/ProcessQueueManager.h" @@ -69,8 +70,11 @@ BlockedEventManager::~BlockedEventManager() { } void BlockedEventManager::Feedback(int64_t key) { - lock_guard lock(mFeedbackQueueMux); - mFeedbackQueue.emplace_back(key); + { + lock_guard lock(mFeedbackQueueMux); + mFeedbackQueue.emplace_back(key); + } + LogInput::GetInstance()->Trigger(); } void BlockedEventManager::UpdateBlockEvent( diff --git a/core/file_server/event_handler/LogInput.cpp b/core/file_server/event_handler/LogInput.cpp index 38b6e4ab36..299e660c68 100644 --- a/core/file_server/event_handler/LogInput.cpp +++ b/core/file_server/event_handler/LogInput.cpp @@ -88,9 +88,12 @@ void LogInput::Start() { mInteruptFlag = false; mLastRunTime = FileServer::GetInstance()->GetMetricsRecordRef().CreateIntGauge(METRIC_RUNNER_LAST_RUN_TIME); - mRegisterdHandlersTotal = FileServer::GetInstance()->GetMetricsRecordRef().CreateIntGauge(METRIC_RUNNER_FILE_WATCHED_DIRS_TOTAL); - mActiveReadersTotal = FileServer::GetInstance()->GetMetricsRecordRef().CreateIntGauge(METRIC_RUNNER_FILE_ACTIVE_READERS_TOTAL); - mEnableFileIncludedByMultiConfigs = FileServer::GetInstance()->GetMetricsRecordRef().CreateIntGauge(METRIC_RUNNER_FILE_ENABLE_FILE_INCLUDED_BY_MULTI_CONFIGS_FLAG); + mRegisterdHandlersTotal + = FileServer::GetInstance()->GetMetricsRecordRef().CreateIntGauge(METRIC_RUNNER_FILE_WATCHED_DIRS_TOTAL); + mActiveReadersTotal + = FileServer::GetInstance()->GetMetricsRecordRef().CreateIntGauge(METRIC_RUNNER_FILE_ACTIVE_READERS_TOTAL); + mEnableFileIncludedByMultiConfigs = FileServer::GetInstance()->GetMetricsRecordRef().CreateIntGauge( + METRIC_RUNNER_FILE_ENABLE_FILE_INCLUDED_BY_MULTI_CONFIGS_FLAG); new Thread([this]() { ProcessLoop(); }); } @@ -118,19 +121,14 @@ void LogInput::TryReadEvents(bool forceRead) { if (mInteruptFlag) return; - if (!forceRead) { - int64_t curMicroSeconds = GetCurrentTimeInMicroSeconds(); - if (curMicroSeconds - mLastReadEventMicroSeconds >= INT64_FLAG(read_fs_events_interval)) - mLastReadEventMicroSeconds = curMicroSeconds; - else - return; - } else - mLastReadEventMicroSeconds = GetCurrentTimeInMicroSeconds(); - - vector inotifyEvents; - EventDispatcher::GetInstance()->ReadInotifyEvents(inotifyEvents); - if (inotifyEvents.size() > 0) { - PushEventQueue(inotifyEvents); + int64_t curMicroSeconds = GetCurrentTimeInMicroSeconds(); + if (forceRead || curMicroSeconds - mLastReadEventMicroSeconds >= INT64_FLAG(read_fs_events_interval)) { + vector inotifyEvents; + EventDispatcher::GetInstance()->ReadInotifyEvents(inotifyEvents); + if (inotifyEvents.size() > 0) { + PushEventQueue(inotifyEvents); + } + mLastReadEventMicroSeconds = curMicroSeconds; } vector feedbackEvents; @@ -212,8 +210,7 @@ bool LogInput::ReadLocalEvents() { } // set discard old data flag, so that history data will not be dropped. BOOL_FLAG(ilogtail_discard_old_data) = false; - LOG_INFO(sLogger, - ("load local events", GetLocalEventDataFileName())("event count", localEventJson.size())); + LOG_INFO(sLogger, ("load local events", GetLocalEventDataFileName())("event count", localEventJson.size())); for (Json::ValueIterator iter = localEventJson.begin(); iter != localEventJson.end(); ++iter) { const Json::Value& eventItem = *iter; if (!eventItem.isObject()) { @@ -395,8 +392,11 @@ void* LogInput::ProcessLoop() { delete ev; else ProcessEvent(dispatcher, ev); - } else - usleep(INT32_FLAG(log_input_thread_wait_interval)); + } else { + unique_lock lock(mFeedbackMux); + mFeedbackCV.wait_for(lock, chrono::microseconds(INT32_FLAG(log_input_thread_wait_interval))); + } + if (mIdleFlag) continue; diff --git a/core/file_server/event_handler/LogInput.h b/core/file_server/event_handler/LogInput.h index 05a6ebbc6f..aadeb17082 100644 --- a/core/file_server/event_handler/LogInput.h +++ b/core/file_server/event_handler/LogInput.h @@ -60,6 +60,8 @@ class LogInput : public LogRunnable { int32_t GetLastReadEventTime() { return mLastReadEventTime; } + void Trigger() { mFeedbackCV.notify_one(); } + private: LogInput(); ~LogInput(); @@ -89,6 +91,9 @@ class LogInput : public LogRunnable { mutable std::mutex mThreadRunningMux; mutable std::condition_variable mStopCV; + mutable std::mutex mFeedbackMux; + mutable std::condition_variable mFeedbackCV; + #ifdef APSARA_UNIT_TEST_MAIN friend class LogInputUnittest; friend class EventDispatcherTest; From fd95ac6d6483c5ccef5c7786665662a3023dcefb Mon Sep 17 00:00:00 2001 From: henryzhx8 Date: Mon, 11 Nov 2024 14:51:27 +0800 Subject: [PATCH 3/5] change default sls_serialize_size_expansion_ratio --- core/plugin/flusher/sls/FlusherSLS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/plugin/flusher/sls/FlusherSLS.cpp b/core/plugin/flusher/sls/FlusherSLS.cpp index b0fd4d629a..97ab507be8 100644 --- a/core/plugin/flusher/sls/FlusherSLS.cpp +++ b/core/plugin/flusher/sls/FlusherSLS.cpp @@ -60,7 +60,7 @@ DEFINE_FLAG_INT32(unknow_error_try_max, "discard data when try times > this valu DEFINE_FLAG_BOOL(global_network_success, "global network success flag, default false", false); DEFINE_FLAG_BOOL(enable_metricstore_channel, "only works for metrics data for enhance metrics query performance", true); DEFINE_FLAG_INT32(max_send_log_group_size, "bytes", 10 * 1024 * 1024); -DEFINE_FLAG_INT32(sls_serialize_size_expansion_ratio, "", 1.15); +DEFINE_FLAG_DOUBLE(sls_serialize_size_expansion_ratio, "", 1.2); DECLARE_FLAG_BOOL(send_prefer_real_ip); From 314785e56ba27ade75a07f2b007d476ef9ef8cc9 Mon Sep 17 00:00:00 2001 From: henryzhx8 Date: Tue, 12 Nov 2024 13:33:27 +0800 Subject: [PATCH 4/5] fix flusher_sls ut (#1874) --- core/plugin/flusher/sls/FlusherSLS.cpp | 2 +- core/unittest/flusher/FlusherSLSUnittest.cpp | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/plugin/flusher/sls/FlusherSLS.cpp b/core/plugin/flusher/sls/FlusherSLS.cpp index 97ab507be8..524b4550b9 100644 --- a/core/plugin/flusher/sls/FlusherSLS.cpp +++ b/core/plugin/flusher/sls/FlusherSLS.cpp @@ -477,7 +477,7 @@ bool FlusherSLS::Init(const Json::Value& config, Json::Value& optionalGoPipeline } DefaultFlushStrategyOptions strategy{ - static_cast(INT32_FLAG(max_send_log_group_size) / INT32_FLAG(sls_serialize_size_expansion_ratio)), + static_cast(INT32_FLAG(max_send_log_group_size) / DOUBLE_FLAG(sls_serialize_size_expansion_ratio)), static_cast(INT32_FLAG(batch_send_metric_size)), static_cast(INT32_FLAG(merge_log_count_limit)), static_cast(INT32_FLAG(batch_send_interval))}; diff --git a/core/unittest/flusher/FlusherSLSUnittest.cpp b/core/unittest/flusher/FlusherSLSUnittest.cpp index 1e1251fd51..73dc1e97e3 100644 --- a/core/unittest/flusher/FlusherSLSUnittest.cpp +++ b/core/unittest/flusher/FlusherSLSUnittest.cpp @@ -39,6 +39,7 @@ DECLARE_FLAG_INT32(batch_send_interval); DECLARE_FLAG_INT32(merge_log_count_limit); DECLARE_FLAG_INT32(batch_send_metric_size); DECLARE_FLAG_INT32(max_send_log_group_size); +DECLARE_FLAG_DOUBLE(sls_serialize_size_expansion_ratio); using namespace std; @@ -108,7 +109,7 @@ void FlusherSLSUnittest::OnSuccessfulInit() { APSARA_TEST_TRUE(flusher->mShardHashKeys.empty()); APSARA_TEST_EQUAL(static_cast(INT32_FLAG(merge_log_count_limit)), flusher->mBatcher.GetEventFlushStrategy().GetMinCnt()); - APSARA_TEST_EQUAL(static_cast(INT32_FLAG(max_send_log_group_size)), + APSARA_TEST_EQUAL(static_cast(INT32_FLAG(max_send_log_group_size) / DOUBLE_FLAG(sls_serialize_size_expansion_ratio)), flusher->mBatcher.GetEventFlushStrategy().GetMaxSizeBytes()); APSARA_TEST_EQUAL(static_cast(INT32_FLAG(batch_send_metric_size)), flusher->mBatcher.GetEventFlushStrategy().GetMinSizeBytes()); From 4741ec819d7267fa39e3a71ceb31ca9943c5b0bc Mon Sep 17 00:00:00 2001 From: Bingchang Chen Date: Tue, 12 Nov 2024 13:50:57 +0800 Subject: [PATCH 5/5] test: refactor E2E trigger (#1861) * test: refactor E2E trigger * fix * fix * fix * fix * fix * fix * fix * fix --- .github/workflows/benchmark.yaml | 7 + core/unittest/pipeline/PipelineUnittest.cpp | 1 + docker/Dockerfile.e2e-test | 5 +- .../case.feature | 2 +- .../case.feature | 2 +- .../case.feature | 2 +- .../case.feature | 2 +- .../case.feature | 2 +- test/config/config.go | 5 + test/engine/setup/docker_compose.go | 8 +- test/engine/steps.go | 36 ++-- .../trigger/{protocol.go => ebpf/http.go} | 4 +- test/engine/trigger/ebpf/remote_mmap.py | 22 ++ .../{ebpf_trigger.go => ebpf/security.go} | 27 +-- test/engine/trigger/file.go | 114 ---------- test/engine/trigger/generator/apsara_test.go | 73 ------- .../trigger/generator/delimiter_test.go | 127 ----------- .../trigger/generator/ebpf_file_mmap_test.go | 51 ----- test/engine/trigger/generator/helper.go | 85 -------- test/engine/trigger/generator/json_test.go | 119 ----------- test/engine/trigger/generator/regex_test.go | 198 ------------------ test/engine/trigger/helper.go | 21 +- test/engine/trigger/log/file.go | 105 ++++++++++ .../trigger/{http.go => log/http_server.go} | 2 +- test/engine/trigger/log/remote_file.py | 172 +++++++++++++++ .../trigger/log/remote_file_benchmark.py | 70 +++++++ test/engine/trigger/trigger.go | 96 --------- test/requirements.txt | 1 + 28 files changed, 449 insertions(+), 910 deletions(-) rename test/engine/trigger/{protocol.go => ebpf/http.go} (90%) create mode 100644 test/engine/trigger/ebpf/remote_mmap.py rename test/engine/trigger/{ebpf_trigger.go => ebpf/security.go} (76%) delete mode 100644 test/engine/trigger/file.go delete mode 100644 test/engine/trigger/generator/apsara_test.go delete mode 100644 test/engine/trigger/generator/delimiter_test.go delete mode 100644 test/engine/trigger/generator/ebpf_file_mmap_test.go delete mode 100644 test/engine/trigger/generator/helper.go delete mode 100644 test/engine/trigger/generator/json_test.go delete mode 100644 test/engine/trigger/generator/regex_test.go create mode 100644 test/engine/trigger/log/file.go rename test/engine/trigger/{http.go => log/http_server.go} (99%) create mode 100644 test/engine/trigger/log/remote_file.py create mode 100644 test/engine/trigger/log/remote_file_benchmark.py delete mode 100644 test/engine/trigger/trigger.go create mode 100644 test/requirements.txt diff --git a/.github/workflows/benchmark.yaml b/.github/workflows/benchmark.yaml index 5cc0ff4411..2f29670fca 100644 --- a/.github/workflows/benchmark.yaml +++ b/.github/workflows/benchmark.yaml @@ -37,6 +37,7 @@ jobs: strategy: matrix: go-version: [ 1.19.10 ] + python-version: [ 3.8 ] runner: [ ubuntu-latest ] fail-fast: true permissions: @@ -62,6 +63,11 @@ jobs: with: go-version: ${{ matrix.go-version }} + - name: Set up Python ${{ matrix.python-version }} + uses: actions/setup-python@v5 + with: + python-version: ${{ matrix.python-version }} + - name: Check out code uses: actions/checkout@v2 with: @@ -83,6 +89,7 @@ jobs: BUILD_LOGTAIL_UT: OFF WITHOUTGDB: ON run: | + pip3 install -r test/requirements.txt make benchmark git stash diff --git a/core/unittest/pipeline/PipelineUnittest.cpp b/core/unittest/pipeline/PipelineUnittest.cpp index 1cf92420d5..396e84627c 100644 --- a/core/unittest/pipeline/PipelineUnittest.cpp +++ b/core/unittest/pipeline/PipelineUnittest.cpp @@ -2916,6 +2916,7 @@ void PipelineUnittest::TestWaitAllItemsInProcessFinished() const { APSARA_TEST_NOT_EQUAL(std::future_status::ready, future.wait_for(std::chrono::seconds(0))); pipeline->mInProcessCnt.store(0); // recover + usleep(3000); APSARA_TEST_EQUAL(std::future_status::ready, future.wait_for(std::chrono::seconds(0))); } diff --git a/docker/Dockerfile.e2e-test b/docker/Dockerfile.e2e-test index 785df4caa3..e2acd47d42 100644 --- a/docker/Dockerfile.e2e-test +++ b/docker/Dockerfile.e2e-test @@ -1,12 +1,11 @@ -FROM golang:1.19 +FROM python:3.8 -RUN go env -w GOPROXY="https://goproxy.cn,direct" RUN mkdir -p /tmp/loongcollector WORKDIR /root COPY . ./loongcollector WORKDIR /root/loongcollector/test -RUN go mod download +RUN pip3 install -r requirements.txt CMD ["sh", "-c", "while true; do sleep 3600; done"] \ No newline at end of file diff --git a/test/benchmark/test_cases/performance_file_to_blackhole_filebeat/case.feature b/test/benchmark/test_cases/performance_file_to_blackhole_filebeat/case.feature index b8bbf86d1a..58e2ec8234 100644 --- a/test/benchmark/test_cases/performance_file_to_blackhole_filebeat/case.feature +++ b/test/benchmark/test_cases/performance_file_to_blackhole_filebeat/case.feature @@ -8,5 +8,5 @@ Feature: performance file to blackhole filebeat Given docker-compose boot type {benchmark} When start docker-compose {performance_file_to_blackhole_filebeat} When start monitor {filebeat} - When generate random nginx logs to file, speed {10}MB/s, total {3}min, to file {./a.log} + When generate random nginx logs to file, speed {10}MB/s, total {3}min, to file {./test_cases/performance_file_to_blackhole_filebeat/a.log} When wait monitor until log processing finished diff --git a/test/benchmark/test_cases/performance_file_to_blackhole_fluentbit/case.feature b/test/benchmark/test_cases/performance_file_to_blackhole_fluentbit/case.feature index 449511f10d..43de7c8c04 100644 --- a/test/benchmark/test_cases/performance_file_to_blackhole_fluentbit/case.feature +++ b/test/benchmark/test_cases/performance_file_to_blackhole_fluentbit/case.feature @@ -8,5 +8,5 @@ Feature: performance file to blackhole fluentbit Given docker-compose boot type {benchmark} When start docker-compose {performance_file_to_blackhole_fluentbit} When start monitor {fluent-bit} - When generate random nginx logs to file, speed {10}MB/s, total {3}min, to file {./a.log} + When generate random nginx logs to file, speed {10}MB/s, total {3}min, to file {./test_cases/performance_file_to_blackhole_fluentbit/a.log} When wait monitor until log processing finished diff --git a/test/benchmark/test_cases/performance_file_to_blackhole_ilogtail/case.feature b/test/benchmark/test_cases/performance_file_to_blackhole_ilogtail/case.feature index 92bb93f5d6..67e7913180 100644 --- a/test/benchmark/test_cases/performance_file_to_blackhole_ilogtail/case.feature +++ b/test/benchmark/test_cases/performance_file_to_blackhole_ilogtail/case.feature @@ -8,5 +8,5 @@ Feature: performance file to blackhole iLogtail Given docker-compose boot type {benchmark} When start docker-compose {performance_file_to_blackhole_ilogtail} When start monitor {ilogtailC} - When generate random nginx logs to file, speed {10}MB/s, total {3}min, to file {./a.log} + When generate random nginx logs to file, speed {10}MB/s, total {3}min, to file {./test_cases/performance_file_to_blackhole_ilogtail/a.log} When wait monitor until log processing finished diff --git a/test/benchmark/test_cases/performance_file_to_blackhole_ilogtailspl/case.feature b/test/benchmark/test_cases/performance_file_to_blackhole_ilogtailspl/case.feature index 327b8d27a1..0a2cc6403f 100644 --- a/test/benchmark/test_cases/performance_file_to_blackhole_ilogtailspl/case.feature +++ b/test/benchmark/test_cases/performance_file_to_blackhole_ilogtailspl/case.feature @@ -8,5 +8,5 @@ Feature: performance file to blackhole iLogtail Given docker-compose boot type {benchmark} When start docker-compose {performance_file_to_blackhole_ilogtailspl} When start monitor {ilogtailC} - When generate random nginx logs to file, speed {10}MB/s, total {3}min, to file {./a.log} + When generate random nginx logs to file, speed {10}MB/s, total {3}min, to file {./test_cases/performance_file_to_blackhole_ilogtailspl/a.log} When wait monitor until log processing finished diff --git a/test/benchmark/test_cases/performance_file_to_blackhole_vector/case.feature b/test/benchmark/test_cases/performance_file_to_blackhole_vector/case.feature index b0e54b85c1..334b2b3cbb 100644 --- a/test/benchmark/test_cases/performance_file_to_blackhole_vector/case.feature +++ b/test/benchmark/test_cases/performance_file_to_blackhole_vector/case.feature @@ -8,5 +8,5 @@ Feature: performance file to blackhole vector Given docker-compose boot type {benchmark} When start docker-compose {performance_file_to_blackhole_vector} When start monitor {vector} - When generate random nginx logs to file, speed {10}MB/s, total {3}min, to file {./a.log} + When generate random nginx logs to file, speed {10}MB/s, total {3}min, to file {./test_cases/performance_file_to_blackhole_vector/a.log} When wait monitor until log processing finished diff --git a/test/config/config.go b/test/config/config.go index 915bd08c6d..0ddd20c517 100644 --- a/test/config/config.go +++ b/test/config/config.go @@ -15,6 +15,7 @@ package config import ( "os" + "path/filepath" "strconv" "time" @@ -72,6 +73,10 @@ func ParseConfig() { TestConfig.GeneratedLogDir = "/tmp/loongcollector" } TestConfig.WorkDir = os.Getenv("WORK_DIR") + if len(TestConfig.WorkDir) == 0 { + testFileDir, _ := os.Getwd() + TestConfig.WorkDir = filepath.Dir(testFileDir) + } // SSH TestConfig.SSHUsername = os.Getenv("SSH_USERNAME") diff --git a/test/engine/setup/docker_compose.go b/test/engine/setup/docker_compose.go index 6a7e3fc4de..3a78dfb710 100644 --- a/test/engine/setup/docker_compose.go +++ b/test/engine/setup/docker_compose.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "os" + "os/exec" "path/filepath" "time" @@ -128,5 +129,10 @@ func (d *DockerComposeEnv) ExecOnLogtail(command string) (string, error) { } func (d *DockerComposeEnv) ExecOnSource(ctx context.Context, command string) (string, error) { - return "", fmt.Errorf("not implemented") + // exec on host of docker compose + fmt.Println(command) + cmd := exec.Command("sh", "-c", command) + output, err := cmd.CombinedOutput() + fmt.Println(string(output)) + return string(output), err } diff --git a/test/engine/steps.go b/test/engine/steps.go index 7b19db1988..336a5ff8ac 100644 --- a/test/engine/steps.go +++ b/test/engine/steps.go @@ -10,8 +10,11 @@ import ( "github.com/alibaba/ilogtail/test/engine/cleanup" "github.com/alibaba/ilogtail/test/engine/control" "github.com/alibaba/ilogtail/test/engine/setup" + "github.com/alibaba/ilogtail/test/engine/setup/monitor" "github.com/alibaba/ilogtail/test/engine/setup/subscriber" "github.com/alibaba/ilogtail/test/engine/trigger" + "github.com/alibaba/ilogtail/test/engine/trigger/ebpf" + "github.com/alibaba/ilogtail/test/engine/trigger/log" "github.com/alibaba/ilogtail/test/engine/verify" ) @@ -27,6 +30,7 @@ func ScenarioInitializer(ctx *godog.ScenarioContext) { ctx.Given(`^remove http config \{(.*)\}`, control.RemoveHTTPConfig) ctx.Given(`^subcribe data from \{(\S+)\} with config`, subscriber.InitSubscriber) ctx.Given(`^mkdir \{(.*)\}`, setup.Mkdir) + ctx.Given(`^docker-compose boot type \{(\S+)\}$`, setup.SetDockerComposeBootType) // ------------------------------------------ // When @@ -41,19 +45,25 @@ func ScenarioInitializer(ctx *godog.ScenarioContext) { // generate ctx.When(`^begin trigger`, trigger.BeginTrigger) - ctx.When(`^generate \{(\d+)\} regex logs to file \{(.*)\}, with interval \{(\d+)\}ms$`, trigger.RegexSingle) - ctx.When(`^generate \{(\d+)\} multiline regex logs to file \{(.*)\}, with interval \{(\d+)\}ms$`, trigger.RegexMultiline) - ctx.When(`^generate \{(\d+)\} regex gbk logs to file \{(.*)\}, with interval \{(\d+)\}ms$`, trigger.RegexSingleGBK) - ctx.When(`^generate \{(\d+)\} http logs, with interval \{(\d+)\}ms, url: \{(.*)\}, method: \{(.*)\}, body:`, trigger.HTTP) - ctx.When(`^generate \{(\d+)\} apsara logs to file \{(.*)\}, with interval \{(\d+)\}ms$`, trigger.Apsara) - ctx.When(`^generate \{(\d+)\} delimiter logs to file \{(.*)\}, with interval \{(\d+)\}ms, with delimiter \{(.*)\} and quote \{(.*)\}$`, trigger.DelimiterSingle) - ctx.When(`^generate \{(\d+)\} multiline delimiter logs to file \{(.*)\}, with interval \{(\d+)\}ms, with delimiter \{(.*)\} and quote \{(.*)\}$`, trigger.DelimiterMultiline) - ctx.When(`^generate \{(\d+)\} json logs to file \{(.*)\}, with interval \{(\d+)\}ms$`, trigger.JSONSingle) - ctx.When(`^generate \{(\d+)\} multiline json logs to file \{(.*)\}, with interval \{(\d+)\}ms$`, trigger.JSONMultiline) - ctx.When(`^execute \{(\d+)\} commands to generate process security events`, trigger.TrigerProcessSecurityEvents) - ctx.When(`^execute \{(\d+)\} commands to generate network security events on url \{(.*)\}$`, trigger.TrigerNetworksSecurityEvents) - ctx.When(`^execute \{(\d+)\} commands to generate file security events on files \{(.*)\}$`, trigger.TrigerFileSecurityEvents) - ctx.When(`^generate \{(\d+)\} HTTP requests, with interval \{(\d+)\}ms, url: \{(.*)\}`, trigger.TrigerHTTP) + // log + ctx.When(`^generate \{(\d+)\} regex logs to file \{(.*)\}, with interval \{(\d+)\}ms$`, log.RegexSingle) + ctx.When(`^generate \{(\d+)\} multiline regex logs to file \{(.*)\}, with interval \{(\d+)\}ms$`, log.RegexMultiline) + ctx.When(`^generate \{(\d+)\} regex gbk logs to file \{(.*)\}, with interval \{(\d+)\}ms$`, log.RegexSingleGBK) + ctx.When(`^generate \{(\d+)\} http logs, with interval \{(\d+)\}ms, url: \{(.*)\}, method: \{(.*)\}, body:`, log.HTTP) + ctx.When(`^generate \{(\d+)\} apsara logs to file \{(.*)\}, with interval \{(\d+)\}ms$`, log.Apsara) + ctx.When(`^generate \{(\d+)\} delimiter logs to file \{(.*)\}, with interval \{(\d+)\}ms, with delimiter \{(.*)\} and quote \{(.*)\}$`, log.DelimiterSingle) + ctx.When(`^generate \{(\d+)\} multiline delimiter logs to file \{(.*)\}, with interval \{(\d+)\}ms, with delimiter \{(.*)\} and quote \{(.*)\}$`, log.DelimiterMultiline) + ctx.When(`^generate \{(\d+)\} json logs to file \{(.*)\}, with interval \{(\d+)\}ms$`, log.JSONSingle) + ctx.When(`^generate \{(\d+)\} multiline json logs to file \{(.*)\}, with interval \{(\d+)\}ms$`, log.JSONMultiline) + ctx.When(`^generate random nginx logs to file, speed \{(\d+)\}MB/s, total \{(\d+)\}min, to file \{(.*)\}`, log.Nginx) + ctx.When(`^start monitor \{(\S+)\}`, monitor.StartMonitor) + ctx.When(`^wait monitor until log processing finished$`, monitor.WaitMonitorUntilProcessingFinished) + + // ebpf + ctx.When(`^execute \{(\d+)\} commands to generate process security events`, ebpf.ProcessSecurityEvents) + ctx.When(`^execute \{(\d+)\} commands to generate network security events on url \{(.*)\}$`, ebpf.NetworksSecurityEvents) + ctx.When(`^execute \{(\d+)\} commands to generate file security events on files \{(.*)\}$`, ebpf.FileSecurityEvents) + ctx.When(`^generate \{(\d+)\} HTTP requests, with interval \{(\d+)\}ms, url: \{(.*)\}`, ebpf.HTTP) // ------------------------------------------ // Then diff --git a/test/engine/trigger/protocol.go b/test/engine/trigger/ebpf/http.go similarity index 90% rename from test/engine/trigger/protocol.go rename to test/engine/trigger/ebpf/http.go index 646a4aa97b..c3c84f9ddd 100644 --- a/test/engine/trigger/protocol.go +++ b/test/engine/trigger/ebpf/http.go @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -package trigger +package ebpf import ( "context" @@ -22,7 +22,7 @@ import ( "github.com/alibaba/ilogtail/test/engine/setup" ) -func TrigerHTTP(ctx context.Context, count int, interval int, url string) (context.Context, error) { +func HTTP(ctx context.Context, count int, interval int, url string) (context.Context, error) { logger.Debugf(context.Background(), "count:%d interval:%d url:%s", count, interval, url) cmd := fmt.Sprintf("curl -vL %s", url) time.Sleep(time.Second * 5) diff --git a/test/engine/trigger/ebpf/remote_mmap.py b/test/engine/trigger/ebpf/remote_mmap.py new file mode 100644 index 0000000000..1a1efc5bd8 --- /dev/null +++ b/test/engine/trigger/ebpf/remote_mmap.py @@ -0,0 +1,22 @@ +import argparse +import mmap +import os + +def main(): + parser = argparse.ArgumentParser(description='mmap') + parser.add_argument('--commandCnt', type=int, default=10, help='command count') + parser.add_argument('--filename', type=str, default='/tmp/loongcollector/ebpfFileSecurityHook3.log', help='filename') + + args = parser.parse_args() + + with open(args.filename, 'w') as f: + fd = f.fileno() + for i in range(args.commandCnt): + mm = mmap.mmap(fd, 20, prot=mmap.PROT_READ | mmap.PROT_WRITE, flags=mmap.MAP_SHARED) + mm.close() + + os.remove(args.filename) + + +if __name__ == '__main__': + main() diff --git a/test/engine/trigger/ebpf_trigger.go b/test/engine/trigger/ebpf/security.go similarity index 76% rename from test/engine/trigger/ebpf_trigger.go rename to test/engine/trigger/ebpf/security.go index 4b824c557e..34bfc189c3 100644 --- a/test/engine/trigger/ebpf_trigger.go +++ b/test/engine/trigger/ebpf/security.go @@ -11,16 +11,16 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -package trigger +package ebpf import ( "context" - "html/template" + "strconv" "strings" "time" - "github.com/alibaba/ilogtail/test/config" "github.com/alibaba/ilogtail/test/engine/setup" + "github.com/alibaba/ilogtail/test/engine/trigger" ) /* @@ -28,7 +28,7 @@ import ( input_process_security ******************** */ -func TrigerProcessSecurityEvents(ctx context.Context, commandCnt int) (context.Context, error) { +func ProcessSecurityEvents(ctx context.Context, commandCnt int) (context.Context, error) { time.Sleep(5 * time.Second) if err := execveCommands(ctx, commandCnt); err != nil { return ctx, err @@ -51,7 +51,7 @@ func execveCommands(ctx context.Context, commandCnt int) error { input_network_security ******************** */ -func TrigerNetworksSecurityEvents(ctx context.Context, commandCnt int, url string) (context.Context, error) { +func NetworksSecurityEvents(ctx context.Context, commandCnt int, url string) (context.Context, error) { time.Sleep(5 * time.Second) if err := curlURL(ctx, commandCnt, url); err != nil { return ctx, err @@ -74,9 +74,8 @@ func curlURL(ctx context.Context, commandCnt int, url string) error { input_file_security ******************** */ -const triggerFileSecurityTemplate = "cd {{.WorkDir}} && COMMAND_CNT={{.CommandCnt}} FILE_NAME={{.FileName}} {{.Command}}" -func TrigerFileSecurityEvents(ctx context.Context, commandCnt int, filenames string) (context.Context, error) { +func FileSecurityEvents(ctx context.Context, commandCnt int, filenames string) (context.Context, error) { time.Sleep(5 * time.Second) if err := rwFile(ctx, commandCnt, filenames); err != nil { return ctx, err @@ -112,20 +111,10 @@ func rwFile(ctx context.Context, commandCnt int, filenames string) error { } func mmapFile(ctx context.Context, commandCnt int, filenames string) error { - mmapFileCommand := getRunTriggerCommand("TestGenerateMmapCommand") files := strings.Split(filenames, ",") for _, file := range files { - var triggerEBPFCommand strings.Builder - template := template.Must(template.New("trigger").Parse(triggerFileSecurityTemplate)) - if err := template.Execute(&triggerEBPFCommand, map[string]interface{}{ - "WorkDir": config.TestConfig.WorkDir, - "CommandCnt": commandCnt, - "FileName": file, - "Command": mmapFileCommand, - }); err != nil { - return err - } - if _, err := setup.Env.ExecOnSource(ctx, triggerEBPFCommand.String()); err != nil { + mmapFileCommand := trigger.GetRunTriggerCommand("ebpf", "mmap", "commandCnt", strconv.FormatInt(int64(commandCnt), 10), "filename", file) + if _, err := setup.Env.ExecOnSource(ctx, mmapFileCommand); err != nil { return err } } diff --git a/test/engine/trigger/file.go b/test/engine/trigger/file.go deleted file mode 100644 index 373aebbe52..0000000000 --- a/test/engine/trigger/file.go +++ /dev/null @@ -1,114 +0,0 @@ -package trigger - -import ( - "context" - "fmt" - "math/rand" - "os" - "path/filepath" - "time" - - "golang.org/x/time/rate" - - "github.com/alibaba/ilogtail/test/config" -) - -// JSON template -func GenerateRandomNginxLogToFile(ctx context.Context, speed, totalTime int, path string) (context.Context, error) { - - // clear file - path = filepath.Clean(path) - path = filepath.Join(config.CaseHome, path) - fmt.Println(path) - _ = os.WriteFile(path, []byte{}, 0600) - file, _ := os.OpenFile(path, os.O_APPEND|os.O_CREATE|os.O_WRONLY, 0600) // #nosec G304 - - rand.Seed(time.Now().UnixNano()) - maxLogLen := 1024 - nginxLog := genNginxLog() - - limiter := rate.NewLimiter(rate.Limit(speed*1024*1024), maxLogLen) - - timeout := time.After(time.Minute * time.Duration(totalTime)) - - for { - select { - // context is done - case <-ctx.Done(): - // clear file - _ = file.Close() - return ctx, nil - // all time is done - case <-timeout: - // clear file - _ = file.Close() - return ctx, nil - default: - if limiter.AllowN(time.Now(), len(nginxLog)) { - _, _ = file.WriteString(nginxLog + "\n") // #nosec G307 - nginxLog = genNginxLog() - } - } - } -} - -var ipAddresses = []string{ - "103.159.151.180", - "12.55.18.241", - "182.233.128.102", - "221.85.57.231", - "76.245.65.224", - "86.250.231.93", - "44.201.253.252", - "218.7.2.219", - "172.118.174.109", - "208.16.46.154", - "7.138.80.41", - "214.73.25.80", - "83.124.20.79", - "80.226.48.153", - "92.129.204.161", - "212.103.145.159", - "148.188.8.90", - "148.212.244.121", - "106.186.172.157", - "30.127.196.158", -} - -var userAgents = []string{ - "aliyun-sdk-java", - "aliyun-sdk-golang", - "aliyun-sdk-python", -} - -var statusCodes = []string{ - "400", - "401", - "402", - "403", - "404", - "200", -} - -const bytesMean = 5500.0 -const bytesStddev = 1500.0 - -func genNginxLog() string { - nginxLogTemplate := `%s - - [%s] "GET http://www.districtdot-com.biz/syndicate HTTP/1.1" %s %d "http://www.chiefscalable.biz/webservices" "%s"` - currentTime := time.Now().Format("02/Jan/2006:15:04:05 +0800") - ipAddress := ipAddresses[rand.Intn(len(ipAddresses))] // #nosec G404 - statusIdx := rand.Intn(len(statusCodes) * 10) // #nosec G404 - if statusIdx >= len(statusCodes) { - statusIdx = len(statusCodes) - 1 - } - bytesSize := int32(rand.NormFloat64()*bytesStddev + bytesMean) - if bytesSize < 1000 { - bytesSize = 0 - } else if bytesSize > 10000 { - bytesSize = 10000 - } - statusCode := statusCodes[statusIdx] - userAgent := userAgents[rand.Intn(len(userAgents))] // #nosec G404 - - return fmt.Sprintf(nginxLogTemplate, ipAddress, currentTime, statusCode, bytesSize, userAgent) -} diff --git a/test/engine/trigger/generator/apsara_test.go b/test/engine/trigger/generator/apsara_test.go deleted file mode 100644 index 40329c0cfe..0000000000 --- a/test/engine/trigger/generator/apsara_test.go +++ /dev/null @@ -1,73 +0,0 @@ -// Copyright 2024 iLogtail Authors -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -package generator - -import ( - "fmt" - "math/rand" - "os" - "strconv" - "testing" - "time" -) - -// TestGenerateApsara will be executed in the environment being collected. -func TestGenerateApsara(t *testing.T) { - config, err := getGenerateFileLogConfigFromEnv() - if err != nil { - t.Fatalf("get generate file log config from env failed: %v", err) - return - } - testLogContentTmpl := string2Template([]string{ - "[{{.Time}}]\t[{{.Level}}]\t[32337]\t[/build/core/application/Application:12]\tfile:file{{.FileNo}}\tlogNo:{{.LogNo}}\tmark:{{.Mark}}\tmsg:hello world!\n", - "[{{.Time}}]\t[{{.Level}}]\t[20964]\t[/build/core/ilogtail.cpp:127]\tfile:file{{.FileNo}}\tlogNo:{{.LogNo}}\tmark:{{.Mark}}\tmsg:这是一条消息\n", - "[{{.Time}}]\t[{{.Level}}]\t[32337]\t[/build/core/ilogtail.cpp:127]\tfile:file{{.FileNo}}\tlogNo:{{.LogNo}}\tmark:{{.Mark}}\tmsg:hello world!\n", - "[{{.Time}}]\t[{{.Level}}]\t[32337]\t[/build/core/ilogtail.cpp:127]\tfile:file{{.FileNo}}\tlogNo:{{.LogNo}}\tmark:{{.Mark}}\tmsg:这是一条消息\n", - "[{{.Time}}]\t[{{.Level}}]\t[00001]\t[/build/core/ilogtail.cpp:127]\tfile:file{{.FileNo}}\tlogNo:{{.LogNo}}\tmark:{{.Mark}}\tmsg:password:123456\n", - }) - file, err := os.OpenFile(fmt.Sprintf("%s/%s", config.GeneratedLogDir, config.FileName), os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0644) - if err != nil { - t.Fatalf("open file failed: %v", err) - return - } - defer file.Close() - - logIndex := 0 - logNo := rand.Intn(10000) - fileNo := rand.Intn(10000) - for i := 0; i < config.TotalLog; i++ { - var currentTime string - if i%2 == 0 { - currentTime = time.Now().Format("2006-01-02 15:04:05.000000") - } else { - currentTime = strconv.FormatInt(time.Now().UnixNano()/1000, 10) - } - err = testLogContentTmpl[logIndex].Execute(file, map[string]interface{}{ - "Time": currentTime, - "Level": getRandomLogLevel(), - "LogNo": logNo + i, - "FileNo": fileNo, - "Mark": getRandomMark(), - }) - if err != nil { - t.Fatalf("write log failed: %v", err) - return - } - time.Sleep(time.Duration(config.Interval * int(time.Millisecond))) - logIndex++ - if logIndex >= len(testLogContentTmpl) { - logIndex = 0 - } - } -} diff --git a/test/engine/trigger/generator/delimiter_test.go b/test/engine/trigger/generator/delimiter_test.go deleted file mode 100644 index 354b6dea8c..0000000000 --- a/test/engine/trigger/generator/delimiter_test.go +++ /dev/null @@ -1,127 +0,0 @@ -// Copyright 2024 iLogtail Authors -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -package generator - -import ( - "fmt" - "math/rand" - "os" - "testing" - "time" -) - -// TestGenerateDelimiterSingle will be executed in the environment being collected. -func TestGenerateDelimiterSingle(t *testing.T) { - config, err := getGenerateFileLogConfigFromEnv("Delimiter", "Quote") - if err != nil { - t.Fatalf("get generate file log config from env failed: %v", err) - return - } - delimiter := config.Custom["Delimiter"] - if delimiter == "" { - delimiter = " " - } - quote := config.Custom["Quote"] - if quote == "" { - quote = "" - } - testLogContentTmpl := string2Template([]string{ - "{{.Quote}}{{.Mark}}{{.Quote}}{{.Delimiter}}{{.Quote}}file{{.FileNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}{{.LogNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}0.0.0.0{{.Quote}}{{.Delimiter}}{{.Quote}}{{.Time}}{{.Quote}}{{.Delimiter}}{{.Quote}}GET{{.Quote}}{{.Delimiter}}{{.Quote}}/index.html{{.Quote}}{{.Delimiter}}{{.Quote}}HTTP/2.0{{.Quote}}{{.Delimiter}}{{.Quote}}302{{.Quote}}{{.Delimiter}}{{.Quote}}628{{.Quote}}{{.Delimiter}}{{.Quote}}curl/7.10{{.Quote}}\n", - "{{.Quote}}{{.Mark}}{{.Quote}}{{.Delimiter}}{{.Quote}}file{{.FileNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}{{.LogNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}10.45.26.0{{.Quote}}{{.Delimiter}}{{.Quote}}{{.Time}}{{.Quote}}{{.Delimiter}}{{.Quote}}GET{{.Quote}}{{.Delimiter}}{{.Quote}}/{{.Quote}}{{.Delimiter}}{{.Quote}}HTTP/2.0{{.Quote}}{{.Delimiter}}{{.Quote}}302{{.Quote}}{{.Delimiter}}{{.Quote}}218{{.Quote}}{{.Delimiter}}{{.Quote}}go-sdk{{.Quote}}\n", - "{{.Quote}}{{.Mark}}{{.Quote}}{{.Delimiter}}{{.Quote}}file{{.FileNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}{{.LogNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}10.45.26.0{{.Quote}}{{.Delimiter}}{{.Quote}}{{.Time}}{{.Quote}}{{.Delimiter}}{{.Quote}}GET{{.Quote}}{{.Delimiter}}{{.Quote}}/dir/resource.txt{{.Quote}}{{.Delimiter}}{{.Quote}}HTTP/1.1{{.Quote}}{{.Delimiter}}{{.Quote}}404{{.Quote}}{{.Delimiter}}{{.Quote}}744{{.Quote}}{{.Delimiter}}{{.Quote}}Mozilla/5.0{{.Quote}}\n", - "{{.Quote}}{{.Mark}}{{.Quote}}{{.Delimiter}}{{.Quote}}file{{.FileNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}{{.LogNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}127.0.0.1{{.Quote}}{{.Delimiter}}{{.Quote}}{{.Time}}{{.Quote}}{{.Delimiter}}{{.Quote}}PUT{{.Quote}}{{.Delimiter}}{{.Quote}}/{{.Quote}}{{.Delimiter}}{{.Quote}}HTTP/2.0{{.Quote}}{{.Delimiter}}{{.Quote}}200{{.Quote}}{{.Delimiter}}{{.Quote}}320{{.Quote}}{{.Delimiter}}{{.Quote}}curl/7.10{{.Quote}}\n", - "{{.Quote}}{{.Mark}}{{.Quote}}{{.Delimiter}}{{.Quote}}file{{.FileNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}{{.LogNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}192.168.0.3{{.Quote}}{{.Delimiter}}{{.Quote}}{{.Time}}{{.Quote}}{{.Delimiter}}{{.Quote}}PUT{{.Quote}}{{.Delimiter}}{{.Quote}}/dir/resource.txt{{.Quote}}{{.Delimiter}}{{.Quote}}HTTP/1.1{{.Quote}}{{.Delimiter}}{{.Quote}}404{{.Quote}}{{.Delimiter}}{{.Quote}}949{{.Quote}}{{.Delimiter}}{{.Quote}}curl/7.10{{.Quote}}\n", - }) - file, err := os.OpenFile(fmt.Sprintf("%s/%s", config.GeneratedLogDir, config.FileName), os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0644) - if err != nil { - t.Fatalf("open file failed: %v", err) - return - } - defer file.Close() - - logIndex := 0 - logNo := rand.Intn(10000) - fileNo := rand.Intn(10000) - for i := 0; i < config.TotalLog; i++ { - err = testLogContentTmpl[logIndex].Execute(file, map[string]interface{}{ - "Mark": getRandomMark(), - "FileNo": fileNo, - "LogNo": logNo, - "Time": time.Now().Format("2006-01-02 15:04:05.000000000"), - "Delimiter": delimiter, - "Quote": quote, - }) - if err != nil { - t.Fatalf("write log failed: %v", err) - return - } - time.Sleep(time.Duration(config.Interval * int(time.Millisecond))) - logIndex++ - if logIndex >= len(testLogContentTmpl) { - logIndex = 0 - } - } -} - -// TestGenerateDelimiterMultiline will be executed in the environment being collected. -func TestGenerateDelimiterMultiline(t *testing.T) { - config, err := getGenerateFileLogConfigFromEnv("Delimiter", "Quote") - if err != nil { - t.Fatalf("get generate file log config from env failed: %v", err) - return - } - delimiter := config.Custom["Delimiter"] - if delimiter == "" { - delimiter = " " - } - quote := config.Custom["Quote"] - if quote == "" { - quote = "" - } - testLogContentTmpl := string2Template([]string{ - "{{.Quote}}F{{.Quote}}{{.Delimiter}}{{.Quote}}file{{.FileNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}{{.LogNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}0.0.0.0{{.Quote}}{{.Delimiter}}{{.Quote}}{{.Time}}{{.Quote}}{{.Delimiter}}{{.Quote}}GET{{.Quote}}{{.Delimiter}}{{.Quote}}/index.html{{.Quote}}{{.Delimiter}}{{.Quote}}\nHTTP\n/2.0{{.Quote}}{{.Delimiter}}{{.Quote}}302{{.Quote}}{{.Delimiter}}{{.Quote}}628{{.Quote}}{{.Delimiter}}{{.Quote}}curl/7.10{{.Quote}}\n", - "{{.Quote}}-{{.Quote}}{{.Delimiter}}{{.Quote}}file{{.FileNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}{{.LogNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}10.45.26.0{{.Quote}}{{.Delimiter}}{{.Quote}}{{.Time}}{{.Quote}}{{.Delimiter}}{{.Quote}}GET{{.Quote}}{{.Delimiter}}{{.Quote}}/{{.Quote}}{{.Delimiter}}{{.Quote}}\nHTTP\n/2.0{{.Quote}}{{.Delimiter}}{{.Quote}}302{{.Quote}}{{.Delimiter}}{{.Quote}}218{{.Quote}}{{.Delimiter}}{{.Quote}}go-sdk{{.Quote}}\n", - "{{.Quote}}F{{.Quote}}{{.Delimiter}}{{.Quote}}file{{.FileNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}{{.LogNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}10.45.26.0{{.Quote}}{{.Delimiter}}{{.Quote}}{{.Time}}{{.Quote}}{{.Delimiter}}{{.Quote}}GET{{.Quote}}{{.Delimiter}}{{.Quote}}/dir/resource.txt{{.Quote}}{{.Delimiter}}{{.Quote}}\nHTTP\n/1.1{{.Quote}}{{.Delimiter}}{{.Quote}}404{{.Quote}}{{.Delimiter}}{{.Quote}}744{{.Quote}}{{.Delimiter}}{{.Quote}}Mozilla/5.0{{.Quote}}\n", - "{{.Quote}}-{{.Quote}}{{.Delimiter}}{{.Quote}}file{{.FileNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}{{.LogNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}127.0.0.1{{.Quote}}{{.Delimiter}}{{.Quote}}{{.Time}}{{.Quote}}{{.Delimiter}}{{.Quote}}PUT{{.Quote}}{{.Delimiter}}{{.Quote}}/{{.Quote}}{{.Delimiter}}{{.Quote}}\nHTTP\n/2.0{{.Quote}}{{.Delimiter}}{{.Quote}}200{{.Quote}}{{.Delimiter}}{{.Quote}}320{{.Quote}}{{.Delimiter}}{{.Quote}}curl/7.10{{.Quote}}\n", - "{{.Quote}}F{{.Quote}}{{.Delimiter}}{{.Quote}}file{{.FileNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}{{.LogNo}}{{.Quote}}{{.Delimiter}}{{.Quote}}192.168.0.3{{.Quote}}{{.Delimiter}}{{.Quote}}{{.Time}}{{.Quote}}{{.Delimiter}}{{.Quote}}PUT{{.Quote}}{{.Delimiter}}{{.Quote}}/dir/resource.txt{{.Quote}}{{.Delimiter}}{{.Quote}}\nHTTP\n/1.1{{.Quote}}{{.Delimiter}}{{.Quote}}404{{.Quote}}{{.Delimiter}}{{.Quote}}949{{.Quote}}{{.Delimiter}}{{.Quote}}curl/7.10{{.Quote}}\n", - }) - file, err := os.OpenFile(fmt.Sprintf("%s/%s", config.GeneratedLogDir, config.FileName), os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0644) - if err != nil { - t.Fatalf("open file failed: %v", err) - return - } - defer file.Close() - - logIndex := 0 - logNo := rand.Intn(10000) - fileNo := rand.Intn(10000) - for i := 0; i < config.TotalLog; i++ { - err = testLogContentTmpl[logIndex].Execute(file, map[string]interface{}{ - "FileNo": fileNo, - "LogNo": logNo, - "Time": time.Now().Format("2006-01-02 15:04:05.000000000"), - "Delimiter": delimiter, - "Quote": quote, - }) - if err != nil { - t.Fatalf("write log failed: %v", err) - return - } - time.Sleep(time.Duration(config.Interval * int(time.Millisecond))) - logIndex++ - if logIndex >= len(testLogContentTmpl) { - logIndex = 0 - } - } -} diff --git a/test/engine/trigger/generator/ebpf_file_mmap_test.go b/test/engine/trigger/generator/ebpf_file_mmap_test.go deleted file mode 100644 index 56e221b54d..0000000000 --- a/test/engine/trigger/generator/ebpf_file_mmap_test.go +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright 2024 iLogtail Authors -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -package generator - -import ( - "os" - "strconv" - "syscall" - "testing" -) - -func TestGenerateMmapCommand(t *testing.T) { - commandCnt := getEnvOrDefault("COMMAND_CNT", "10") - commandCntNum, err := strconv.Atoi(commandCnt) - if err != nil { - t.Fatalf("parse COMMAND_CNT failed: %v", err) - return - } - filename := getEnvOrDefault("FILE_NAME", "/tmp/loongcollector/ebpfFileSecurityHook3.log") - f, err := os.Create(filename) - if err != nil { - panic(err) - } - fd := int(f.Fd()) - for i := 0; i < commandCntNum; i++ { - b, innerErr := syscall.Mmap(fd, 0, 20, syscall.PROT_READ|syscall.PROT_WRITE, syscall.MAP_SHARED) - if innerErr != nil { - panic(innerErr) - } - innerErr = syscall.Munmap(b) - if innerErr != nil { - panic(innerErr) - } - } - err = os.Remove(filename) - if err != nil { - t.Fatalf("remove file failed: %v", err) - return - } -} diff --git a/test/engine/trigger/generator/helper.go b/test/engine/trigger/generator/helper.go deleted file mode 100644 index 2514bb2614..0000000000 --- a/test/engine/trigger/generator/helper.go +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright 2024 iLogtail Authors -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -package generator - -import ( - "crypto/rand" - "fmt" - "math/big" - "os" - "strconv" - "text/template" - - "github.com/pkg/errors" -) - -var Levels = []string{"ERROR", "INFO", "DEBUG", "WARNING"} - -type GenerateFileLogConfig struct { - GeneratedLogDir string - TotalLog int - Interval int - FileName string - Custom map[string]string -} - -func getGenerateFileLogConfigFromEnv(customKeys ...string) (*GenerateFileLogConfig, error) { - gneratedLogDir := getEnvOrDefault("GENERATED_LOG_DIR", "/tmp/loongcollector") - totalLog, err := strconv.Atoi(getEnvOrDefault("TOTAL_LOG", "100")) - if err != nil { - return nil, errors.Wrap(err, "parse TOTAL_LOG failed") - } - interval, err := strconv.Atoi(getEnvOrDefault("INTERVAL", "1")) - if err != nil { - return nil, errors.Wrap(err, "parse INTERVAL failed") - } - fileName := getEnvOrDefault("FILENAME", "default.log") - custom := make(map[string]string) - for _, key := range customKeys { - custom[key] = getEnvOrDefault(key, "") - } - return &GenerateFileLogConfig{ - GeneratedLogDir: gneratedLogDir, - TotalLog: totalLog, - Interval: interval, - FileName: fileName, - Custom: custom, - }, nil -} - -func string2Template(strings []string) []*template.Template { - templates := make([]*template.Template, len(strings)) - for i, str := range strings { - templates[i], _ = template.New(fmt.Sprintf("template_%d", i)).Parse(str) - } - return templates -} - -func getRandomLogLevel() string { - randInt, _ := rand.Int(rand.Reader, big.NewInt(int64(len(Levels)))) - return Levels[randInt.Int64()] -} - -func getRandomMark() string { - marks := []string{"-", "F"} - randInt, _ := rand.Int(rand.Reader, big.NewInt(int64(len(marks)))) - return marks[randInt.Int64()] -} - -func getEnvOrDefault(env, fallback string) string { - if value, ok := os.LookupEnv(env); ok { - return value - } - return fallback -} diff --git a/test/engine/trigger/generator/json_test.go b/test/engine/trigger/generator/json_test.go deleted file mode 100644 index 0b4ef51b33..0000000000 --- a/test/engine/trigger/generator/json_test.go +++ /dev/null @@ -1,119 +0,0 @@ -// Copyright 2024 iLogtail Authors -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -package generator - -import ( - "fmt" - "math/rand" - "os" - "strconv" - "testing" - "time" -) - -// TestGenerateJSONSingle will be executed in the environment being collected. -func TestGenerateJSONSingle(t *testing.T) { - config, err := getGenerateFileLogConfigFromEnv() - if err != nil { - t.Fatalf("get generate file log config from env failed: %v", err) - return - } - testLogContentTmpl := string2Template([]string{ - `{"mark":"{{.Mark}}","file":"file{{.FileNo}}","logNo":{{.LogNo}},"time":"{{.Time}}","ip":"0.0.0.0","method":"POST","userAgent":"mozilla firefox","size":263} -`, - `{"mark":"{{.Mark}}","file":"file{{.FileNo}}","logNo":{{.LogNo}},"time":"{{.Time}}","ip":"0.0.0.0","method":"GET","userAgent":"go-sdk","size":569} -`, - `{"mark":"{{.Mark}}","file":"file{{.FileNo}}","logNo":{{.LogNo}},"time":"{{.Time}}","ip":"0.0.0.0","method":"HEAD","userAgent":"go-sdk","size":210} -`, - `{"mark":"{{.Mark}}","file":"file{{.FileNo}}","logNo":{{.LogNo}},"time":"{{.Time}}","ip":"192.168.0.3","method":"PUT","userAgent":"curl/7.10","size":267} -`, - }) - file, err := os.OpenFile(fmt.Sprintf("%s/%s", config.GeneratedLogDir, config.FileName), os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0644) - if err != nil { - t.Fatalf("open file failed: %v", err) - return - } - defer file.Close() - - logIndex := 0 - logNo := rand.Intn(10000) - fileNo := rand.Intn(10000) - for i := 0; i < config.TotalLog; i++ { - var currentTime string - if i%2 == 0 { - currentTime = time.Now().Format("2006-01-02T15:04:05.999999999") - } else { - currentTime = strconv.FormatInt(time.Now().UnixNano()/1000, 10) - } - testLogContentTmpl[logIndex].Execute(file, map[string]interface{}{ - "Mark": getRandomMark(), - "FileNo": fileNo, - "LogNo": logNo + i, - "Time": currentTime, - }) - - time.Sleep(time.Duration(config.Interval * int(time.Millisecond))) - logIndex++ - if logIndex >= len(testLogContentTmpl) { - logIndex = 0 - } - } -} - -func TestGenerateJSONMultiline(t *testing.T) { - config, err := getGenerateFileLogConfigFromEnv() - if err != nil { - t.Fatalf("get generate file log config from env failed: %v", err) - return - } - testLogContentTmpl := string2Template([]string{ - `{"mark":"{{.Mark}}","file":"file{{.FileNo}}","logNo":{{.LogNo}},"time":"{{.Time}}","ip":"0.0.0.0","method":"POST","userAgent":"mozilla firefox", -"size":263} -`, - `{"mark":"{{.Mark}}","file":"file{{.FileNo}}","logNo":{{.LogNo}},"time":"{{.Time}}","ip":"0.0.0.0","method":"GET","userAgent":"go-sdk", -"size":569} -`, - `{"mark":"{{.Mark}}","file":"file{{.FileNo}}","logNo":{{.LogNo}},"time":"{{.Time}}","ip":"0.0.0.0","method":"HEAD","userAgent":"go-sdk", -"size":210} -`, - `{"mark":"{{.Mark}}","file":"file{{.FileNo}}","logNo":{{.LogNo}},"time":"{{.Time}}","ip":"192.168.0.3","method":"PUT","userAgent":"curl/7.10", -"size":267} -`, - }) - file, err := os.OpenFile(fmt.Sprintf("%s/%s", config.GeneratedLogDir, config.FileName), os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0644) - if err != nil { - t.Fatalf("open file failed: %v", err) - return - } - defer file.Close() - - logIndex := 0 - logNo := rand.Intn(10000) - fileNo := rand.Intn(10000) - for i := 0; i < config.TotalLog; i++ { - currentTime := time.Now().Format("2006-01-02T15:04:05.999999999") - testLogContentTmpl[logIndex].Execute(file, map[string]interface{}{ - "Mark": getRandomMark(), - "FileNo": fileNo, - "LogNo": logNo + i, - "Time": currentTime, - }) - - time.Sleep(time.Duration(config.Interval * int(time.Millisecond))) - logIndex++ - if logIndex >= len(testLogContentTmpl) { - logIndex = 0 - } - } -} diff --git a/test/engine/trigger/generator/regex_test.go b/test/engine/trigger/generator/regex_test.go deleted file mode 100644 index c50e65dc6e..0000000000 --- a/test/engine/trigger/generator/regex_test.go +++ /dev/null @@ -1,198 +0,0 @@ -// Copyright 2024 iLogtail Authors -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -package generator - -import ( - "bytes" - "fmt" - "io" - "math/rand" - "os" - "testing" - "time" - - "golang.org/x/text/encoding/simplifiedchinese" - "golang.org/x/text/transform" -) - -// TestGenerateRegexLogSingle will be executed in the environment being collected. -func TestGenerateRegexLogSingle(t *testing.T) { - config, err := getGenerateFileLogConfigFromEnv() - if err != nil { - t.Fatalf("get config failed: %v", err) - return - } - testLogContentTmpl := string2Template([]string{ - `{{.Mark}} file{{.FileNo}}:{{.LogNo}} 127.0.0.1 - [{{.Time}}] "HEAD / HTTP/2.0" 302 809 "未知" "这是一条消息,password:123456" -`, - `{{.Mark}} file{{.FileNo}}:{{.LogNo}} 127.0.0.1 - [{{.Time}}] "GET /index.html HTTP/2.0" 200 139 "Mozilla/5.0" "这是一条消息,password:123456,这是第二条消息,password:00000" -`, - `{{.Mark}} file{{.FileNo}}:{{.LogNo}} 10.45.26.0 - [{{.Time}}] "PUT /index.html HTTP/1.1" 200 913 "curl/7.10" "这是一条消息" -`, - `{{.Mark}} file{{.FileNo}}:{{.LogNo}} 192.168.0.3 - [{{.Time}}] "PUT /dir/resource.txt HTTP/2.0" 501 355 "go-sdk" "这是一条消息,password:123456" -`, - }) - file, err := os.OpenFile(fmt.Sprintf("%s/%s", config.GeneratedLogDir, config.FileName), os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0644) - if err != nil { - t.Fatalf("open file failed: %v", err) - return - } - defer file.Close() - - logIndex := 0 - logNo := rand.Intn(10000) - fileNo := rand.Intn(10000) - location, err := time.LoadLocation("Asia/Shanghai") - if err != nil { - t.Fatalf("load location failed: %v", err) - return - } - for i := 0; i < config.TotalLog; i++ { - err = testLogContentTmpl[logIndex].Execute(file, map[string]interface{}{ - "Time": time.Now().In(location).Format("2006-01-02T15:04:05.000000"), - "Mark": getRandomMark(), - "FileNo": fileNo, - "LogNo": logNo + i, - }) - if err != nil { - t.Fatalf("write log failed: %v", err) - return - } - time.Sleep(time.Duration(config.Interval * int(time.Millisecond))) - logIndex++ - if logIndex >= len(testLogContentTmpl) { - logIndex = 0 - } - } -} - -// TestGenerateRegexLogSingleGBK will be executed in the environment being collected. -func TestGenerateRegexLogSingleGBK(t *testing.T) { - config, err := getGenerateFileLogConfigFromEnv() - if err != nil { - t.Fatalf("get config failed: %v", err) - return - } - encoder := simplifiedchinese.GBK.NewEncoder() - testLogContentTmpl := string2Template([]string{ - `{{.Mark}} file{{.FileNo}}:{{.LogNo}} 127.0.0.1 - [{{.Time}}] "HEAD / HTTP/2.0" 302 809 "未知" "这是一条消息,password:123456" -`, - `{{.Mark}} file{{.FileNo}}:{{.LogNo}} 127.0.0.1 - [{{.Time}}] "GET /index.html HTTP/2.0" 200 139 "Mozilla/5.0" "这是一条消息,password:123456,这是第二条消息,password:00000" -`, - `{{.Mark}} file{{.FileNo}}:{{.LogNo}} 10.45.26.0 - [{{.Time}}] "PUT /index.html HTTP/1.1" 200 913 "curl/7.10" "这是一条消息" -`, - `{{.Mark}} file{{.FileNo}}:{{.LogNo}} 192.168.0.3 - [{{.Time}}] "PUT /dir/resource.txt HTTP/2.0" 501 355 "go-sdk" "这是一条消息,password:123456" -`, - }) - file, err := os.OpenFile(fmt.Sprintf("%s/%s", config.GeneratedLogDir, config.FileName), os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0644) - if err != nil { - t.Fatalf("open file failed: %v", err) - return - } - defer file.Close() - - logIndex := 0 - logNo := rand.Intn(10000) - fileNo := rand.Intn(10000) - location, err := time.LoadLocation("Asia/Shanghai") - if err != nil { - t.Fatalf("load location failed: %v", err) - return - } - for i := 0; i < config.TotalLog; i++ { - var buffer bytes.Buffer - _ = testLogContentTmpl[logIndex].Execute(&buffer, map[string]interface{}{ - "Time": time.Now().In(location).Format("2006-01-02T15:04:05.000000"), - "Mark": getRandomMark(), - "FileNo": fileNo, - "LogNo": logNo + i, - }) - data, err1 := io.ReadAll(transform.NewReader(&buffer, encoder)) - if err1 != nil { - t.Fatalf("encode log failed: %v", err1) - } - _, err := io.WriteString(file, string(data)) - if err != nil { - t.Fatalf("write log failed: %v", err) - return - } - time.Sleep(time.Duration(config.Interval * int(time.Millisecond))) - logIndex++ - if logIndex >= len(testLogContentTmpl) { - logIndex = 0 - } - } -} - -func TestGenerateRegexLogMultiline(t *testing.T) { - config, err := getGenerateFileLogConfigFromEnv() - if err != nil { - t.Fatalf("get config failed: %v", err) - return - } - testLogContentTmpl := string2Template([]string{ - `{{.Mark}} file{{.FileNo}}:{{.LogNo}} [{{.Time}}] [{{.Level}}] java.lang.Exception: exception happened -at com.aliyun.sls.devops.logGenerator.type.RegexMultiLog.f1(RegexMultiLog.java:73) -at com.aliyun.sls.devops.logGenerator.type.RegexMultiLog.run(RegexMultiLog.java:34) -at java.base/java.lang.Thread.run(Thread.java:833) -`, - `{{.Mark}} file{{.FileNo}}:{{.LogNo}} [{{.Time}}] [{{.Level}}] java.lang.Exception: 发生异常 -at com.aliyun.sls.devops.logGenerator.type.RegexMultiLog.f2(RegexMultiLog.java:80) -at com.aliyun.sls.devops.logGenerator.type.RegexMultiLog.f1(RegexMultiLog.java:75) -at com.aliyun.sls.devops.logGenerator.type.RegexMultiLog.run(RegexMultiLog.java:34) -at java.base/java.lang.Thread.run(Thread.java:833) -`, - `{{.Mark}} file{{.FileNo}}:{{.LogNo}} [{{.Time}}] [{{.Level}}] java.lang.Exception: exception happened -at com.aliyun.sls.devops.logGenerator.type.RegexMultiLog.f5(RegexMultiLog.java:100) -at com.aliyun.sls.devops.logGenerator.type.RegexMultiLog.f4(RegexMultiLog.java:96) -at com.aliyun.sls.devops.logGenerator.type.RegexMultiLog.f3(RegexMultiLog.java:89) -at com.aliyun.sls.devops.logGenerator.type.RegexMultiLog.f2(RegexMultiLog.java:82) -at com.aliyun.sls.devops.logGenerator.type.RegexMultiLog.f1(RegexMultiLog.java:75) -at com.aliyun.sls.devops.logGenerator.type.RegexMultiLog.run(RegexMultiLog.java:34) -at java.base/java.lang.Thread.run(Thread.java:833) -`, - }) - file, err := os.OpenFile(fmt.Sprintf("%s/%s", config.GeneratedLogDir, config.FileName), os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0644) - if err != nil { - t.Fatalf("open file failed: %v", err) - return - } - defer file.Close() - logIndex := 0 - logNo := rand.Intn(10000) - fileNo := rand.Intn(10000) - location, err := time.LoadLocation("Asia/Shanghai") - if err != nil { - t.Fatalf("load location failed: %v", err) - return - } - for i := 0; i < config.TotalLog; i++ { - err = testLogContentTmpl[logIndex].Execute(file, map[string]interface{}{ - "Time": time.Now().In(location).Format("2006-01-02T15:04:05.000000"), - "Level": getRandomLogLevel(), - "FileNo": fileNo, - "LogNo": logNo + i, - "Mark": getRandomMark(), - }) - if err != nil { - t.Fatalf("write log failed: %v", err) - return - } - time.Sleep(time.Duration(config.Interval * int(time.Millisecond))) - logIndex++ - if logIndex >= len(testLogContentTmpl) { - logIndex = 0 - } - } -} diff --git a/test/engine/trigger/helper.go b/test/engine/trigger/helper.go index 8e0fb12171..696c63d0b2 100644 --- a/test/engine/trigger/helper.go +++ b/test/engine/trigger/helper.go @@ -14,11 +14,26 @@ package trigger import ( + "context" "fmt" + "path/filepath" + "strings" + "time" + + "github.com/alibaba/ilogtail/test/config" ) -const commandTemplate = "/usr/local/go/bin/go test -count=1 -v -run ^%s$ github.com/alibaba/ilogtail/test/engine/trigger/generator" +const commandTemplate = "python3 %s.py %s" + +func BeginTrigger(ctx context.Context) (context.Context, error) { + startTime := time.Now().Unix() + return context.WithValue(ctx, config.StartTimeContextKey, int32(startTime)), nil +} -func getRunTriggerCommand(triggerName string) string { - return fmt.Sprintf(commandTemplate, triggerName) +func GetRunTriggerCommand(scenrio, triggerName string, kvs ...string) string { + args := make([]string, 0) + for i := 0; i < len(kvs); i += 2 { + args = append(args, fmt.Sprintf("--%s", kvs[i]), kvs[i+1]) + } + return fmt.Sprintf(commandTemplate, filepath.Join(config.TestConfig.WorkDir, "engine", "trigger", scenrio, "remote_"+triggerName), strings.Join(args, " ")) } diff --git a/test/engine/trigger/log/file.go b/test/engine/trigger/log/file.go new file mode 100644 index 0000000000..ddea428021 --- /dev/null +++ b/test/engine/trigger/log/file.go @@ -0,0 +1,105 @@ +// Copyright 2024 iLogtail Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package log + +import ( + "context" + "encoding/json" + "fmt" + "strconv" + "strings" + "time" + + "github.com/alibaba/ilogtail/test/engine/setup" + "github.com/alibaba/ilogtail/test/engine/trigger" +) + +func RegexSingle(ctx context.Context, totalLog int, path string, interval int) (context.Context, error) { + return generate(ctx, "regex", path, totalLog, interval) +} + +func RegexSingleGBK(ctx context.Context, totalLog int, path string, interval int) (context.Context, error) { + return generate(ctx, "regexGBK", path, totalLog, interval) +} + +func RegexMultiline(ctx context.Context, totalLog int, path string, interval int) (context.Context, error) { + return generate(ctx, "regexMultiline", path, totalLog, interval) +} + +func JSONSingle(ctx context.Context, totalLog int, path string, interval int) (context.Context, error) { + return generate(ctx, "json", path, totalLog, interval) +} + +func JSONMultiline(ctx context.Context, totalLog int, path string, interval int) (context.Context, error) { + return generate(ctx, "jsonMultiline", path, totalLog, interval) +} + +func Apsara(ctx context.Context, totalLog int, path string, interval int) (context.Context, error) { + return generate(ctx, "apsara", path, totalLog, interval) +} + +func DelimiterSingle(ctx context.Context, totalLog int, path string, interval int, delimiter, quote string) (context.Context, error) { + return generate(ctx, "delimiter", path, totalLog, interval, "delimiter", delimiter, "quote", quote) +} + +func DelimiterMultiline(ctx context.Context, totalLog int, path string, interval int, delimiter, quote string) (context.Context, error) { + return generate(ctx, "delimiterMultiline", path, totalLog, interval, "delimiter", delimiter, "quote", quote) +} + +func Nginx(ctx context.Context, rate, duration int, path string) (context.Context, error) { + return generateBenchmark(ctx, "nginx", path, rate, duration) +} + +func generate(ctx context.Context, mode, path string, count, interval int, customKV ...string) (context.Context, error) { + time.Sleep(3 * time.Second) + customKVString := make(map[string]string) + for i := 0; i < len(customKV); i += 2 { + customKVString[customKV[i]] = customKV[i+1] + } + jsonStr, err := json.Marshal(customKVString) + if err != nil { + return ctx, err + } + command := trigger.GetRunTriggerCommand("log", "file", "mode", mode, "path", path, "count", strconv.Itoa(count), "interval", strconv.Itoa(interval), "custom", wrapperCustomArgs(string(jsonStr))) + fmt.Println(command) + go func() { + if _, err := setup.Env.ExecOnSource(ctx, command); err != nil { + fmt.Println(err) + } + }() + return ctx, nil +} + +func generateBenchmark(ctx context.Context, mode, path string, rate, duration int, customKV ...string) (context.Context, error) { + time.Sleep(3 * time.Second) + customKVString := make(map[string]string) + for i := 0; i < len(customKV); i += 2 { + customKVString[customKV[i]] = customKV[i+1] + } + jsonStr, err := json.Marshal(customKVString) + if err != nil { + return ctx, err + } + command := trigger.GetRunTriggerCommand("log", "file_benchmark", "mode", mode, "path", path, "rate", strconv.Itoa(rate), "duration", strconv.Itoa(duration), "custom", wrapperCustomArgs(string(jsonStr))) + if _, err := setup.Env.ExecOnSource(ctx, command); err != nil { + return ctx, err + } + return ctx, nil +} + +func wrapperCustomArgs(customArgs string) string { + fmt.Println(customArgs) + customArgs = strings.ReplaceAll(customArgs, "\\", "\\\\") + return "\"" + strings.ReplaceAll(customArgs, "\"", "\\\"") + "\"" +} diff --git a/test/engine/trigger/http.go b/test/engine/trigger/log/http_server.go similarity index 99% rename from test/engine/trigger/http.go rename to test/engine/trigger/log/http_server.go index 4a580e4c16..d2eec5b961 100644 --- a/test/engine/trigger/http.go +++ b/test/engine/trigger/log/http_server.go @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -package trigger +package log import ( "context" diff --git a/test/engine/trigger/log/remote_file.py b/test/engine/trigger/log/remote_file.py new file mode 100644 index 0000000000..1415c48e36 --- /dev/null +++ b/test/engine/trigger/log/remote_file.py @@ -0,0 +1,172 @@ +import argparse +import json as jsonlib +import logging +import random +import time + +from logging.handlers import TimedRotatingFileHandler +from datetime import datetime +from faker import Faker +from faker.providers import internet, user_agent, lorem, misc + + +def apsara(args, logger, faker): + fileNo = random.randint(1, 1000) + for i in range(args.count): + logger.info(f'[{datetime.now().strftime("%Y-%m-%d %H:%M:%S.%f")}]\t[{get_random_level()}]\t[{random.randint(1, 10000)}]\t[/build/core/application/Application:{i}]\tfile:file{fileNo}\tlogNo:{i}\tmark:{get_random_mark()}\tmsg:{faker.sentence()}') + if args.interval > 0: + time.sleep(args.interval / 1000) + +def delimiter(args, logger, faker): + custom_args = args.custom + quote = custom_args.get('quote', '') + delimiter = custom_args.get('delimiter', ' ') + fileNo = random.randint(1, 1000) + for i in range(args.count): + logParts = [ + f'{quote}{get_random_mark()}{quote}', + f'{quote}file{fileNo}{quote}', + f'{quote}logNo:{i}{quote}', + f'{quote}{faker.ipv4()}{quote}', + f'{quote}{datetime.now().strftime("%Y-%m-%d %H:%M:%S.%f")}{quote}', + f'{quote}{faker.http_method()}{quote}', + f'{quote}{faker.uri_path()}{quote}', + f'{quote}HTTP/2.0{quote}', + f'{quote}{faker.http_status_code()}{quote}', + f'{quote}{random.randint(1, 10000)}{quote}', + f'{quote}{faker.user_agent()}{quote}' + ] + log = delimiter.join(logParts) + logger.info(log) + if args.interval > 0: + time.sleep(args.interval / 1000) + +def delimiterMultiline(args, logger, faker): + custom_args = args.custom + quote = custom_args.get('quote', '') + delimiter = custom_args.get('delimiter', ' ') + fileNo = random.randint(1, 1000) + for i in range(args.count): + logParts = [ + f'{quote}{get_random_mark()}{quote}', + f'{quote}fi\nle{fileNo}{quote}', + f'{quote}logNo\n:{i}{quote}', + f'{quote}{faker.ipv4()}{quote}', + f'{quote}{datetime.now().strftime("%Y-%m-%d %H:%M:%S.%f")}{quote}', + f'{quote}{faker.http_method()}{quote}', + f'{quote}{faker.uri_path()}{quote}', + f'{quote}HT\nTP/2.0{quote}', + f'{quote}{faker.http_status_code()}{quote}', + f'{quote}{random.randint(1, 10000)}{quote}', + f'{quote}{faker.user_agent()}{quote}' + ] + log = delimiter.join(logParts) + logger.info(log) + if args.interval > 0: + time.sleep(args.interval / 1000) + +def json(args, logger, faker): + fileNo = random.randint(1, 1000) + for i in range(args.count): + logger.info(f'{{"mark":"{get_random_mark()}", "file":"file{fileNo}", "logNo":{i}, "time":"{datetime.now().strftime("%Y-%m-%d %H:%M:%S.%f")}", "ip": "{faker.ipv4()}", "method": "{faker.http_method()}", "userAgent": "{faker.user_agent()}", "size": {random.randint(1, 10000)}}}') + if args.interval > 0: + time.sleep(args.interval / 1000) + +def jsonMultiline(args, logger, faker): + fileNo = random.randint(1, 1000) + for i in range(args.count): + logParts = [ + f'"mark":"{get_random_mark()}"', + f'"file":"file{fileNo}"', + f'"logNo":{i}', + f'"time":"{datetime.now().strftime("%Y-%m-%d %H:%M:%S.%f")}"', + f'"ip": "{faker.ipv4()}"', + f'"method": "{faker.http_method()}"', + f'"userAgent": "{faker.user_agent()}"', + f'"size": {random.randint(1, 10000)}' + ] + log = '{' + ',\n'.join(logParts) + '}' + logger.info(log) + if args.interval > 0: + time.sleep(args.interval / 1000) + +def regex(args, logger, faker): + fileNo = random.randint(1, 1000) + for i in range(args.count): + logger.info(f'{get_random_mark()} file{fileNo}:{i} {faker.ipv4()} - [{datetime.now().strftime("%Y-%m-%d %H:%M:%S.%f")}] "{faker.http_method()} {faker.uri_path()} HTTP/2.0" {faker.http_status_code()} {random.randint(1, 10000)} "{faker.user_agent()}" "{faker.sentence()}"') + if args.interval > 0: + time.sleep(args.interval / 1000) + +def regexGBK(args, logger, faker): + fileNo = random.randint(1, 1000) + for i in range(args.count): + log = f'{get_random_mark()} file{fileNo}:{i} {faker.ipv4()} - [{datetime.now().strftime("%Y-%m-%d %H:%M:%S.%f")}] "{faker.http_method()} {faker.uri_path()} HTTP/2.0" {faker.http_status_code()} {random.randint(1, 10000)} "{faker.user_agent()}" "{faker.sentence()}"' + logger.info(str(log.encode('gbk'))) + if args.interval > 0: + time.sleep(args.interval / 1000) + +def regexMultiline(args, logger, faker): + fileNo = random.randint(1, 1000) + for i in range(args.count): + multilineLog = '\n'.join(faker.sentences(nb=random.randint(1, 5))) + logger.info(f'{get_random_mark()} file{fileNo}:{i} [{datetime.now().strftime("%Y-%m-%d %H:%M:%S.%f")}] [{get_random_level()}] java.lang.Exception: {multilineLog}') + + if args.interval > 0: + time.sleep(args.interval / 1000) + +def get_random_level(): + return random.choice(['DEBUG', 'INFO', 'WARNING', 'ERROR']) + +def get_random_mark(): + return random.choice(['-', 'F']) + +def parse_custom_arg_to_dict(custom_arg): + return jsonlib.loads(custom_arg) + +def main(): + parser = argparse.ArgumentParser(description='Log Generator Arg Parser') + parser.add_argument('--mode', type=str, default='regex', help='Log Type') + parser.add_argument('--path', type=str, default='default.log', help='Log Path') + parser.add_argument('--count', type=int, default=100, help='Log Count') + parser.add_argument('--interval', type=int, default=1, help='Log Interval (ms), < 0 means no interval') + parser.add_argument('--custom', type=parse_custom_arg_to_dict, help='Custom Args, in the format of json') + + args = parser.parse_args() + + logger = logging.getLogger('log_generator') + logger.setLevel(logging.INFO) + # 快速轮转来模拟比较极端的情况 + handler = TimedRotatingFileHandler(args.path, when="s", interval=5, backupCount=3) + formatter = logging.Formatter('%(message)s') + handler.setFormatter(formatter) + handler.flush = lambda: handler.stream.flush() + logger.addHandler(handler) + + # 随机生成器 + faker = Faker() + faker.add_provider(internet) + faker.add_provider(user_agent) + faker.add_provider(lorem) + faker.add_provider(misc) + + # 生成数据 + if args.mode == 'apsara': + apsara(args, logger, faker) + elif args.mode == 'delimiter': + delimiter(args, logger, faker) + elif args.mode == 'delimiterMultiline': + delimiterMultiline(args, logger, faker) + elif args.mode == 'json': + json(args, logger, faker) + elif args.mode == 'jsonMultiline': + jsonMultiline(args, logger, faker) + elif args.mode == 'regex': + regex(args, logger, faker) + elif args.mode == 'regexGBK': + regexGBK(args, logger, faker) + elif args.mode == 'regexMultiline': + regexMultiline(args, logger, faker) + + +if __name__ == '__main__': + main() diff --git a/test/engine/trigger/log/remote_file_benchmark.py b/test/engine/trigger/log/remote_file_benchmark.py new file mode 100644 index 0000000000..b9f53e00da --- /dev/null +++ b/test/engine/trigger/log/remote_file_benchmark.py @@ -0,0 +1,70 @@ +import argparse +import json +import logging +import math +import random +import time + +from logging.handlers import TimedRotatingFileHandler +from datetime import datetime +from faker import Faker +from faker.providers import internet, user_agent, lorem, misc + +BATCH_SIZE = 100 + +def nginx(args, logger, faker): + startTime = time.perf_counter() + exampleLog = '' + for _ in range(BATCH_SIZE): + exampleLog += f'{faker.ipv4()} - - [{datetime.now().strftime("%d/%b/%Y:%H:%M:%S %z")}] "{faker.http_method()} {faker.url()} HTTP/1.1" {faker.http_status_code()} {random.randint(1, 10000)} "{faker.url()}" "{faker.user_agent()}\n"' + randomLogCost = (time.perf_counter() - startTime) / BATCH_SIZE + writeTimePerSecond = math.floor(args.rate * 1024 * 1024 / (len(exampleLog.encode('utf-8')))) + sleepInterval = 1 / writeTimePerSecond - randomLogCost + + startTime = datetime.now() + while True: + now = datetime.now() + fakeLog = f'{faker.ipv4()} - - [{now.strftime("%d/%b/%Y:%H:%M:%S %z")}] "{faker.http_method()} {faker.url()} HTTP/1.1" {faker.http_status_code()} {random.randint(1, 10000)} "{faker.url()}" "{faker.user_agent()}"\n' * BATCH_SIZE + logger.info(fakeLog[:-1]) + if sleepInterval > 0: + start = time.perf_counter() + while (time.perf_counter() - start) < sleepInterval: + pass + if (now - startTime).seconds > args.duration * 60: + break + +def parse_custom_arg_to_dict(custom_arg): + return json.loads(custom_arg) + +def main(): + parser = argparse.ArgumentParser(description='Log Generator Arg Parser') + parser.add_argument('--mode', type=str, default='nginx', help='Log Type') + parser.add_argument('--path', type=str, default='default.log', help='Log Path') + parser.add_argument('--rate', type=int, default=10, help='Log Generate Rate (MB/s)') + parser.add_argument('--duration', type=int, default=60, help='Log Generate Duration (min)') + parser.add_argument('--custom', nargs='*', type=parse_custom_arg_to_dict, help='Custom Args, in the format of key=value') + + args = parser.parse_args() + + logger = logging.getLogger('log_generator') + logger.setLevel(logging.INFO) + # 快速轮转来模拟比较极端的情况 + handler = TimedRotatingFileHandler(args.path, when="s", interval=70, backupCount=3) + formatter = logging.Formatter('%(message)s') + handler.setFormatter(formatter) + logger.addHandler(handler) + + # 随机生成器 + faker = Faker() + faker.add_provider(internet) + faker.add_provider(user_agent) + faker.add_provider(lorem) + faker.add_provider(misc) + + # 生成数据 + if args.mode == 'nginx': + nginx(args, logger, faker) + + +if __name__ == '__main__': + main() diff --git a/test/engine/trigger/trigger.go b/test/engine/trigger/trigger.go deleted file mode 100644 index 63653c5cb6..0000000000 --- a/test/engine/trigger/trigger.go +++ /dev/null @@ -1,96 +0,0 @@ -// Copyright 2024 iLogtail Authors -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -package trigger - -import ( - "context" - "strings" - "text/template" - "time" - - "github.com/alibaba/ilogtail/test/config" - "github.com/alibaba/ilogtail/test/engine/setup" -) - -const triggerTemplate = "cd {{.WorkDir}} && TOTAL_LOG={{.TotalLog}} INTERVAL={{.Interval}} FILENAME={{.Filename}} GENERATED_LOG_DIR={{.GeneratedLogDir}} {{.Custom}} {{.Command}}" - -func RegexSingle(ctx context.Context, totalLog int, path string, interval int) (context.Context, error) { - return generate(ctx, totalLog, path, interval, "TestGenerateRegexLogSingle") -} - -func RegexSingleGBK(ctx context.Context, totalLog int, path string, interval int) (context.Context, error) { - return generate(ctx, totalLog, path, interval, "TestGenerateRegexLogSingleGBK") -} - -func RegexMultiline(ctx context.Context, totalLog int, path string, interval int) (context.Context, error) { - return generate(ctx, totalLog, path, interval, "TestGenerateRegexLogMultiline") -} - -func JSONSingle(ctx context.Context, totalLog int, path string, interval int) (context.Context, error) { - return generate(ctx, totalLog, path, interval, "TestGenerateJSONSingle") -} - -func JSONMultiline(ctx context.Context, totalLog int, path string, interval int) (context.Context, error) { - return generate(ctx, totalLog, path, interval, "TestGenerateJSONMultiline") -} - -func Apsara(ctx context.Context, totalLog int, path string, interval int) (context.Context, error) { - return generate(ctx, totalLog, path, interval, "TestGenerateApsara") -} - -func DelimiterSingle(ctx context.Context, totalLog int, path string, interval int, delimiter, quote string) (context.Context, error) { - return generate(ctx, totalLog, path, interval, "TestGenerateDelimiterSingle", "Delimiter", delimiter, "Quote", quote) -} - -func DelimiterMultiline(ctx context.Context, totalLog int, path string, interval int, delimiter, quote string) (context.Context, error) { - return generate(ctx, totalLog, path, interval, "TestGenerateDelimiterMultiline", "Delimiter", delimiter, "Quote", quote) -} - -func generate(ctx context.Context, totalLog int, path string, interval int, commandName string, customKV ...string) (context.Context, error) { - time.Sleep(3 * time.Second) - command := getRunTriggerCommand(commandName) - var triggerCommand strings.Builder - template := template.Must(template.New("trigger").Parse(triggerTemplate)) - splittedPath := strings.Split(path, "/") - dir := strings.Join(splittedPath[:len(splittedPath)-1], "/") - filename := splittedPath[len(splittedPath)-1] - customString := strings.Builder{} - for i := 0; i < len(customKV); i++ { - customString.WriteString(customKV[i]) - customString.WriteString("=") - customString.WriteString(customKV[i+1]) - customString.WriteString(" ") - i++ - } - if err := template.Execute(&triggerCommand, map[string]interface{}{ - "WorkDir": config.TestConfig.WorkDir, - "TotalLog": totalLog, - "Interval": interval, - "GeneratedLogDir": dir, - "Filename": filename, - "Custom": customString.String(), - "Command": command, - }); err != nil { - return ctx, err - } - if _, err := setup.Env.ExecOnSource(ctx, triggerCommand.String()); err != nil { - return ctx, err - } - return ctx, nil -} - -func BeginTrigger(ctx context.Context) (context.Context, error) { - startTime := time.Now().Unix() - return context.WithValue(ctx, config.StartTimeContextKey, int32(startTime)), nil -} diff --git a/test/requirements.txt b/test/requirements.txt new file mode 100644 index 0000000000..ea45cd03b7 --- /dev/null +++ b/test/requirements.txt @@ -0,0 +1 @@ +Faker \ No newline at end of file