mirror of
https://github.com/TeaOSLab/EdgeNode.git
synced 2025-11-12 14:30:54 +08:00
实现新的计数器算法(将时间分片, 统计更加精准)
This commit is contained in:
@@ -10,9 +10,9 @@ import (
|
|||||||
teaconst "github.com/TeaOSLab/EdgeNode/internal/const"
|
teaconst "github.com/TeaOSLab/EdgeNode/internal/const"
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/iplibrary"
|
"github.com/TeaOSLab/EdgeNode/internal/iplibrary"
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/stats"
|
"github.com/TeaOSLab/EdgeNode/internal/stats"
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/ttlcache"
|
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/utils"
|
"github.com/TeaOSLab/EdgeNode/internal/utils"
|
||||||
connutils "github.com/TeaOSLab/EdgeNode/internal/utils/conns"
|
connutils "github.com/TeaOSLab/EdgeNode/internal/utils/conns"
|
||||||
|
"github.com/TeaOSLab/EdgeNode/internal/utils/counters"
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/utils/fasttime"
|
"github.com/TeaOSLab/EdgeNode/internal/utils/fasttime"
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/waf"
|
"github.com/TeaOSLab/EdgeNode/internal/waf"
|
||||||
"github.com/iwind/TeaGo/Tea"
|
"github.com/iwind/TeaGo/Tea"
|
||||||
@@ -24,6 +24,8 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
var synFloodCounter = counters.NewCounter().WithGC()
|
||||||
|
|
||||||
// ClientConn 客户端连接
|
// ClientConn 客户端连接
|
||||||
type ClientConn struct {
|
type ClientConn struct {
|
||||||
BaseClientConn
|
BaseClientConn
|
||||||
@@ -289,14 +291,13 @@ func (this *ClientConn) LastErr() error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (this *ClientConn) resetSYNFlood() {
|
func (this *ClientConn) resetSYNFlood() {
|
||||||
ttlcache.SharedCache.Delete("SYN_FLOOD:" + this.RawIP())
|
synFloodCounter.ResetKey("SYN_FLOOD:" + this.RawIP())
|
||||||
}
|
}
|
||||||
|
|
||||||
func (this *ClientConn) increaseSYNFlood(synFloodConfig *firewallconfigs.SYNFloodConfig) {
|
func (this *ClientConn) increaseSYNFlood(synFloodConfig *firewallconfigs.SYNFloodConfig) {
|
||||||
var ip = this.RawIP()
|
var ip = this.RawIP()
|
||||||
if len(ip) > 0 && !iplibrary.IsInWhiteList(ip) && (!synFloodConfig.IgnoreLocal || !utils.IsLocalIP(ip)) {
|
if len(ip) > 0 && !iplibrary.IsInWhiteList(ip) && (!synFloodConfig.IgnoreLocal || !utils.IsLocalIP(ip)) {
|
||||||
var timestamp = fasttime.Now().UnixNextMinute()
|
var result = synFloodCounter.IncreaseKey("SYN_FLOOD:"+ip, 60)
|
||||||
var result = ttlcache.SharedCache.IncreaseInt64("SYN_FLOOD:"+ip, 1, timestamp, true)
|
|
||||||
var minAttempts = synFloodConfig.MinAttempts
|
var minAttempts = synFloodConfig.MinAttempts
|
||||||
if minAttempts < 5 {
|
if minAttempts < 5 {
|
||||||
minAttempts = 5
|
minAttempts = 5
|
||||||
@@ -305,7 +306,7 @@ func (this *ClientConn) increaseSYNFlood(synFloodConfig *firewallconfigs.SYNFloo
|
|||||||
// 非TLS,设置为两倍,防止误封
|
// 非TLS,设置为两倍,防止误封
|
||||||
minAttempts = 2 * minAttempts
|
minAttempts = 2 * minAttempts
|
||||||
}
|
}
|
||||||
if result >= int64(minAttempts) {
|
if result >= types.Uint64(minAttempts) {
|
||||||
var timeout = synFloodConfig.TimeoutSeconds
|
var timeout = synFloodConfig.TimeoutSeconds
|
||||||
if timeout <= 0 {
|
if timeout <= 0 {
|
||||||
timeout = 600
|
timeout = 600
|
||||||
|
|||||||
@@ -213,7 +213,7 @@ func BenchmarkCache_Add_Parallel(b *testing.B) {
|
|||||||
b.RunParallel(func(pb *testing.PB) {
|
b.RunParallel(func(pb *testing.PB) {
|
||||||
for pb.Next() {
|
for pb.Next() {
|
||||||
var j = atomic.AddInt64(&i, 1)
|
var j = atomic.AddInt64(&i, 1)
|
||||||
cache.Write(types.String(j), j, fasttime.Now().Unix()+i%1024)
|
cache.Write(types.String(j%1e6), j, fasttime.Now().Unix()+i%1024)
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|||||||
173
internal/utils/counters/counter.go
Normal file
173
internal/utils/counters/counter.go
Normal file
@@ -0,0 +1,173 @@
|
|||||||
|
// Copyright 2023 GoEdge CDN goedge.cdn@gmail.com. All rights reserved. Official site: https://goedge.cn .
|
||||||
|
|
||||||
|
package counters
|
||||||
|
|
||||||
|
import (
|
||||||
|
syncutils "github.com/TeaOSLab/EdgeNode/internal/utils/sync"
|
||||||
|
"github.com/cespare/xxhash"
|
||||||
|
"github.com/iwind/TeaGo/Tea"
|
||||||
|
"runtime"
|
||||||
|
"time"
|
||||||
|
)
|
||||||
|
|
||||||
|
type Counter struct {
|
||||||
|
countMaps uint64
|
||||||
|
locker *syncutils.RWMutex
|
||||||
|
itemMaps []map[uint64]*Item
|
||||||
|
|
||||||
|
gcTicker *time.Ticker
|
||||||
|
gcIndex int
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewCounter create new counter
|
||||||
|
func NewCounter() *Counter {
|
||||||
|
var count = runtime.NumCPU() * 4
|
||||||
|
if count < 8 {
|
||||||
|
count = 8
|
||||||
|
}
|
||||||
|
|
||||||
|
var itemMaps = []map[uint64]*Item{}
|
||||||
|
for i := 0; i < count; i++ {
|
||||||
|
itemMaps = append(itemMaps, map[uint64]*Item{})
|
||||||
|
}
|
||||||
|
|
||||||
|
var counter = &Counter{
|
||||||
|
countMaps: uint64(count),
|
||||||
|
locker: syncutils.NewRWMutex(count),
|
||||||
|
itemMaps: itemMaps,
|
||||||
|
}
|
||||||
|
|
||||||
|
return counter
|
||||||
|
}
|
||||||
|
|
||||||
|
// WithGC start the counter with gc automatically
|
||||||
|
func (this *Counter) WithGC() *Counter {
|
||||||
|
if this.gcTicker != nil {
|
||||||
|
return this
|
||||||
|
}
|
||||||
|
this.gcTicker = time.NewTicker(10 * time.Second)
|
||||||
|
if Tea.IsTesting() {
|
||||||
|
this.gcTicker = time.NewTicker(1 * time.Second)
|
||||||
|
}
|
||||||
|
go func() {
|
||||||
|
for range this.gcTicker.C {
|
||||||
|
this.GC()
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
return this
|
||||||
|
}
|
||||||
|
|
||||||
|
// Increase key
|
||||||
|
func (this *Counter) Increase(key uint64, lifeSeconds int) uint64 {
|
||||||
|
var index = int(key % this.countMaps)
|
||||||
|
this.locker.RLock(index)
|
||||||
|
var item = this.itemMaps[index][key]
|
||||||
|
this.locker.RUnlock(index)
|
||||||
|
if item == nil { // no need to care about duplication
|
||||||
|
item = NewItem(lifeSeconds)
|
||||||
|
this.locker.Lock(index)
|
||||||
|
|
||||||
|
// check again
|
||||||
|
oldItem, ok := this.itemMaps[index][key]
|
||||||
|
if !ok {
|
||||||
|
this.itemMaps[index][key] = item
|
||||||
|
} else {
|
||||||
|
item = oldItem
|
||||||
|
}
|
||||||
|
|
||||||
|
this.locker.Unlock(index)
|
||||||
|
}
|
||||||
|
|
||||||
|
this.locker.Lock(index)
|
||||||
|
var result = item.Increase()
|
||||||
|
this.locker.Unlock(index)
|
||||||
|
return result
|
||||||
|
}
|
||||||
|
|
||||||
|
// IncreaseKey increase string key
|
||||||
|
func (this *Counter) IncreaseKey(key string, lifeSeconds int) uint64 {
|
||||||
|
return this.Increase(this.hash(key), lifeSeconds)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Get value of key
|
||||||
|
func (this *Counter) Get(key uint64) uint64 {
|
||||||
|
var index = int(key % this.countMaps)
|
||||||
|
this.locker.RLock(index)
|
||||||
|
defer this.locker.RUnlock(index)
|
||||||
|
var item = this.itemMaps[index][key]
|
||||||
|
if item != nil {
|
||||||
|
return item.Sum()
|
||||||
|
}
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetKey get value of string key
|
||||||
|
func (this *Counter) GetKey(key string) uint64 {
|
||||||
|
return this.Get(this.hash(key))
|
||||||
|
}
|
||||||
|
|
||||||
|
// Reset key
|
||||||
|
func (this *Counter) Reset(key uint64) {
|
||||||
|
var index = int(key % this.countMaps)
|
||||||
|
this.locker.RLock(index)
|
||||||
|
var item = this.itemMaps[index][key]
|
||||||
|
this.locker.RUnlock(index)
|
||||||
|
|
||||||
|
if item != nil {
|
||||||
|
this.locker.Lock(index)
|
||||||
|
delete(this.itemMaps[index], key)
|
||||||
|
this.locker.Unlock(index)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// ResetKey string key
|
||||||
|
func (this *Counter) ResetKey(key string) {
|
||||||
|
this.Reset(this.hash(key))
|
||||||
|
}
|
||||||
|
|
||||||
|
// TotalItems get items count
|
||||||
|
func (this *Counter) TotalItems() int {
|
||||||
|
var total = 0
|
||||||
|
|
||||||
|
for i := 0; i < int(this.countMaps); i++ {
|
||||||
|
this.locker.RLock(i)
|
||||||
|
total += len(this.itemMaps[i])
|
||||||
|
this.locker.RUnlock(i)
|
||||||
|
}
|
||||||
|
|
||||||
|
return total
|
||||||
|
}
|
||||||
|
|
||||||
|
// GC garbage expired items
|
||||||
|
func (this *Counter) GC() {
|
||||||
|
var gcIndex = this.gcIndex
|
||||||
|
|
||||||
|
this.gcIndex++
|
||||||
|
if this.gcIndex >= int(this.countMaps) {
|
||||||
|
this.gcIndex = 0
|
||||||
|
}
|
||||||
|
|
||||||
|
this.locker.RLock(gcIndex)
|
||||||
|
var itemMap = this.itemMaps[gcIndex]
|
||||||
|
var expiredKeys = []uint64{}
|
||||||
|
for key, item := range itemMap {
|
||||||
|
if item.IsExpired() {
|
||||||
|
expiredKeys = append(expiredKeys, key)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
this.locker.RUnlock(gcIndex)
|
||||||
|
|
||||||
|
if len(expiredKeys) > 0 {
|
||||||
|
for _, key := range expiredKeys {
|
||||||
|
this.locker.Lock(gcIndex)
|
||||||
|
delete(itemMap, key)
|
||||||
|
this.locker.Unlock(gcIndex)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// calculate hash of the key
|
||||||
|
func (this *Counter) hash(key string) uint64 {
|
||||||
|
return xxhash.Sum64String(key)
|
||||||
|
}
|
||||||
143
internal/utils/counters/counter_test.go
Normal file
143
internal/utils/counters/counter_test.go
Normal file
@@ -0,0 +1,143 @@
|
|||||||
|
// Copyright 2023 GoEdge CDN goedge.cdn@gmail.com. All rights reserved. Official site: https://goedge.cn .
|
||||||
|
|
||||||
|
package counters_test
|
||||||
|
|
||||||
|
import (
|
||||||
|
"github.com/TeaOSLab/EdgeNode/internal/utils/counters"
|
||||||
|
"github.com/TeaOSLab/EdgeNode/internal/utils/testutils"
|
||||||
|
"github.com/iwind/TeaGo/assert"
|
||||||
|
"github.com/iwind/TeaGo/types"
|
||||||
|
"runtime"
|
||||||
|
"sync/atomic"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestCounter_Increase(t *testing.T) {
|
||||||
|
var a = assert.NewAssertion(t)
|
||||||
|
|
||||||
|
var counter = counters.NewCounter()
|
||||||
|
a.IsTrue(counter.Increase(1, 10) == 1)
|
||||||
|
a.IsTrue(counter.Increase(1, 10) == 2)
|
||||||
|
a.IsTrue(counter.Increase(2, 10) == 1)
|
||||||
|
|
||||||
|
counter.Reset(1)
|
||||||
|
a.IsTrue(counter.Get(1) == 0) // changed
|
||||||
|
a.IsTrue(counter.Get(2) == 1) // not changed
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestCounter_IncreaseKey(t *testing.T) {
|
||||||
|
var a = assert.NewAssertion(t)
|
||||||
|
|
||||||
|
var counter = counters.NewCounter()
|
||||||
|
a.IsTrue(counter.IncreaseKey("1", 10) == 1)
|
||||||
|
a.IsTrue(counter.IncreaseKey("1", 10) == 2)
|
||||||
|
a.IsTrue(counter.IncreaseKey("2", 10) == 1)
|
||||||
|
|
||||||
|
counter.ResetKey("1")
|
||||||
|
a.IsTrue(counter.GetKey("1") == 0) // changed
|
||||||
|
a.IsTrue(counter.GetKey("2") == 1) // not changed
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestCounter_GC(t *testing.T) {
|
||||||
|
if !testutils.IsSingleTesting() {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
var counter = counters.NewCounter()
|
||||||
|
counter.Increase(1, 20)
|
||||||
|
time.Sleep(1 * time.Second)
|
||||||
|
counter.Increase(1, 20)
|
||||||
|
time.Sleep(1 * time.Second)
|
||||||
|
counter.Increase(1, 20)
|
||||||
|
counter.GC()
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestCounter_GC2(t *testing.T) {
|
||||||
|
var counter = counters.NewCounter()
|
||||||
|
for i := 0; i < runtime.NumCPU()*32; i++ {
|
||||||
|
counter.GC()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func BenchmarkCounter_Increase(b *testing.B) {
|
||||||
|
runtime.GOMAXPROCS(4)
|
||||||
|
|
||||||
|
var counter = counters.NewCounter()
|
||||||
|
b.ResetTimer()
|
||||||
|
|
||||||
|
var i uint64
|
||||||
|
b.RunParallel(func(pb *testing.PB) {
|
||||||
|
for pb.Next() {
|
||||||
|
counter.Increase(atomic.AddUint64(&i, 1)%1e6, 20)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
|
||||||
|
//b.Log(counter.TotalItems())
|
||||||
|
}
|
||||||
|
|
||||||
|
func BenchmarkCounter_IncreaseKey(b *testing.B) {
|
||||||
|
runtime.GOMAXPROCS(4)
|
||||||
|
|
||||||
|
var counter = counters.NewCounter()
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
var ticker = time.NewTicker(100 * time.Millisecond)
|
||||||
|
for range ticker.C {
|
||||||
|
counter.GC()
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
b.ResetTimer()
|
||||||
|
|
||||||
|
var i uint64
|
||||||
|
b.RunParallel(func(pb *testing.PB) {
|
||||||
|
for pb.Next() {
|
||||||
|
counter.IncreaseKey(types.String(atomic.AddUint64(&i, 1)%1e6), 20)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
|
||||||
|
//b.Log(counter.TotalItems())
|
||||||
|
}
|
||||||
|
|
||||||
|
func BenchmarkCounter_IncreaseKey2(b *testing.B) {
|
||||||
|
runtime.GOMAXPROCS(4)
|
||||||
|
|
||||||
|
var counter = counters.NewCounter()
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
var ticker = time.NewTicker(1 * time.Millisecond)
|
||||||
|
for range ticker.C {
|
||||||
|
counter.GC()
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
b.ResetTimer()
|
||||||
|
|
||||||
|
var i uint64
|
||||||
|
b.RunParallel(func(pb *testing.PB) {
|
||||||
|
for pb.Next() {
|
||||||
|
counter.IncreaseKey(types.String(atomic.AddUint64(&i, 1)%1e5), 20)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
|
||||||
|
//b.Log(counter.TotalItems())
|
||||||
|
}
|
||||||
|
|
||||||
|
func BenchmarkCounter_GC(b *testing.B) {
|
||||||
|
runtime.GOMAXPROCS(4)
|
||||||
|
|
||||||
|
var counter = counters.NewCounter()
|
||||||
|
|
||||||
|
for i := uint64(0); i < 1e5; i++ {
|
||||||
|
counter.IncreaseKey(types.String(i), 20)
|
||||||
|
}
|
||||||
|
|
||||||
|
b.ResetTimer()
|
||||||
|
|
||||||
|
b.RunParallel(func(pb *testing.PB) {
|
||||||
|
for pb.Next() {
|
||||||
|
counter.GC()
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
||||||
77
internal/utils/counters/item.go
Normal file
77
internal/utils/counters/item.go
Normal file
@@ -0,0 +1,77 @@
|
|||||||
|
// Copyright 2023 GoEdge CDN goedge.cdn@gmail.com. All rights reserved. Official site: https://goedge.cn .
|
||||||
|
|
||||||
|
package counters
|
||||||
|
|
||||||
|
import (
|
||||||
|
"github.com/TeaOSLab/EdgeNode/internal/utils/fasttime"
|
||||||
|
)
|
||||||
|
|
||||||
|
type Item struct {
|
||||||
|
lifeSeconds int64
|
||||||
|
|
||||||
|
spanSeconds int64
|
||||||
|
spans []*Span
|
||||||
|
|
||||||
|
lastUpdateTime int64
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewItem(lifeSeconds int) *Item {
|
||||||
|
if lifeSeconds <= 0 {
|
||||||
|
lifeSeconds = 60
|
||||||
|
}
|
||||||
|
var spanSeconds = lifeSeconds / 10
|
||||||
|
if spanSeconds < 1 {
|
||||||
|
spanSeconds = 1
|
||||||
|
}
|
||||||
|
var countSpans = lifeSeconds/spanSeconds + 1 /** prevent index out of bounds **/
|
||||||
|
var spans = []*Span{}
|
||||||
|
for i := 0; i < countSpans; i++ {
|
||||||
|
spans = append(spans, NewSpan())
|
||||||
|
}
|
||||||
|
|
||||||
|
return &Item{
|
||||||
|
lifeSeconds: int64(lifeSeconds),
|
||||||
|
spanSeconds: int64(spanSeconds),
|
||||||
|
spans: spans,
|
||||||
|
lastUpdateTime: fasttime.Now().Unix(),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (this *Item) Increase() uint64 {
|
||||||
|
var currentTime = fasttime.Now().Unix()
|
||||||
|
var spanIndex = int(currentTime % this.lifeSeconds / this.spanSeconds)
|
||||||
|
var span = this.spans[spanIndex]
|
||||||
|
var roundTime = currentTime / this.spanSeconds * this.spanSeconds
|
||||||
|
|
||||||
|
this.lastUpdateTime = currentTime
|
||||||
|
|
||||||
|
if span.Timestamp < roundTime {
|
||||||
|
span.Timestamp = roundTime // update time
|
||||||
|
span.Count = 0 // reset count
|
||||||
|
}
|
||||||
|
span.Count++
|
||||||
|
|
||||||
|
return this.Sum()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (this *Item) Sum() uint64 {
|
||||||
|
var result uint64 = 0
|
||||||
|
var currentTimestamp = fasttime.Now().Unix()
|
||||||
|
for _, span := range this.spans {
|
||||||
|
if span.Timestamp >= currentTimestamp-this.lifeSeconds {
|
||||||
|
result += span.Count
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return result
|
||||||
|
}
|
||||||
|
|
||||||
|
func (this *Item) Reset() {
|
||||||
|
for _, span := range this.spans {
|
||||||
|
span.Count = 0
|
||||||
|
span.Timestamp = 0
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (this *Item) IsExpired() bool {
|
||||||
|
return this.lastUpdateTime < fasttime.Now().Unix()-this.lifeSeconds-this.spanSeconds
|
||||||
|
}
|
||||||
54
internal/utils/counters/item_test.go
Normal file
54
internal/utils/counters/item_test.go
Normal file
@@ -0,0 +1,54 @@
|
|||||||
|
// Copyright 2023 GoEdge CDN goedge.cdn@gmail.com. All rights reserved. Official site: https://goedge.cn .
|
||||||
|
|
||||||
|
package counters_test
|
||||||
|
|
||||||
|
import (
|
||||||
|
"github.com/TeaOSLab/EdgeNode/internal/utils/counters"
|
||||||
|
"github.com/TeaOSLab/EdgeNode/internal/utils/testutils"
|
||||||
|
"github.com/iwind/TeaGo/assert"
|
||||||
|
timeutil "github.com/iwind/TeaGo/utils/time"
|
||||||
|
"runtime"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestItem_Increase(t *testing.T) {
|
||||||
|
// run only under single testing
|
||||||
|
if !testutils.IsSingleTesting() {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
var a = assert.NewAssertion(t)
|
||||||
|
|
||||||
|
var item = counters.NewItem(20)
|
||||||
|
for i := 0; i < 100; i++ {
|
||||||
|
t.Log(item.Increase(), timeutil.Format("i:s"))
|
||||||
|
time.Sleep(2 * time.Second)
|
||||||
|
}
|
||||||
|
|
||||||
|
item.Reset()
|
||||||
|
a.IsTrue(item.Sum() == 0)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestItem_IsExpired(t *testing.T) {
|
||||||
|
if !testutils.IsSingleTesting() {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
var item = counters.NewItem(10)
|
||||||
|
t.Log(item.IsExpired())
|
||||||
|
time.Sleep(10 * time.Second)
|
||||||
|
t.Log(item.IsExpired())
|
||||||
|
time.Sleep(2 * time.Second)
|
||||||
|
t.Log(item.IsExpired())
|
||||||
|
}
|
||||||
|
|
||||||
|
func BenchmarkItem_Increase(b *testing.B) {
|
||||||
|
runtime.GOMAXPROCS(1)
|
||||||
|
|
||||||
|
var item = counters.NewItem(60)
|
||||||
|
|
||||||
|
for i := 0; i < b.N; i++ {
|
||||||
|
item.Increase()
|
||||||
|
}
|
||||||
|
}
|
||||||
12
internal/utils/counters/span.go
Normal file
12
internal/utils/counters/span.go
Normal file
@@ -0,0 +1,12 @@
|
|||||||
|
// Copyright 2023 GoEdge CDN goedge.cdn@gmail.com. All rights reserved. Official site: https://goedge.cn .
|
||||||
|
|
||||||
|
package counters
|
||||||
|
|
||||||
|
type Span struct {
|
||||||
|
Timestamp int64
|
||||||
|
Count uint64
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewSpan() *Span {
|
||||||
|
return &Span{}
|
||||||
|
}
|
||||||
@@ -50,3 +50,7 @@ func (this *RWMutex) TryLock(index int) bool {
|
|||||||
func (this *RWMutex) TryRLock(index int) bool {
|
func (this *RWMutex) TryRLock(index int) bool {
|
||||||
return this.lockers[index%this.countLockers].TryRLock()
|
return this.lockers[index%this.countLockers].TryRLock()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (this *RWMutex) RWMutex(index int) *sync.RWMutex {
|
||||||
|
return this.lockers[index%this.countLockers]
|
||||||
|
}
|
||||||
|
|||||||
@@ -7,7 +7,6 @@ import (
|
|||||||
"fmt"
|
"fmt"
|
||||||
"github.com/TeaOSLab/EdgeCommon/pkg/nodeconfigs"
|
"github.com/TeaOSLab/EdgeCommon/pkg/nodeconfigs"
|
||||||
"github.com/TeaOSLab/EdgeCommon/pkg/serverconfigs/firewallconfigs"
|
"github.com/TeaOSLab/EdgeCommon/pkg/serverconfigs/firewallconfigs"
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/ttlcache"
|
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/waf/requests"
|
"github.com/TeaOSLab/EdgeNode/internal/waf/requests"
|
||||||
"github.com/iwind/TeaGo/types"
|
"github.com/iwind/TeaGo/types"
|
||||||
"net/http"
|
"net/http"
|
||||||
@@ -121,7 +120,7 @@ func (this *JSCookieAction) increaseFails(req requests.Request, policyId int64,
|
|||||||
|
|
||||||
var key = "JS_COOKIE:FAILS:" + req.WAFRemoteIP() + ":" + types.String(req.WAFServerId()) + ":" + req.WAFRaw().URL.String()
|
var key = "JS_COOKIE:FAILS:" + req.WAFRemoteIP() + ":" + types.String(req.WAFServerId()) + ":" + req.WAFRaw().URL.String()
|
||||||
|
|
||||||
var countFails = ttlcache.SharedCache.IncreaseInt64(key, 1, time.Now().Unix()+300, true)
|
var countFails = SharedCounter.IncreaseKey(key, 300)
|
||||||
if int(countFails) >= maxFails {
|
if int(countFails) >= maxFails {
|
||||||
SharedIPBlackList.RecordIP(IPTypeAll, firewallconfigs.FirewallScopeService, req.WAFServerId(), req.WAFRemoteIP(), time.Now().Unix()+int64(failBlockTimeout), policyId, true, groupId, setId, "JS_COOKIE验证连续失败超过"+types.String(maxFails)+"次")
|
SharedIPBlackList.RecordIP(IPTypeAll, firewallconfigs.FirewallScopeService, req.WAFServerId(), req.WAFRemoteIP(), time.Now().Unix()+int64(failBlockTimeout), policyId, true, groupId, setId, "JS_COOKIE验证连续失败超过"+types.String(maxFails)+"次")
|
||||||
return false
|
return false
|
||||||
|
|||||||
@@ -4,7 +4,6 @@ package waf
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/TeaOSLab/EdgeCommon/pkg/serverconfigs/firewallconfigs"
|
"github.com/TeaOSLab/EdgeCommon/pkg/serverconfigs/firewallconfigs"
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/ttlcache"
|
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/utils"
|
"github.com/TeaOSLab/EdgeNode/internal/utils"
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/waf/requests"
|
"github.com/TeaOSLab/EdgeNode/internal/waf/requests"
|
||||||
"github.com/iwind/TeaGo/types"
|
"github.com/iwind/TeaGo/types"
|
||||||
@@ -27,7 +26,7 @@ func CaptchaIncreaseFails(req requests.Request, actionConfig *CaptchaAction, pol
|
|||||||
if maxFails <= 3 {
|
if maxFails <= 3 {
|
||||||
maxFails = 3 // 不能小于3,防止意外刷新出现
|
maxFails = 3 // 不能小于3,防止意外刷新出现
|
||||||
}
|
}
|
||||||
var countFails = ttlcache.SharedCache.IncreaseInt64(CaptchaCacheKey(req, pageCode), 1, time.Now().Unix()+300, true)
|
var countFails = SharedCounter.IncreaseKey(CaptchaCacheKey(req, pageCode), 300)
|
||||||
if int(countFails) >= maxFails {
|
if int(countFails) >= maxFails {
|
||||||
SharedIPBlackList.RecordIP(IPTypeAll, firewallconfigs.FirewallScopeService, req.WAFServerId(), req.WAFRemoteIP(), time.Now().Unix()+int64(failBlockTimeout), policyId, true, groupId, setId, "CAPTCHA验证连续失败超过"+types.String(maxFails)+"次")
|
SharedIPBlackList.RecordIP(IPTypeAll, firewallconfigs.FirewallScopeService, req.WAFServerId(), req.WAFRemoteIP(), time.Now().Unix()+int64(failBlockTimeout), policyId, true, groupId, setId, "CAPTCHA验证连续失败超过"+types.String(maxFails)+"次")
|
||||||
return false
|
return false
|
||||||
@@ -38,9 +37,9 @@ func CaptchaIncreaseFails(req requests.Request, actionConfig *CaptchaAction, pol
|
|||||||
|
|
||||||
// CaptchaDeleteCacheKey 清除计数
|
// CaptchaDeleteCacheKey 清除计数
|
||||||
func CaptchaDeleteCacheKey(req requests.Request) {
|
func CaptchaDeleteCacheKey(req requests.Request) {
|
||||||
ttlcache.SharedCache.Delete(CaptchaCacheKey(req, CaptchaPageCodeInit))
|
SharedCounter.ResetKey(CaptchaCacheKey(req, CaptchaPageCodeInit))
|
||||||
ttlcache.SharedCache.Delete(CaptchaCacheKey(req, CaptchaPageCodeShow))
|
SharedCounter.ResetKey(CaptchaCacheKey(req, CaptchaPageCodeShow))
|
||||||
ttlcache.SharedCache.Delete(CaptchaCacheKey(req, CaptchaPageCodeSubmit))
|
SharedCounter.ResetKey(CaptchaCacheKey(req, CaptchaPageCodeSubmit))
|
||||||
}
|
}
|
||||||
|
|
||||||
// CaptchaCacheKey 获取Captcha缓存Key
|
// CaptchaCacheKey 获取Captcha缓存Key
|
||||||
|
|||||||
@@ -1,22 +1,19 @@
|
|||||||
package checkpoints
|
package checkpoints
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/ttlcache"
|
"github.com/TeaOSLab/EdgeNode/internal/utils/counters"
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/waf/requests"
|
"github.com/TeaOSLab/EdgeNode/internal/waf/requests"
|
||||||
"github.com/iwind/TeaGo/maps"
|
"github.com/iwind/TeaGo/maps"
|
||||||
"github.com/iwind/TeaGo/types"
|
"github.com/iwind/TeaGo/types"
|
||||||
"regexp"
|
"regexp"
|
||||||
"sync"
|
|
||||||
"time"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
|
var ccCounter = counters.NewCounter().WithGC()
|
||||||
|
|
||||||
// CCCheckpoint ${cc.arg}
|
// CCCheckpoint ${cc.arg}
|
||||||
// TODO implement more traffic rules
|
// TODO implement more traffic rules
|
||||||
type CCCheckpoint struct {
|
type CCCheckpoint struct {
|
||||||
Checkpoint
|
Checkpoint
|
||||||
|
|
||||||
cache *ttlcache.Cache
|
|
||||||
once sync.Once
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (this *CCCheckpoint) Init() {
|
func (this *CCCheckpoint) Init() {
|
||||||
@@ -24,33 +21,25 @@ func (this *CCCheckpoint) Init() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (this *CCCheckpoint) Start() {
|
func (this *CCCheckpoint) Start() {
|
||||||
if this.cache != nil {
|
|
||||||
this.cache.Destroy()
|
|
||||||
}
|
|
||||||
this.cache = ttlcache.NewCache()
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (this *CCCheckpoint) RequestValue(req requests.Request, param string, options maps.Map, ruleId int64) (value interface{}, hasRequestBody bool, sysErr error, userErr error) {
|
func (this *CCCheckpoint) RequestValue(req requests.Request, param string, options maps.Map, ruleId int64) (value interface{}, hasRequestBody bool, sysErr error, userErr error) {
|
||||||
value = 0
|
value = 0
|
||||||
|
|
||||||
if this.cache == nil {
|
|
||||||
this.once.Do(func() {
|
|
||||||
this.Start()
|
|
||||||
})
|
|
||||||
if this.cache == nil {
|
|
||||||
return
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
periodString, ok := options["period"]
|
periodString, ok := options["period"]
|
||||||
if !ok {
|
if !ok {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
period := types.Int64(periodString)
|
var period = types.Int(periodString)
|
||||||
if period < 1 {
|
if period < 1 {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if period > 7*86400 {
|
||||||
|
period = 7 * 86400
|
||||||
|
}
|
||||||
|
|
||||||
v, _ := options["userType"]
|
v, _ := options["userType"]
|
||||||
userType := types.String(v)
|
userType := types.String(v)
|
||||||
|
|
||||||
@@ -114,7 +103,7 @@ func (this *CCCheckpoint) RequestValue(req requests.Request, param string, optio
|
|||||||
if len(key) == 0 {
|
if len(key) == 0 {
|
||||||
key = req.WAFRemoteIP()
|
key = req.WAFRemoteIP()
|
||||||
}
|
}
|
||||||
value = this.cache.IncreaseInt64(types.String(ruleId)+"@"+key, int64(1), time.Now().Unix()+period, false)
|
value = ccCounter.IncreaseKey(types.String(ruleId)+"@"+key, types.Int(period))
|
||||||
}
|
}
|
||||||
|
|
||||||
return
|
return
|
||||||
@@ -203,8 +192,5 @@ func (this *CCCheckpoint) Options() []OptionInterface {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (this *CCCheckpoint) Stop() {
|
func (this *CCCheckpoint) Stop() {
|
||||||
if this.cache != nil {
|
|
||||||
this.cache.Destroy()
|
|
||||||
this.cache = nil
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -4,17 +4,16 @@ package checkpoints
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/ttlcache"
|
"github.com/TeaOSLab/EdgeNode/internal/utils/counters"
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/waf/requests"
|
"github.com/TeaOSLab/EdgeNode/internal/waf/requests"
|
||||||
"github.com/TeaOSLab/EdgeNode/internal/zero"
|
"github.com/TeaOSLab/EdgeNode/internal/zero"
|
||||||
"github.com/iwind/TeaGo/maps"
|
"github.com/iwind/TeaGo/maps"
|
||||||
"github.com/iwind/TeaGo/types"
|
"github.com/iwind/TeaGo/types"
|
||||||
"path/filepath"
|
"path/filepath"
|
||||||
"strings"
|
"strings"
|
||||||
"time"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
var ccCache = ttlcache.NewCache()
|
var cc2Counter = counters.NewCounter().WithGC()
|
||||||
|
|
||||||
var commonFileExtensionsMap = map[string]zero.Zero{
|
var commonFileExtensionsMap = map[string]zero.Zero{
|
||||||
".ico": zero.New(),
|
".ico": zero.New(),
|
||||||
@@ -47,9 +46,11 @@ func (this *CC2Checkpoint) RequestValue(req requests.Request, param string, opti
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
var period = options.GetInt64("period")
|
var period = options.GetInt("period")
|
||||||
if period <= 0 {
|
if period <= 0 {
|
||||||
period = 60
|
period = 60
|
||||||
|
} else if period > 7*86400 {
|
||||||
|
period = 7 * 86400
|
||||||
}
|
}
|
||||||
|
|
||||||
var threshold = options.GetInt64("threshold")
|
var threshold = options.GetInt64("threshold")
|
||||||
@@ -71,9 +72,8 @@ func (this *CC2Checkpoint) RequestValue(req requests.Request, param string, opti
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
var expiresAt = time.Now().Unix() + period
|
|
||||||
var ccKey = "WAF-CC-" + types.String(ruleId) + "-" + strings.Join(keyValues, "@")
|
var ccKey = "WAF-CC-" + types.String(ruleId) + "-" + strings.Join(keyValues, "@")
|
||||||
value = ccCache.IncreaseInt64(ccKey, 1, expiresAt, false)
|
value = cc2Counter.IncreaseKey(ccKey, period)
|
||||||
|
|
||||||
// 基于指纹统计
|
// 基于指纹统计
|
||||||
var enableFingerprint = true
|
var enableFingerprint = true
|
||||||
@@ -92,8 +92,8 @@ func (this *CC2Checkpoint) RequestValue(req requests.Request, param string, opti
|
|||||||
fpKeyValues = append(fpKeyValues, req.Format(types.String(key)))
|
fpKeyValues = append(fpKeyValues, req.Format(types.String(key)))
|
||||||
}
|
}
|
||||||
var fpCCKey = "WAF-CC-" + types.String(ruleId) + "-" + strings.Join(fpKeyValues, "@")
|
var fpCCKey = "WAF-CC-" + types.String(ruleId) + "-" + strings.Join(fpKeyValues, "@")
|
||||||
var fpValue = ccCache.IncreaseInt64(fpCCKey, 1, expiresAt, false)
|
var fpValue = cc2Counter.IncreaseKey(fpCCKey, period)
|
||||||
if fpValue > value.(int64) {
|
if fpValue > value.(uint64) {
|
||||||
value = fpValue
|
value = fpValue
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
7
internal/waf/counter.go
Normal file
7
internal/waf/counter.go
Normal file
@@ -0,0 +1,7 @@
|
|||||||
|
// Copyright 2023 GoEdge CDN goedge.cdn@gmail.com. All rights reserved. Official site: https://goedge.cn .
|
||||||
|
|
||||||
|
package waf
|
||||||
|
|
||||||
|
import "github.com/TeaOSLab/EdgeNode/internal/utils/counters"
|
||||||
|
|
||||||
|
var SharedCounter = counters.NewCounter().WithGC()
|
||||||
Reference in New Issue
Block a user