mirror of
				https://github.com/TeaOSLab/EdgeNode.git
				synced 2025-11-04 07:40:56 +08:00 
			
		
		
		
	实现新的计数器算法(将时间分片, 统计更加精准)
This commit is contained in:
		@@ -10,9 +10,9 @@ import (
 | 
			
		||||
	teaconst "github.com/TeaOSLab/EdgeNode/internal/const"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeNode/internal/iplibrary"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeNode/internal/stats"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeNode/internal/ttlcache"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeNode/internal/utils"
 | 
			
		||||
	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/waf"
 | 
			
		||||
	"github.com/iwind/TeaGo/Tea"
 | 
			
		||||
@@ -24,6 +24,8 @@ import (
 | 
			
		||||
	"time"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
var synFloodCounter = counters.NewCounter().WithGC()
 | 
			
		||||
 | 
			
		||||
// ClientConn 客户端连接
 | 
			
		||||
type ClientConn struct {
 | 
			
		||||
	BaseClientConn
 | 
			
		||||
@@ -289,14 +291,13 @@ func (this *ClientConn) LastErr() error {
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (this *ClientConn) resetSYNFlood() {
 | 
			
		||||
	ttlcache.SharedCache.Delete("SYN_FLOOD:" + this.RawIP())
 | 
			
		||||
	synFloodCounter.ResetKey("SYN_FLOOD:" + this.RawIP())
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (this *ClientConn) increaseSYNFlood(synFloodConfig *firewallconfigs.SYNFloodConfig) {
 | 
			
		||||
	var ip = this.RawIP()
 | 
			
		||||
	if len(ip) > 0 && !iplibrary.IsInWhiteList(ip) && (!synFloodConfig.IgnoreLocal || !utils.IsLocalIP(ip)) {
 | 
			
		||||
		var timestamp = fasttime.Now().UnixNextMinute()
 | 
			
		||||
		var result = ttlcache.SharedCache.IncreaseInt64("SYN_FLOOD:"+ip, 1, timestamp, true)
 | 
			
		||||
		var result = synFloodCounter.IncreaseKey("SYN_FLOOD:"+ip, 60)
 | 
			
		||||
		var minAttempts = synFloodConfig.MinAttempts
 | 
			
		||||
		if minAttempts < 5 {
 | 
			
		||||
			minAttempts = 5
 | 
			
		||||
@@ -305,7 +306,7 @@ func (this *ClientConn) increaseSYNFlood(synFloodConfig *firewallconfigs.SYNFloo
 | 
			
		||||
			// 非TLS,设置为两倍,防止误封
 | 
			
		||||
			minAttempts = 2 * minAttempts
 | 
			
		||||
		}
 | 
			
		||||
		if result >= int64(minAttempts) {
 | 
			
		||||
		if result >= types.Uint64(minAttempts) {
 | 
			
		||||
			var timeout = synFloodConfig.TimeoutSeconds
 | 
			
		||||
			if timeout <= 0 {
 | 
			
		||||
				timeout = 600
 | 
			
		||||
 
 | 
			
		||||
@@ -213,7 +213,7 @@ func BenchmarkCache_Add_Parallel(b *testing.B) {
 | 
			
		||||
	b.RunParallel(func(pb *testing.PB) {
 | 
			
		||||
		for pb.Next() {
 | 
			
		||||
			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 {
 | 
			
		||||
	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"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeCommon/pkg/nodeconfigs"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeCommon/pkg/serverconfigs/firewallconfigs"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeNode/internal/ttlcache"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeNode/internal/waf/requests"
 | 
			
		||||
	"github.com/iwind/TeaGo/types"
 | 
			
		||||
	"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 countFails = ttlcache.SharedCache.IncreaseInt64(key, 1, time.Now().Unix()+300, true)
 | 
			
		||||
	var countFails = SharedCounter.IncreaseKey(key, 300)
 | 
			
		||||
	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)+"次")
 | 
			
		||||
		return false
 | 
			
		||||
 
 | 
			
		||||
@@ -4,7 +4,6 @@ package waf
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"github.com/TeaOSLab/EdgeCommon/pkg/serverconfigs/firewallconfigs"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeNode/internal/ttlcache"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeNode/internal/utils"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeNode/internal/waf/requests"
 | 
			
		||||
	"github.com/iwind/TeaGo/types"
 | 
			
		||||
@@ -27,7 +26,7 @@ func CaptchaIncreaseFails(req requests.Request, actionConfig *CaptchaAction, pol
 | 
			
		||||
		if maxFails <= 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 {
 | 
			
		||||
			SharedIPBlackList.RecordIP(IPTypeAll, firewallconfigs.FirewallScopeService, req.WAFServerId(), req.WAFRemoteIP(), time.Now().Unix()+int64(failBlockTimeout), policyId, true, groupId, setId, "CAPTCHA验证连续失败超过"+types.String(maxFails)+"次")
 | 
			
		||||
			return false
 | 
			
		||||
@@ -38,9 +37,9 @@ func CaptchaIncreaseFails(req requests.Request, actionConfig *CaptchaAction, pol
 | 
			
		||||
 | 
			
		||||
// CaptchaDeleteCacheKey 清除计数
 | 
			
		||||
func CaptchaDeleteCacheKey(req requests.Request) {
 | 
			
		||||
	ttlcache.SharedCache.Delete(CaptchaCacheKey(req, CaptchaPageCodeInit))
 | 
			
		||||
	ttlcache.SharedCache.Delete(CaptchaCacheKey(req, CaptchaPageCodeShow))
 | 
			
		||||
	ttlcache.SharedCache.Delete(CaptchaCacheKey(req, CaptchaPageCodeSubmit))
 | 
			
		||||
	SharedCounter.ResetKey(CaptchaCacheKey(req, CaptchaPageCodeInit))
 | 
			
		||||
	SharedCounter.ResetKey(CaptchaCacheKey(req, CaptchaPageCodeShow))
 | 
			
		||||
	SharedCounter.ResetKey(CaptchaCacheKey(req, CaptchaPageCodeSubmit))
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// CaptchaCacheKey 获取Captcha缓存Key
 | 
			
		||||
 
 | 
			
		||||
@@ -1,22 +1,19 @@
 | 
			
		||||
package checkpoints
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"github.com/TeaOSLab/EdgeNode/internal/ttlcache"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeNode/internal/utils/counters"
 | 
			
		||||
	"github.com/TeaOSLab/EdgeNode/internal/waf/requests"
 | 
			
		||||
	"github.com/iwind/TeaGo/maps"
 | 
			
		||||
	"github.com/iwind/TeaGo/types"
 | 
			
		||||
	"regexp"
 | 
			
		||||
	"sync"
 | 
			
		||||
	"time"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
var ccCounter = counters.NewCounter().WithGC()
 | 
			
		||||
 | 
			
		||||
// CCCheckpoint ${cc.arg}
 | 
			
		||||
// TODO implement more traffic rules
 | 
			
		||||
type CCCheckpoint struct {
 | 
			
		||||
	Checkpoint
 | 
			
		||||
 | 
			
		||||
	cache *ttlcache.Cache
 | 
			
		||||
	once  sync.Once
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (this *CCCheckpoint) Init() {
 | 
			
		||||
@@ -24,33 +21,25 @@ func (this *CCCheckpoint) Init() {
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
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) {
 | 
			
		||||
	value = 0
 | 
			
		||||
 | 
			
		||||
	if this.cache == nil {
 | 
			
		||||
		this.once.Do(func() {
 | 
			
		||||
			this.Start()
 | 
			
		||||
		})
 | 
			
		||||
		if this.cache == nil {
 | 
			
		||||
			return
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	periodString, ok := options["period"]
 | 
			
		||||
	if !ok {
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
	period := types.Int64(periodString)
 | 
			
		||||
	var period = types.Int(periodString)
 | 
			
		||||
	if period < 1 {
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	if period > 7*86400 {
 | 
			
		||||
		period = 7 * 86400
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	v, _ := options["userType"]
 | 
			
		||||
	userType := types.String(v)
 | 
			
		||||
 | 
			
		||||
@@ -114,7 +103,7 @@ func (this *CCCheckpoint) RequestValue(req requests.Request, param string, optio
 | 
			
		||||
		if len(key) == 0 {
 | 
			
		||||
			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
 | 
			
		||||
@@ -203,8 +192,5 @@ func (this *CCCheckpoint) Options() []OptionInterface {
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (this *CCCheckpoint) Stop() {
 | 
			
		||||
	if this.cache != nil {
 | 
			
		||||
		this.cache.Destroy()
 | 
			
		||||
		this.cache = nil
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
}
 | 
			
		||||
 
 | 
			
		||||
@@ -4,17 +4,16 @@ package checkpoints
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"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/zero"
 | 
			
		||||
	"github.com/iwind/TeaGo/maps"
 | 
			
		||||
	"github.com/iwind/TeaGo/types"
 | 
			
		||||
	"path/filepath"
 | 
			
		||||
	"strings"
 | 
			
		||||
	"time"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
var ccCache = ttlcache.NewCache()
 | 
			
		||||
var cc2Counter = counters.NewCounter().WithGC()
 | 
			
		||||
 | 
			
		||||
var commonFileExtensionsMap = map[string]zero.Zero{
 | 
			
		||||
	".ico":   zero.New(),
 | 
			
		||||
@@ -47,9 +46,11 @@ func (this *CC2Checkpoint) RequestValue(req requests.Request, param string, opti
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	var period = options.GetInt64("period")
 | 
			
		||||
	var period = options.GetInt("period")
 | 
			
		||||
	if period <= 0 {
 | 
			
		||||
		period = 60
 | 
			
		||||
	} else if period > 7*86400 {
 | 
			
		||||
		period = 7 * 86400
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	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, "@")
 | 
			
		||||
	value = ccCache.IncreaseInt64(ccKey, 1, expiresAt, false)
 | 
			
		||||
	value = cc2Counter.IncreaseKey(ccKey, period)
 | 
			
		||||
 | 
			
		||||
	// 基于指纹统计
 | 
			
		||||
	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)))
 | 
			
		||||
			}
 | 
			
		||||
			var fpCCKey = "WAF-CC-" + types.String(ruleId) + "-" + strings.Join(fpKeyValues, "@")
 | 
			
		||||
			var fpValue = ccCache.IncreaseInt64(fpCCKey, 1, expiresAt, false)
 | 
			
		||||
			if fpValue > value.(int64) {
 | 
			
		||||
			var fpValue = cc2Counter.IncreaseKey(fpCCKey, period)
 | 
			
		||||
			if fpValue > value.(uint64) {
 | 
			
		||||
				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