Message protocol update (message size) (#663)
* feat(tracker): update message schema with BatchMetadata; separate message-related responsibilities; add message size * chore(docker): removing edge busybox, as the main repo updated * feat(backend): updated message protocol templates * feat(backend): added support of message size * feat(backend): implemented iterator for new message protocol (with message size) Co-authored-by: Alex Kaminskii <alex@openreplay.com>
This commit is contained in:
parent
8cda6bb1f1
commit
5887ab5ddb
74 changed files with 7016 additions and 4015 deletions
|
|
@ -57,7 +57,7 @@ ENV TZ=UTC \
|
|||
PARTITIONS_NUMBER=16 \
|
||||
QUEUE_MESSAGE_SIZE_LIMIT=1048576 \
|
||||
BEACON_SIZE_LIMIT=1000000 \
|
||||
USE_FAILOVER=false \
|
||||
USE_FAILOVER=true \
|
||||
GROUP_STORAGE_FAILOVER=failover \
|
||||
TOPIC_STORAGE_FAILOVER=storage-failover
|
||||
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@ package main
|
|||
import (
|
||||
"context"
|
||||
"log"
|
||||
"openreplay/backend/pkg/monitoring"
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
"os"
|
||||
"os/signal"
|
||||
"syscall"
|
||||
|
|
@ -13,8 +13,8 @@ import (
|
|||
"openreplay/backend/internal/assets/cacher"
|
||||
config "openreplay/backend/internal/config/assets"
|
||||
"openreplay/backend/pkg/messages"
|
||||
"openreplay/backend/pkg/monitoring"
|
||||
"openreplay/backend/pkg/queue"
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
)
|
||||
|
||||
func main() {
|
||||
|
|
@ -34,22 +34,25 @@ func main() {
|
|||
consumer := queue.NewMessageConsumer(
|
||||
cfg.GroupCache,
|
||||
[]string{cfg.TopicCache},
|
||||
func(sessionID uint64, message messages.Message, e *types.Meta) {
|
||||
switch msg := message.(type) {
|
||||
case *messages.AssetCache:
|
||||
cacher.CacheURL(sessionID, msg.URL)
|
||||
totalAssets.Add(context.Background(), 1)
|
||||
case *messages.ErrorEvent:
|
||||
if msg.Source != "js_exception" {
|
||||
return
|
||||
}
|
||||
sourceList, err := assets.ExtractJSExceptionSources(&msg.Payload)
|
||||
if err != nil {
|
||||
log.Printf("Error on source extraction: %v", err)
|
||||
return
|
||||
}
|
||||
for _, source := range sourceList {
|
||||
cacher.CacheJSFile(source)
|
||||
func(sessionID uint64, iter messages.Iterator, meta *types.Meta) {
|
||||
for iter.Next() {
|
||||
if iter.Type() == messages.MsgAssetCache {
|
||||
msg := iter.Message().Decode().(*messages.AssetCache)
|
||||
cacher.CacheURL(sessionID, msg.URL)
|
||||
totalAssets.Add(context.Background(), 1)
|
||||
} else if iter.Type() == messages.MsgErrorEvent {
|
||||
msg := iter.Message().Decode().(*messages.ErrorEvent)
|
||||
if msg.Source != "js_exception" {
|
||||
continue
|
||||
}
|
||||
sourceList, err := assets.ExtractJSExceptionSources(&msg.Payload)
|
||||
if err != nil {
|
||||
log.Printf("Error on source extraction: %v", err)
|
||||
continue
|
||||
}
|
||||
for _, source := range sourceList {
|
||||
cacher.CacheJSFile(source)
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
|
|
|
|||
|
|
@ -3,24 +3,23 @@ package main
|
|||
import (
|
||||
"errors"
|
||||
"log"
|
||||
"openreplay/backend/internal/config/db"
|
||||
"openreplay/backend/internal/db/datasaver"
|
||||
"openreplay/backend/pkg/handlers"
|
||||
custom2 "openreplay/backend/pkg/handlers/custom"
|
||||
"openreplay/backend/pkg/monitoring"
|
||||
"openreplay/backend/pkg/sessions"
|
||||
"time"
|
||||
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
"os"
|
||||
"os/signal"
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"openreplay/backend/internal/config/db"
|
||||
"openreplay/backend/internal/db/datasaver"
|
||||
"openreplay/backend/pkg/db/cache"
|
||||
"openreplay/backend/pkg/db/postgres"
|
||||
"openreplay/backend/pkg/handlers"
|
||||
custom2 "openreplay/backend/pkg/handlers/custom"
|
||||
logger "openreplay/backend/pkg/log"
|
||||
"openreplay/backend/pkg/messages"
|
||||
"openreplay/backend/pkg/monitoring"
|
||||
"openreplay/backend/pkg/queue"
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
"openreplay/backend/pkg/sessions"
|
||||
)
|
||||
|
||||
func main() {
|
||||
|
|
@ -51,49 +50,60 @@ func main() {
|
|||
saver.InitStats()
|
||||
statsLogger := logger.NewQueueStats(cfg.LoggerTimeout)
|
||||
|
||||
keepMessage := func(tp int) bool {
|
||||
return tp == messages.MsgMetadata || tp == messages.MsgIssueEvent || tp == messages.MsgSessionStart || tp == messages.MsgSessionEnd || tp == messages.MsgUserID || tp == messages.MsgUserAnonymousID || tp == messages.MsgCustomEvent || tp == messages.MsgClickEvent || tp == messages.MsgInputEvent || tp == messages.MsgPageEvent || tp == messages.MsgErrorEvent || tp == messages.MsgFetchEvent || tp == messages.MsgGraphQLEvent || tp == messages.MsgIntegrationEvent || tp == messages.MsgPerformanceTrackAggr || tp == messages.MsgResourceEvent || tp == messages.MsgLongTask || tp == messages.MsgJSException || tp == messages.MsgResourceTiming || tp == messages.MsgRawCustomEvent || tp == messages.MsgCustomIssue || tp == messages.MsgFetch || tp == messages.MsgGraphQL || tp == messages.MsgStateAction || tp == messages.MsgSetInputTarget || tp == messages.MsgSetInputValue || tp == messages.MsgCreateDocument || tp == messages.MsgMouseClick || tp == messages.MsgSetPageLocation || tp == messages.MsgPageLoadTiming || tp == messages.MsgPageRenderTiming
|
||||
}
|
||||
|
||||
// Handler logic
|
||||
handler := func(sessionID uint64, msg messages.Message, meta *types.Meta) {
|
||||
handler := func(sessionID uint64, iter messages.Iterator, meta *types.Meta) {
|
||||
statsLogger.Collect(sessionID, meta)
|
||||
|
||||
// Just save session data into db without additional checks
|
||||
if err := saver.InsertMessage(sessionID, msg); err != nil {
|
||||
if !postgres.IsPkeyViolation(err) {
|
||||
log.Printf("Message Insertion Error %v, SessionID: %v, Message: %v", err, sessionID, msg)
|
||||
for iter.Next() {
|
||||
if !keepMessage(iter.Type()) {
|
||||
continue
|
||||
}
|
||||
return
|
||||
}
|
||||
msg := iter.Message().Decode()
|
||||
log.Printf("process message, type: %d", iter.Type())
|
||||
|
||||
session, err := pg.GetSession(sessionID)
|
||||
if session == nil {
|
||||
if err != nil && !errors.Is(err, cache.NilSessionInCacheError) {
|
||||
log.Printf("Error on session retrieving from cache: %v, SessionID: %v, Message: %v", err, sessionID, msg)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Save statistics to db
|
||||
err = saver.InsertStats(session, msg)
|
||||
if err != nil {
|
||||
log.Printf("Stats Insertion Error %v; Session: %v, Message: %v", err, session, msg)
|
||||
}
|
||||
|
||||
// Handle heuristics and save to temporary queue in memory
|
||||
builderMap.HandleMessage(sessionID, msg, msg.Meta().Index)
|
||||
|
||||
// Process saved heuristics messages as usual messages above in the code
|
||||
builderMap.IterateSessionReadyMessages(sessionID, func(msg messages.Message) {
|
||||
// TODO: DRY code (carefully with the return statement logic)
|
||||
// Just save session data into db without additional checks
|
||||
if err := saver.InsertMessage(sessionID, msg); err != nil {
|
||||
if !postgres.IsPkeyViolation(err) {
|
||||
log.Printf("Message Insertion Error %v; Session: %v, Message %v", err, session, msg)
|
||||
log.Printf("Message Insertion Error %v, SessionID: %v, Message: %v", err, sessionID, msg)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
if err := saver.InsertStats(session, msg); err != nil {
|
||||
log.Printf("Stats Insertion Error %v; Session: %v, Message %v", err, session, msg)
|
||||
session, err := pg.GetSession(sessionID)
|
||||
if session == nil {
|
||||
if err != nil && !errors.Is(err, cache.NilSessionInCacheError) {
|
||||
log.Printf("Error on session retrieving from cache: %v, SessionID: %v, Message: %v", err, sessionID, msg)
|
||||
}
|
||||
return
|
||||
}
|
||||
})
|
||||
|
||||
// Save statistics to db
|
||||
err = saver.InsertStats(session, msg)
|
||||
if err != nil {
|
||||
log.Printf("Stats Insertion Error %v; Session: %v, Message: %v", err, session, msg)
|
||||
}
|
||||
|
||||
// Handle heuristics and save to temporary queue in memory
|
||||
builderMap.HandleMessage(sessionID, msg, msg.Meta().Index)
|
||||
|
||||
// Process saved heuristics messages as usual messages above in the code
|
||||
builderMap.IterateSessionReadyMessages(sessionID, func(msg messages.Message) {
|
||||
if err := saver.InsertMessage(sessionID, msg); err != nil {
|
||||
if !postgres.IsPkeyViolation(err) {
|
||||
log.Printf("Message Insertion Error %v; Session: %v, Message %v", err, session, msg)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
if err := saver.InsertStats(session, msg); err != nil {
|
||||
log.Printf("Stats Insertion Error %v; Session: %v, Message %v", err, session, msg)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// Init consumer
|
||||
|
|
|
|||
|
|
@ -2,25 +2,23 @@ package main
|
|||
|
||||
import (
|
||||
"log"
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
"os"
|
||||
"os/signal"
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"openreplay/backend/internal/config/ender"
|
||||
"openreplay/backend/internal/sessionender"
|
||||
"openreplay/backend/pkg/db/cache"
|
||||
"openreplay/backend/pkg/db/postgres"
|
||||
"openreplay/backend/pkg/monitoring"
|
||||
"time"
|
||||
|
||||
"os"
|
||||
"os/signal"
|
||||
"syscall"
|
||||
|
||||
"openreplay/backend/pkg/intervals"
|
||||
logger "openreplay/backend/pkg/log"
|
||||
"openreplay/backend/pkg/messages"
|
||||
"openreplay/backend/pkg/monitoring"
|
||||
"openreplay/backend/pkg/queue"
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
)
|
||||
|
||||
//
|
||||
func main() {
|
||||
metrics := monitoring.New("ender")
|
||||
|
||||
|
|
@ -45,18 +43,17 @@ func main() {
|
|||
[]string{
|
||||
cfg.TopicRawWeb,
|
||||
},
|
||||
func(sessionID uint64, msg messages.Message, meta *types.Meta) {
|
||||
switch msg.(type) {
|
||||
case *messages.SessionStart, *messages.SessionEnd:
|
||||
// Skip several message types
|
||||
return
|
||||
func(sessionID uint64, iter messages.Iterator, meta *types.Meta) {
|
||||
for iter.Next() {
|
||||
if iter.Type() == messages.MsgSessionStart || iter.Type() == messages.MsgSessionEnd {
|
||||
continue
|
||||
}
|
||||
if iter.Message().Meta().Timestamp == 0 {
|
||||
log.Printf("ZERO TS, sessID: %d, msgType: %d", sessionID, iter.Type())
|
||||
}
|
||||
statsLogger.Collect(sessionID, meta)
|
||||
sessions.UpdateSession(sessionID, meta.Timestamp, iter.Message().Meta().Timestamp)
|
||||
}
|
||||
// Test debug
|
||||
if msg.Meta().Timestamp == 0 {
|
||||
log.Printf("ZERO TS, sessID: %d, msgType: %d", sessionID, msg.TypeID())
|
||||
}
|
||||
statsLogger.Collect(sessionID, meta)
|
||||
sessions.UpdateSession(sessionID, meta.Timestamp, msg.Meta().Timestamp)
|
||||
},
|
||||
false,
|
||||
cfg.MessageSizeLimit,
|
||||
|
|
|
|||
|
|
@ -2,6 +2,12 @@ package main
|
|||
|
||||
import (
|
||||
"log"
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
"os"
|
||||
"os/signal"
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"openreplay/backend/internal/config/heuristics"
|
||||
"openreplay/backend/pkg/handlers"
|
||||
web2 "openreplay/backend/pkg/handlers/web"
|
||||
|
|
@ -9,12 +15,7 @@ import (
|
|||
logger "openreplay/backend/pkg/log"
|
||||
"openreplay/backend/pkg/messages"
|
||||
"openreplay/backend/pkg/queue"
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
"openreplay/backend/pkg/sessions"
|
||||
"os"
|
||||
"os/signal"
|
||||
"syscall"
|
||||
"time"
|
||||
)
|
||||
|
||||
func main() {
|
||||
|
|
@ -55,9 +56,11 @@ func main() {
|
|||
[]string{
|
||||
cfg.TopicRawWeb,
|
||||
},
|
||||
func(sessionID uint64, msg messages.Message, meta *types.Meta) {
|
||||
statsLogger.Collect(sessionID, meta)
|
||||
builderMap.HandleMessage(sessionID, msg, msg.Meta().Index)
|
||||
func(sessionID uint64, iter messages.Iterator, meta *types.Meta) {
|
||||
for iter.Next() {
|
||||
statsLogger.Collect(sessionID, meta)
|
||||
builderMap.HandleMessage(sessionID, iter.Message().Decode(), iter.Message().Meta().Index)
|
||||
}
|
||||
},
|
||||
false,
|
||||
cfg.MessageSizeLimit,
|
||||
|
|
|
|||
|
|
@ -2,22 +2,20 @@ package main
|
|||
|
||||
import (
|
||||
"context"
|
||||
"encoding/binary"
|
||||
"log"
|
||||
"openreplay/backend/internal/sink/assetscache"
|
||||
"openreplay/backend/internal/sink/oswriter"
|
||||
"openreplay/backend/internal/storage"
|
||||
"openreplay/backend/pkg/monitoring"
|
||||
"time"
|
||||
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
"os"
|
||||
"os/signal"
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"openreplay/backend/internal/config/sink"
|
||||
"openreplay/backend/internal/sink/assetscache"
|
||||
"openreplay/backend/internal/sink/oswriter"
|
||||
"openreplay/backend/internal/storage"
|
||||
. "openreplay/backend/pkg/messages"
|
||||
"openreplay/backend/pkg/monitoring"
|
||||
"openreplay/backend/pkg/queue"
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
"openreplay/backend/pkg/url/assets"
|
||||
)
|
||||
|
||||
|
|
@ -58,51 +56,49 @@ func main() {
|
|||
[]string{
|
||||
cfg.TopicRawWeb,
|
||||
},
|
||||
func(sessionID uint64, message Message, _ *types.Meta) {
|
||||
// Process assets
|
||||
message = assetMessageHandler.ParseAssets(sessionID, message)
|
||||
func(sessionID uint64, iter Iterator, meta *types.Meta) {
|
||||
for iter.Next() {
|
||||
// [METRICS] Increase the number of processed messages
|
||||
totalMessages.Add(context.Background(), 1)
|
||||
|
||||
totalMessages.Add(context.Background(), 1)
|
||||
|
||||
// Filter message
|
||||
typeID := message.TypeID()
|
||||
|
||||
// Send SessionEnd trigger to storage service
|
||||
switch message.(type) {
|
||||
case *SessionEnd:
|
||||
if err := producer.Produce(cfg.TopicTrigger, sessionID, Encode(message)); err != nil {
|
||||
log.Printf("can't send SessionEnd to trigger topic: %s; sessID: %d", err, sessionID)
|
||||
// Send SessionEnd trigger to storage service
|
||||
if iter.Type() == MsgSessionEnd {
|
||||
if err := producer.Produce(cfg.TopicTrigger, sessionID, iter.Message().Encode()); err != nil {
|
||||
log.Printf("can't send SessionEnd to trigger topic: %s; sessID: %d", err, sessionID)
|
||||
}
|
||||
continue
|
||||
}
|
||||
return
|
||||
}
|
||||
if !IsReplayerType(typeID) {
|
||||
return
|
||||
}
|
||||
|
||||
// If message timestamp is empty, use at least ts of session start
|
||||
ts := message.Meta().Timestamp
|
||||
if ts == 0 {
|
||||
log.Printf("zero ts; sessID: %d, msg: %+v", sessionID, message)
|
||||
} else {
|
||||
// Log ts of last processed message
|
||||
counter.Update(sessionID, time.UnixMilli(ts))
|
||||
}
|
||||
msg := iter.Message()
|
||||
// Process assets
|
||||
if iter.Type() == MsgSetNodeAttributeURLBased || iter.Type() == MsgSetCSSDataURLBased || iter.Type() == MsgCSSInsertRuleURLBased {
|
||||
msg = assetMessageHandler.ParseAssets(sessionID, msg.Decode())
|
||||
}
|
||||
|
||||
value := message.Encode()
|
||||
var data []byte
|
||||
if IsIOSType(typeID) {
|
||||
data = value
|
||||
} else {
|
||||
data = make([]byte, len(value)+8)
|
||||
copy(data[8:], value[:])
|
||||
binary.LittleEndian.PutUint64(data[0:], message.Meta().Index)
|
||||
}
|
||||
if err := writer.Write(sessionID, data); err != nil {
|
||||
log.Printf("Writer error: %v\n", err)
|
||||
}
|
||||
// Filter message
|
||||
if !IsReplayerType(msg.TypeID()) {
|
||||
continue
|
||||
}
|
||||
|
||||
messageSize.Record(context.Background(), float64(len(data)))
|
||||
savedMessages.Add(context.Background(), 1)
|
||||
// If message timestamp is empty, use at least ts of session start
|
||||
ts := msg.Meta().Timestamp
|
||||
if ts == 0 {
|
||||
log.Printf("zero ts; sessID: %d, msgType: %d", sessionID, iter.Type())
|
||||
} else {
|
||||
// Log ts of last processed message
|
||||
counter.Update(sessionID, time.UnixMilli(ts))
|
||||
}
|
||||
|
||||
// Write encoded message with index to session file
|
||||
data := msg.EncodeWithIndex()
|
||||
if err := writer.Write(sessionID, data); err != nil {
|
||||
log.Printf("Writer error: %v\n", err)
|
||||
}
|
||||
|
||||
// [METRICS] Increase the number of written to the files messages and the message size
|
||||
messageSize.Record(context.Background(), float64(len(data)))
|
||||
savedMessages.Add(context.Background(), 1)
|
||||
}
|
||||
},
|
||||
false,
|
||||
cfg.MessageSizeLimit,
|
||||
|
|
|
|||
|
|
@ -2,8 +2,7 @@ package main
|
|||
|
||||
import (
|
||||
"log"
|
||||
"openreplay/backend/pkg/failover"
|
||||
"openreplay/backend/pkg/monitoring"
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
"os"
|
||||
"os/signal"
|
||||
"strconv"
|
||||
|
|
@ -12,9 +11,10 @@ import (
|
|||
|
||||
config "openreplay/backend/internal/config/storage"
|
||||
"openreplay/backend/internal/storage"
|
||||
"openreplay/backend/pkg/failover"
|
||||
"openreplay/backend/pkg/messages"
|
||||
"openreplay/backend/pkg/monitoring"
|
||||
"openreplay/backend/pkg/queue"
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
s3storage "openreplay/backend/pkg/storage"
|
||||
)
|
||||
|
||||
|
|
@ -43,14 +43,17 @@ func main() {
|
|||
[]string{
|
||||
cfg.TopicTrigger,
|
||||
},
|
||||
func(sessionID uint64, msg messages.Message, meta *types.Meta) {
|
||||
switch m := msg.(type) {
|
||||
case *messages.SessionEnd:
|
||||
if err := srv.UploadKey(strconv.FormatUint(sessionID, 10), 5); err != nil {
|
||||
sessionFinder.Find(sessionID, m.Timestamp)
|
||||
func(sessionID uint64, iter messages.Iterator, meta *types.Meta) {
|
||||
for iter.Next() {
|
||||
if iter.Type() == messages.MsgSessionEnd {
|
||||
msg := iter.Message().Decode().(*messages.SessionEnd)
|
||||
if err := srv.UploadKey(strconv.FormatUint(sessionID, 10), 5); err != nil {
|
||||
log.Printf("can't find session: %d", sessionID)
|
||||
sessionFinder.Find(sessionID, msg.Timestamp)
|
||||
}
|
||||
// Log timestamp of last processed session
|
||||
counter.Update(sessionID, time.UnixMilli(meta.Timestamp))
|
||||
}
|
||||
// Log timestamp of last processed session
|
||||
counter.Update(sessionID, time.UnixMilli(meta.Timestamp))
|
||||
}
|
||||
},
|
||||
true,
|
||||
|
|
|
|||
|
|
@ -71,6 +71,7 @@ func (w *Writer) Write(key uint64, data []byte) error {
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// TODO: add check for the number of recorded bytes to file
|
||||
_, err = file.Write(data)
|
||||
return err
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,67 +1,153 @@
|
|||
package messages
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/pkg/errors"
|
||||
"bytes"
|
||||
"io"
|
||||
"log"
|
||||
"strings"
|
||||
)
|
||||
|
||||
func ReadBatchReader(reader io.Reader, messageHandler func(Message)) error {
|
||||
var index uint64
|
||||
var timestamp int64
|
||||
type Iterator interface {
|
||||
Next() bool // Return true if we have next message
|
||||
Type() int // Return type of the next message
|
||||
Message() Message // Return raw or decoded message
|
||||
}
|
||||
|
||||
for {
|
||||
msg, err := ReadMessage(reader)
|
||||
type iteratorImpl struct {
|
||||
data *bytes.Reader
|
||||
index uint64
|
||||
timestamp int64
|
||||
version uint64
|
||||
msgType uint64
|
||||
msgSize uint64
|
||||
canSkip bool
|
||||
msg Message
|
||||
}
|
||||
|
||||
func NewIterator(data []byte) Iterator {
|
||||
return &iteratorImpl{
|
||||
data: bytes.NewReader(data),
|
||||
}
|
||||
}
|
||||
|
||||
func (i *iteratorImpl) Next() bool {
|
||||
if i.canSkip {
|
||||
if _, err := i.data.Seek(int64(i.msgSize), io.SeekCurrent); err != nil {
|
||||
log.Printf("seek err: %s", err)
|
||||
return false
|
||||
}
|
||||
}
|
||||
i.canSkip = false
|
||||
|
||||
var err error
|
||||
i.msgType, err = ReadUint(i.data)
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
return nil
|
||||
return false
|
||||
}
|
||||
log.Printf("can't read message type: %s", err)
|
||||
return false
|
||||
}
|
||||
|
||||
if i.version > 0 && messageHasSize(i.msgType) {
|
||||
// Read message size if it is a new protocol version
|
||||
i.msgSize, err = ReadSize(i.data)
|
||||
if err != nil {
|
||||
log.Printf("can't read message size: %s", err)
|
||||
return false
|
||||
}
|
||||
i.msg = &RawMessage{
|
||||
tp: i.msgType,
|
||||
size: i.msgSize,
|
||||
meta: &message{},
|
||||
reader: i.data,
|
||||
skipped: &i.canSkip,
|
||||
}
|
||||
i.canSkip = true
|
||||
} else {
|
||||
i.msg, err = ReadMessage(i.msgType, i.data)
|
||||
if err == io.EOF {
|
||||
return false
|
||||
} else if err != nil {
|
||||
if strings.HasPrefix(err.Error(), "Unknown message code:") {
|
||||
code := strings.TrimPrefix(err.Error(), "Unknown message code: ")
|
||||
msg, err = DecodeExtraMessage(code, reader)
|
||||
i.msg, err = DecodeExtraMessage(code, i.data)
|
||||
if err != nil {
|
||||
return fmt.Errorf("can't decode msg: %s", err)
|
||||
log.Printf("can't decode msg: %s", err)
|
||||
return false
|
||||
}
|
||||
} else {
|
||||
return errors.Wrapf(err, "Batch Message decoding error on message with index %v", index)
|
||||
log.Printf("Batch Message decoding error on message with index %v, err: %s", i.index, err)
|
||||
return false
|
||||
}
|
||||
}
|
||||
msg = transformDeprecated(msg)
|
||||
|
||||
isBatchMeta := false
|
||||
switch m := msg.(type) {
|
||||
case *BatchMeta: // Is not required to be present in batch since IOS doesn't have it (though we might change it)
|
||||
if index != 0 { // Might be several 0-0 BatchMeta in a row without a error though
|
||||
return errors.New("Batch Meta found at the end of the batch")
|
||||
}
|
||||
index = m.PageNo<<32 + m.FirstIndex // 2^32 is the maximum count of messages per page (ha-ha)
|
||||
timestamp = m.Timestamp
|
||||
isBatchMeta = true
|
||||
// continue readLoop
|
||||
case *IOSBatchMeta:
|
||||
if index != 0 { // Might be several 0-0 BatchMeta in a row without a error though
|
||||
return errors.New("Batch Meta found at the end of the batch")
|
||||
}
|
||||
index = m.FirstIndex
|
||||
timestamp = int64(m.Timestamp)
|
||||
isBatchMeta = true
|
||||
// continue readLoop
|
||||
case *Timestamp:
|
||||
timestamp = int64(m.Timestamp) // TODO(?): replace timestamp type to int64 everywhere (including encoding part in tracker)
|
||||
// No skipping here for making it easy to encode back the same sequence of message
|
||||
// continue readLoop
|
||||
case *SessionStart:
|
||||
timestamp = int64(m.Timestamp)
|
||||
case *SessionEnd:
|
||||
timestamp = int64(m.Timestamp)
|
||||
}
|
||||
msg.Meta().Index = index
|
||||
msg.Meta().Timestamp = timestamp
|
||||
|
||||
messageHandler(msg)
|
||||
if !isBatchMeta { // Without that indexes will be unique anyway, though shifted by 1 because BatchMeta is not counted in tracker
|
||||
index++
|
||||
}
|
||||
i.msg = transformDeprecated(i.msg)
|
||||
}
|
||||
return errors.New("Error of the codeflow. (Should return on EOF)")
|
||||
|
||||
// Process meta information
|
||||
isBatchMeta := false
|
||||
switch i.msgType {
|
||||
case MsgBatchMetadata:
|
||||
if i.index != 0 { // Might be several 0-0 BatchMeta in a row without an error though
|
||||
log.Printf("Batch Meta found at the end of the batch")
|
||||
return false
|
||||
}
|
||||
m := i.msg.Decode().(*BatchMetadata)
|
||||
i.index = m.PageNo<<32 + m.FirstIndex // 2^32 is the maximum count of messages per page (ha-ha)
|
||||
i.timestamp = m.Timestamp
|
||||
i.version = m.Version
|
||||
isBatchMeta = true
|
||||
log.Printf("new batch version: %d", i.version)
|
||||
|
||||
case MsgBatchMeta: // Is not required to be present in batch since IOS doesn't have it (though we might change it)
|
||||
if i.index != 0 { // Might be several 0-0 BatchMeta in a row without an error though
|
||||
log.Printf("Batch Meta found at the end of the batch")
|
||||
return false
|
||||
}
|
||||
m := i.msg.Decode().(*BatchMeta)
|
||||
i.index = m.PageNo<<32 + m.FirstIndex // 2^32 is the maximum count of messages per page (ha-ha)
|
||||
i.timestamp = m.Timestamp
|
||||
isBatchMeta = true
|
||||
// continue readLoop
|
||||
case MsgIOSBatchMeta:
|
||||
if i.index != 0 { // Might be several 0-0 BatchMeta in a row without an error though
|
||||
log.Printf("Batch Meta found at the end of the batch")
|
||||
return false
|
||||
}
|
||||
m := i.msg.Decode().(*IOSBatchMeta)
|
||||
i.index = m.FirstIndex
|
||||
i.timestamp = int64(m.Timestamp)
|
||||
isBatchMeta = true
|
||||
// continue readLoop
|
||||
case MsgTimestamp:
|
||||
m := i.msg.Decode().(*Timestamp)
|
||||
i.timestamp = int64(m.Timestamp)
|
||||
// No skipping here for making it easy to encode back the same sequence of message
|
||||
// continue readLoop
|
||||
case MsgSessionStart:
|
||||
m := i.msg.Decode().(*SessionStart)
|
||||
i.timestamp = int64(m.Timestamp)
|
||||
case MsgSessionEnd:
|
||||
m := i.msg.Decode().(*SessionEnd)
|
||||
i.timestamp = int64(m.Timestamp)
|
||||
}
|
||||
i.msg.Meta().Index = i.index
|
||||
i.msg.Meta().Timestamp = i.timestamp
|
||||
|
||||
if !isBatchMeta { // Without that indexes will be unique anyway, though shifted by 1 because BatchMeta is not counted in tracker
|
||||
i.index++
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func (i *iteratorImpl) Type() int {
|
||||
return int(i.msgType)
|
||||
}
|
||||
|
||||
func (i *iteratorImpl) Message() Message {
|
||||
return i.msg
|
||||
}
|
||||
|
||||
func messageHasSize(msgType uint64) bool {
|
||||
return !(msgType == 80 || msgType == 81 || msgType == 82)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +1,7 @@
|
|||
package messages
|
||||
|
||||
import (
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"io"
|
||||
)
|
||||
|
|
@ -20,6 +21,21 @@ func (msg *SessionSearch) Encode() []byte {
|
|||
return buf[:p]
|
||||
}
|
||||
|
||||
func (msg *SessionSearch) EncodeWithIndex() []byte {
|
||||
encoded := msg.Encode()
|
||||
if IsIOSType(msg.TypeID()) {
|
||||
return encoded
|
||||
}
|
||||
data := make([]byte, len(encoded)+8)
|
||||
copy(data[8:], encoded[:])
|
||||
binary.LittleEndian.PutUint64(data[0:], msg.Meta().Index)
|
||||
return data
|
||||
}
|
||||
|
||||
func (msg *SessionSearch) Decode() Message {
|
||||
return msg
|
||||
}
|
||||
|
||||
func (msg *SessionSearch) TypeID() int {
|
||||
return 127
|
||||
}
|
||||
|
|
|
|||
|
|
@ -16,6 +16,8 @@ func (m *message) SetMeta(origin *message) {
|
|||
|
||||
type Message interface {
|
||||
Encode() []byte
|
||||
EncodeWithIndex() []byte
|
||||
Decode() Message
|
||||
TypeID() int
|
||||
Meta() *message
|
||||
}
|
||||
|
|
|
|||
File diff suppressed because it is too large
Load diff
|
|
@ -3,6 +3,7 @@ package messages
|
|||
import (
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"log"
|
||||
)
|
||||
|
|
@ -16,15 +17,6 @@ func ReadByte(reader io.Reader) (byte, error) {
|
|||
return p[0], nil
|
||||
}
|
||||
|
||||
// func SkipBytes(reader io.ReadSeeker) error {
|
||||
// n, err := ReadUint(reader)
|
||||
// if err != nil {
|
||||
// return err
|
||||
// }
|
||||
// _, err = reader.Seek(n, io.SeekCurrent);
|
||||
// return err
|
||||
// }
|
||||
|
||||
func ReadData(reader io.Reader) ([]byte, error) {
|
||||
n, err := ReadUint(reader)
|
||||
if err != nil {
|
||||
|
|
@ -153,3 +145,28 @@ func WriteJson(v interface{}, buf []byte, p int) int {
|
|||
}
|
||||
return WriteData(data, buf, p)
|
||||
}
|
||||
|
||||
func WriteSize(size uint64, buf []byte, p int) {
|
||||
var m uint64 = 255
|
||||
for i := 0; i < 3; i++ {
|
||||
buf[p+i] = byte(size & m)
|
||||
size = size >> 8
|
||||
}
|
||||
fmt.Println(buf)
|
||||
}
|
||||
|
||||
func ReadSize(reader io.Reader) (uint64, error) {
|
||||
buf := make([]byte, 3)
|
||||
n, err := io.ReadFull(reader, buf)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
if n != 3 {
|
||||
return 0, fmt.Errorf("read only %d of 3 size bytes", n)
|
||||
}
|
||||
var size uint64
|
||||
for i, b := range buf {
|
||||
size += uint64(b) << (8 * i)
|
||||
}
|
||||
return size, nil
|
||||
}
|
||||
|
|
|
|||
69
backend/pkg/messages/raw.go
Normal file
69
backend/pkg/messages/raw.go
Normal file
|
|
@ -0,0 +1,69 @@
|
|||
package messages
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"io"
|
||||
"log"
|
||||
)
|
||||
|
||||
// RawMessage is a not decoded message
|
||||
type RawMessage struct {
|
||||
tp uint64
|
||||
size uint64
|
||||
data []byte
|
||||
reader *bytes.Reader
|
||||
meta *message
|
||||
encoded bool
|
||||
skipped *bool
|
||||
}
|
||||
|
||||
func (m *RawMessage) Encode() []byte {
|
||||
if m.encoded {
|
||||
return m.data
|
||||
}
|
||||
m.data = make([]byte, m.size+1)
|
||||
m.data[0] = uint8(m.tp)
|
||||
m.encoded = true
|
||||
*m.skipped = false
|
||||
n, err := io.ReadFull(m.reader, m.data[1:])
|
||||
if err != nil {
|
||||
log.Printf("message encode err: %s", err)
|
||||
return nil
|
||||
}
|
||||
log.Printf("encode: read %d of %d bytes", n, m.size)
|
||||
return m.data
|
||||
}
|
||||
|
||||
func (m *RawMessage) EncodeWithIndex() []byte {
|
||||
if !m.encoded {
|
||||
m.Encode()
|
||||
}
|
||||
if IsIOSType(int(m.tp)) {
|
||||
return m.data
|
||||
}
|
||||
data := make([]byte, len(m.data)+8)
|
||||
copy(data[8:], m.data[:])
|
||||
binary.LittleEndian.PutUint64(data[0:], m.Meta().Index)
|
||||
return data
|
||||
}
|
||||
|
||||
func (m *RawMessage) Decode() Message {
|
||||
if !m.encoded {
|
||||
m.Encode()
|
||||
}
|
||||
msg, err := ReadMessage(m.tp, bytes.NewReader(m.data[1:]))
|
||||
if err != nil {
|
||||
log.Printf("decode err: %s", err)
|
||||
}
|
||||
msg.Meta().SetMeta(m.meta)
|
||||
return msg
|
||||
}
|
||||
|
||||
func (m *RawMessage) TypeID() int {
|
||||
return int(m.tp)
|
||||
}
|
||||
|
||||
func (m *RawMessage) Meta() *message {
|
||||
return m.meta
|
||||
}
|
||||
File diff suppressed because it is too large
Load diff
|
|
@ -1,19 +1,12 @@
|
|||
package queue
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"log"
|
||||
|
||||
"openreplay/backend/pkg/messages"
|
||||
"openreplay/backend/pkg/queue/types"
|
||||
)
|
||||
|
||||
func NewMessageConsumer(group string, topics []string, handler types.DecodedMessageHandler, autoCommit bool, messageSizeLimit int) types.Consumer {
|
||||
func NewMessageConsumer(group string, topics []string, handler types.RawMessageHandler, autoCommit bool, messageSizeLimit int) types.Consumer {
|
||||
return NewConsumer(group, topics, func(sessionID uint64, value []byte, meta *types.Meta) {
|
||||
if err := messages.ReadBatchReader(bytes.NewReader(value), func(msg messages.Message) {
|
||||
handler(sessionID, msg, meta)
|
||||
}); err != nil {
|
||||
log.Printf("Decode error: %v\n", err)
|
||||
}
|
||||
handler(sessionID, messages.NewIterator(value), meta)
|
||||
}, autoCommit, messageSizeLimit)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -26,3 +26,4 @@ type Meta struct {
|
|||
|
||||
type MessageHandler func(uint64, []byte, *Meta)
|
||||
type DecodedMessageHandler func(uint64, messages.Message, *Meta)
|
||||
type RawMessageHandler func(uint64, messages.Iterator, *Meta)
|
||||
|
|
|
|||
|
|
@ -60,10 +60,12 @@ func NewSessionFinder(cfg *config.Config, stg *storage.Storage) (SessionFinder,
|
|||
[]string{
|
||||
cfg.TopicFailover,
|
||||
},
|
||||
func(sessionID uint64, msg messages.Message, meta *types.Meta) {
|
||||
switch m := msg.(type) {
|
||||
case *messages.SessionSearch:
|
||||
finder.findSession(sessionID, m.Timestamp, m.Partition)
|
||||
func(sessionID uint64, iter messages.Iterator, meta *types.Meta) {
|
||||
for iter.Next() {
|
||||
if iter.Type() == 127 {
|
||||
m := iter.Message().Decode().(*messages.SessionSearch)
|
||||
finder.findSession(sessionID, m.Timestamp, m.Partition)
|
||||
}
|
||||
}
|
||||
},
|
||||
true,
|
||||
|
|
|
|||
|
|
@ -1,13 +1,31 @@
|
|||
# Special one for Batch Meta. Message id could define the version
|
||||
message 80, 'BatchMeta', :replayer => false do
|
||||
# Depricated since tracker 3.6.0 in favor of BatchMetadata
|
||||
message 80, 'BatchMeta', :tracker => false, :replayer => false do
|
||||
uint 'PageNo'
|
||||
uint 'FirstIndex'
|
||||
int 'Timestamp'
|
||||
end
|
||||
|
||||
# since tracker 3.6.0
|
||||
message 81, 'BatchMetadata', :replayer => false do
|
||||
uint 'Version'
|
||||
uint 'PageNo'
|
||||
uint 'FirstIndex'
|
||||
int 'Timestamp'
|
||||
string 'Location'
|
||||
end
|
||||
|
||||
# since tracker 3.6.0
|
||||
message 82, 'PartitionedMessage', :replayer => false do
|
||||
uint 'PartNo'
|
||||
uint 'PartTotal'
|
||||
end
|
||||
|
||||
|
||||
message 0, 'Timestamp' do
|
||||
uint 'Timestamp'
|
||||
end
|
||||
message 1, 'SessionStart', :js => false, :replayer => false do
|
||||
message 1, 'SessionStart', :tracker => false, :replayer => false do
|
||||
uint 'Timestamp'
|
||||
uint 'ProjectID'
|
||||
string 'TrackerVersion'
|
||||
|
|
@ -26,10 +44,10 @@ message 1, 'SessionStart', :js => false, :replayer => false do
|
|||
string 'UserID'
|
||||
end
|
||||
# Depricated (not used) since OpenReplay tracker 3.0.0
|
||||
message 2, 'SessionDisconnect', :js => false do
|
||||
message 2, 'SessionDisconnect', :tracker => false do
|
||||
uint 'Timestamp'
|
||||
end
|
||||
message 3, 'SessionEnd', :js => false, :replayer => false do
|
||||
message 3, 'SessionEnd', :tracker => false, :replayer => false do
|
||||
uint 'Timestamp'
|
||||
end
|
||||
message 4, 'SetPageLocation' do
|
||||
|
|
@ -80,8 +98,7 @@ message 14, 'SetNodeData' do
|
|||
uint 'ID'
|
||||
string 'Data'
|
||||
end
|
||||
# Depricated starting from 5.5.11 in favor of SetStyleData
|
||||
message 15, 'SetCSSData', :js => false do
|
||||
message 15, 'SetCSSData', :tracker => false do
|
||||
uint 'ID'
|
||||
string 'Data'
|
||||
end
|
||||
|
|
@ -108,7 +125,7 @@ message 20, 'MouseMove' do
|
|||
uint 'Y'
|
||||
end
|
||||
# Depricated since OpenReplay 1.2.0
|
||||
message 21, 'MouseClickDepricated', :js => false, :replayer => false do
|
||||
message 21, 'MouseClickDepricated', :tracker => false, :replayer => false do
|
||||
uint 'ID'
|
||||
uint 'HesitationTime'
|
||||
string 'Label'
|
||||
|
|
@ -138,7 +155,7 @@ message 25, 'JSException', :replayer => false do
|
|||
string 'Message'
|
||||
string 'Payload'
|
||||
end
|
||||
message 26, 'IntegrationEvent', :js => false, :replayer => false do
|
||||
message 26, 'IntegrationEvent', :tracker => false, :replayer => false do
|
||||
uint 'Timestamp'
|
||||
string 'Source'
|
||||
string 'Name'
|
||||
|
|
@ -159,7 +176,7 @@ message 30, 'Metadata', :replayer => false do
|
|||
string 'Key'
|
||||
string 'Value'
|
||||
end
|
||||
message 31, 'PageEvent', :js => false, :replayer => false do
|
||||
message 31, 'PageEvent', :tracker => false, :replayer => false do
|
||||
uint 'MessageID'
|
||||
uint 'Timestamp'
|
||||
string 'URL'
|
||||
|
|
@ -178,21 +195,21 @@ message 31, 'PageEvent', :js => false, :replayer => false do
|
|||
uint 'VisuallyComplete'
|
||||
uint 'TimeToInteractive'
|
||||
end
|
||||
message 32, 'InputEvent', :js => false, :replayer => false do
|
||||
message 32, 'InputEvent', :tracker => false, :replayer => false do
|
||||
uint 'MessageID'
|
||||
uint 'Timestamp'
|
||||
string 'Value'
|
||||
boolean 'ValueMasked'
|
||||
string 'Label'
|
||||
end
|
||||
message 33, 'ClickEvent', :js => false, :replayer => false do
|
||||
message 33, 'ClickEvent', :tracker => false, :replayer => false do
|
||||
uint 'MessageID'
|
||||
uint 'Timestamp'
|
||||
uint 'HesitationTime'
|
||||
string 'Label'
|
||||
string 'Selector'
|
||||
end
|
||||
message 34, 'ErrorEvent', :js => false, :replayer => false do
|
||||
message 34, 'ErrorEvent', :tracker => false, :replayer => false do
|
||||
uint 'MessageID'
|
||||
uint 'Timestamp'
|
||||
string 'Source'
|
||||
|
|
@ -200,7 +217,7 @@ message 34, 'ErrorEvent', :js => false, :replayer => false do
|
|||
string 'Message'
|
||||
string 'Payload'
|
||||
end
|
||||
message 35, 'ResourceEvent', :js => false, :replayer => false do
|
||||
message 35, 'ResourceEvent', :tracker => false, :replayer => false do
|
||||
uint 'MessageID'
|
||||
uint 'Timestamp'
|
||||
uint 'Duration'
|
||||
|
|
@ -214,7 +231,7 @@ message 35, 'ResourceEvent', :js => false, :replayer => false do
|
|||
string 'Method'
|
||||
uint 'Status'
|
||||
end
|
||||
message 36, 'CustomEvent', :js => false, :replayer => false do
|
||||
message 36, 'CustomEvent', :tracker => false, :replayer => false do
|
||||
uint 'MessageID'
|
||||
uint 'Timestamp'
|
||||
string 'Name'
|
||||
|
|
@ -255,7 +272,7 @@ end
|
|||
message 42, 'StateAction', :replayer => false do
|
||||
string 'Type'
|
||||
end
|
||||
message 43, 'StateActionEvent', :js => false, :replayer => false do
|
||||
message 43, 'StateActionEvent', :tracker => false, :replayer => false do
|
||||
uint 'MessageID'
|
||||
uint 'Timestamp'
|
||||
string 'Type'
|
||||
|
|
@ -291,7 +308,7 @@ message 49, 'PerformanceTrack' do
|
|||
uint 'TotalJSHeapSize'
|
||||
uint 'UsedJSHeapSize'
|
||||
end
|
||||
message 50, 'GraphQLEvent', :js => false, :replayer => false do
|
||||
message 50, 'GraphQLEvent', :tracker => false, :replayer => false do
|
||||
uint 'MessageID'
|
||||
uint 'Timestamp'
|
||||
string 'OperationKind'
|
||||
|
|
@ -299,7 +316,7 @@ message 50, 'GraphQLEvent', :js => false, :replayer => false do
|
|||
string 'Variables'
|
||||
string 'Response'
|
||||
end
|
||||
message 51, 'FetchEvent', :js => false, :replayer => false do
|
||||
message 51, 'FetchEvent', :tracker => false, :replayer => false do
|
||||
uint 'MessageID'
|
||||
uint 'Timestamp'
|
||||
string 'Method'
|
||||
|
|
@ -309,7 +326,7 @@ message 51, 'FetchEvent', :js => false, :replayer => false do
|
|||
uint 'Status'
|
||||
uint 'Duration'
|
||||
end
|
||||
message 52, 'DOMDrop', :js => false, :replayer => false do
|
||||
message 52, 'DOMDrop', :tracker => false, :replayer => false do
|
||||
uint 'Timestamp'
|
||||
end
|
||||
message 53, 'ResourceTiming', :replayer => false do
|
||||
|
|
@ -329,7 +346,7 @@ end
|
|||
message 55, 'SetPageVisibility' do
|
||||
boolean 'hidden'
|
||||
end
|
||||
message 56, 'PerformanceTrackAggr', :js => false, :replayer => false do
|
||||
message 56, 'PerformanceTrackAggr', :tracker => false, :replayer => false do
|
||||
uint 'TimestampStart'
|
||||
uint 'TimestampEnd'
|
||||
uint 'MinFPS'
|
||||
|
|
@ -366,7 +383,7 @@ message 61, 'SetCSSDataURLBased', :replayer => false do
|
|||
string 'Data'
|
||||
string 'BaseURL'
|
||||
end
|
||||
message 62, 'IssueEvent', :replayer => false, :js => false do
|
||||
message 62, 'IssueEvent', :replayer => false, :tracker => false do
|
||||
uint 'MessageID'
|
||||
uint 'Timestamp'
|
||||
string 'Type'
|
||||
|
|
@ -382,11 +399,7 @@ message 64, 'CustomIssue', :replayer => false do
|
|||
string 'Name'
|
||||
string 'Payload'
|
||||
end
|
||||
# Since 5.6.6; only for websocket (might be probably replaced with ws.close())
|
||||
# Depricated
|
||||
message 65, 'PageClose', :replayer => false do
|
||||
end
|
||||
message 66, 'AssetCache', :replayer => false, :js => false do
|
||||
message 66, 'AssetCache', :replayer => false, :tracker => false do
|
||||
string 'URL'
|
||||
end
|
||||
message 67, 'CSSInsertRuleURLBased', :replayer => false do
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ func SkipBytes(reader io.ReadSeeker) error {
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
_, err := reader.Seek(n, io.SeekCurrent);
|
||||
_, err = reader.Seek(int64(n), io.SeekCurrent)
|
||||
return err
|
||||
}
|
||||
|
||||
|
|
@ -29,13 +29,13 @@ func ReadData(reader io.Reader) ([]byte, error) {
|
|||
return nil, err
|
||||
}
|
||||
p := make([]byte, n)
|
||||
_, err := io.ReadFull(reader, p)
|
||||
_, err = io.ReadFull(reader, p)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return p, nil
|
||||
}
|
||||
|
||||
|
||||
func ReadUint(reader io.Reader) (uint64, error) {
|
||||
var x uint64
|
||||
var s uint
|
||||
|
|
|
|||
31
mobs/run.rb
31
mobs/run.rb
|
|
@ -3,24 +3,23 @@ require 'erb'
|
|||
|
||||
# TODO: change method names to correct (CapitalCase and camelCase, not CamalCase and firstLower)
|
||||
class String
|
||||
def camel_case
|
||||
return self if self !~ /_/ && self =~ /[A-Z]+.*/
|
||||
split('_').map{|e| e.capitalize}.join.upperize
|
||||
end
|
||||
|
||||
def camel_case_lower
|
||||
self.split('_').inject([]){ |buffer,e| buffer.push(buffer.empty? ? e : e.capitalize) }.join.upperize
|
||||
end
|
||||
|
||||
def upperize
|
||||
def upperize_abbreviations
|
||||
self.sub('Id', 'ID').sub('Url', 'URL')
|
||||
end
|
||||
|
||||
def first_lower
|
||||
# pascal_case
|
||||
def pascal_case
|
||||
return self if self !~ /_/ && self =~ /[A-Z]+.*/
|
||||
split('_').map{|e| e.capitalize}.join.upperize_abbreviations
|
||||
end
|
||||
|
||||
# camelCase
|
||||
def camel_case
|
||||
self.sub(/^[A-Z]+/) {|f| f.downcase }
|
||||
end
|
||||
|
||||
def underscore
|
||||
# snake_case
|
||||
def snake_case
|
||||
self.gsub(/::/, '/').
|
||||
gsub(/([A-Z]+)([A-Z][a-z])/,'\1_\2').
|
||||
gsub(/([a-z\d])([A-Z])/,'\1_\2').
|
||||
|
|
@ -85,16 +84,17 @@ end
|
|||
$context = :web
|
||||
|
||||
class Message
|
||||
attr_reader :id, :name, :js, :replayer, :swift, :seq_index, :attributes, :context
|
||||
def initialize(name:, id:, js: $context == :web, replayer: $context == :web, swift: $context == :ios, seq_index: false, &block)
|
||||
attr_reader :id, :name, :tracker, :replayer, :swift, :seq_index, :attributes, :context
|
||||
def initialize(name:, id:, tracker: $context == :web, replayer: $context == :web, swift: $context == :ios, seq_index: false, &block)
|
||||
@id = id
|
||||
@name = name
|
||||
@js = js
|
||||
@tracker = tracker
|
||||
@replayer = replayer
|
||||
@swift = swift
|
||||
@seq_index = seq_index
|
||||
@context = $context
|
||||
@attributes = []
|
||||
# opts.each { |key, value| send "#{key}=", value }
|
||||
instance_eval &block
|
||||
end
|
||||
|
||||
|
|
@ -131,6 +131,7 @@ Dir["templates/*.erb"].each do |tpl|
|
|||
path = tpl.split '/'
|
||||
t = '../' + path[1].gsub('~', '/')
|
||||
t = t[0..-5]
|
||||
# TODO: .gen subextention
|
||||
File.write(t, e.result)
|
||||
puts tpl + ' --> ' + t
|
||||
end
|
||||
|
|
|
|||
|
|
@ -1,5 +1,12 @@
|
|||
// Auto-generated, do not edit
|
||||
package messages
|
||||
|
||||
const (
|
||||
<% $messages.each do |msg| %>
|
||||
Msg<%= msg.name %> = <%= msg.id %>
|
||||
<% end %>
|
||||
)
|
||||
|
||||
<% $messages.each do |msg| %>
|
||||
type <%= msg.name %> struct {
|
||||
message
|
||||
|
|
@ -12,10 +19,25 @@ func (msg *<%= msg.name %>) Encode() []byte {
|
|||
buf[0] = <%= msg.id %>
|
||||
p := 1
|
||||
<%= msg.attributes.map { |attr|
|
||||
" p = Write#{attr.type.to_s.camel_case}(msg.#{attr.name}, buf, p)" }.join "\n" %>
|
||||
" p = Write#{attr.type.to_s.pascal_case}(msg.#{attr.name}, buf, p)" }.join "\n" %>
|
||||
return buf[:p]
|
||||
}
|
||||
|
||||
func (msg *<%= msg.name %>) EncodeWithIndex() []byte {
|
||||
encoded := msg.Encode()
|
||||
if IsIOSType(msg.TypeID()) {
|
||||
return encoded
|
||||
}
|
||||
data := make([]byte, len(encoded)+8)
|
||||
copy(data[8:], encoded[:])
|
||||
binary.LittleEndian.PutUint64(data[0:], msg.Meta().Index)
|
||||
return data
|
||||
}
|
||||
|
||||
func (msg *<%= msg.name %>) Decode() Message {
|
||||
return msg
|
||||
}
|
||||
|
||||
func (msg *<%= msg.name %>) TypeID() int {
|
||||
return <%= msg.id %>
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,20 +6,24 @@ import (
|
|||
"io"
|
||||
)
|
||||
|
||||
func ReadMessage(reader io.Reader) (Message, error) {
|
||||
t, err := ReadUint(reader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
<% $messages.each do |msg| %>
|
||||
func Decode<%= msg.name %>(reader io.Reader) (Message, error) {
|
||||
var err error = nil
|
||||
msg := &<%= msg.name %>{}
|
||||
<%= msg.attributes.map { |attr|
|
||||
" if msg.#{attr.name}, err = Read#{attr.type.to_s.pascal_case}(reader); err != nil {
|
||||
return nil, err
|
||||
}" }.join "\n" %>
|
||||
return msg, err
|
||||
}
|
||||
|
||||
<% end %>
|
||||
|
||||
func ReadMessage(t uint64, reader io.Reader) (Message, error) {
|
||||
switch t {
|
||||
<% $messages.each do |msg| %>
|
||||
case <%= msg.id %>:
|
||||
msg := &<%= msg.name %>{}
|
||||
<%= msg.attributes.map { |attr|
|
||||
" if msg.#{attr.name}, err = Read#{attr.type.to_s.camel_case}(reader); err != nil {
|
||||
return nil, err
|
||||
}" }.join "\n" %>
|
||||
return msg, nil
|
||||
return Decode<%= msg.name %>(reader)
|
||||
<% end %>
|
||||
}
|
||||
return nil, fmt.Errorf("Unknown message code: %v", t)
|
||||
|
|
|
|||
|
|
@ -9,8 +9,8 @@ class Message(ABC):
|
|||
class <%= msg.name %>(Message):
|
||||
__id__ = <%= msg.id %>
|
||||
|
||||
def __init__(self, <%= msg.attributes.map { |attr| "#{attr.name.underscore}" }.join ", " %>):
|
||||
<%= msg.attributes.map { |attr| "self.#{attr.name.underscore} = #{attr.name.underscore}" }.join "\n "
|
||||
def __init__(self, <%= msg.attributes.map { |attr| "#{attr.name.snake_case}" }.join ", " %>):
|
||||
<%= msg.attributes.map { |attr| "self.#{attr.name.snake_case} = #{attr.name.snake_case}" }.join "\n "
|
||||
%>
|
||||
|
||||
<% end %>
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ class MessageCodec(Codec):
|
|||
if message_id == <%= msg.id %>:
|
||||
return <%= msg.name %>(
|
||||
<%= msg.attributes.map { |attr|
|
||||
"#{attr.name.underscore}=self.read_#{attr.type.to_s}(reader)" }
|
||||
"#{attr.name.snake_case}=self.read_#{attr.type.to_s}(reader)" }
|
||||
.join ",\n "
|
||||
%>
|
||||
)
|
||||
|
|
|
|||
|
|
@ -16,14 +16,14 @@ export default class RawMessageReader extends PrimitiveReader {
|
|||
if (tp === null) { return resetPointer() }
|
||||
|
||||
switch (tp) {
|
||||
<% $messages.select { |msg| msg.js || msg.replayer }.each do |msg| %>
|
||||
<% $messages.select { |msg| msg.tracker || msg.replayer }.each do |msg| %>
|
||||
case <%= msg.id %>: {
|
||||
<%= msg.attributes.map { |attr|
|
||||
" const #{attr.name.first_lower} = this.read#{attr.type.to_s.camel_case}(); if (#{attr.name.first_lower} === null) { return resetPointer() }" }.join "\n" %>
|
||||
" const #{attr.name.camel_case} = this.read#{attr.type.to_s.pascal_case}(); if (#{attr.name.camel_case} === null) { return resetPointer() }" }.join "\n" %>
|
||||
return {
|
||||
tp: "<%= msg.name.underscore %>",
|
||||
tp: "<%= msg.name.snake_case %>",
|
||||
<%= msg.attributes.map { |attr|
|
||||
" #{attr.name.first_lower}," }.join "\n" %>
|
||||
" #{attr.name.camel_case}," }.join "\n" %>
|
||||
};
|
||||
}
|
||||
<% end %>
|
||||
|
|
|
|||
|
|
@ -3,11 +3,11 @@
|
|||
import type { Timed } from './timed'
|
||||
import type { RawMessage } from './raw'
|
||||
import type {
|
||||
<%= $messages.select { |msg| msg.js || msg.replayer }.map { |msg| " Raw#{msg.name.underscore.camel_case}," }.join "\n" %>
|
||||
<%= $messages.select { |msg| msg.tracker || msg.replayer }.map { |msg| " Raw#{msg.name.snake_case.pascal_case}," }.join "\n" %>
|
||||
} from './raw'
|
||||
|
||||
export type Message = RawMessage & Timed
|
||||
|
||||
<% $messages.select { |msg| msg.js || msg.replayer }.each do |msg| %>
|
||||
export type <%= msg.name.underscore.camel_case %> = Raw<%= msg.name.underscore.camel_case %> & Timed
|
||||
<% $messages.select { |msg| msg.tracker || msg.replayer }.each do |msg| %>
|
||||
export type <%= msg.name.snake_case.pascal_case %> = Raw<%= msg.name.snake_case.pascal_case %> & Timed
|
||||
<% end %>
|
||||
|
|
@ -1,14 +1,14 @@
|
|||
// Auto-generated, do not edit
|
||||
|
||||
export const TP_MAP = {
|
||||
<%= $messages.select { |msg| msg.js || msg.replayer }.map { |msg| " #{msg.id}: \"#{msg.name.underscore}\"," }.join "\n" %>
|
||||
<%= $messages.select { |msg| msg.tracker || msg.replayer }.map { |msg| " #{msg.id}: \"#{msg.name.snake_case}\"," }.join "\n" %>
|
||||
}
|
||||
|
||||
<% $messages.select { |msg| msg.js || msg.replayer }.each do |msg| %>
|
||||
export interface Raw<%= msg.name.underscore.camel_case %> {
|
||||
tp: "<%= msg.name.underscore %>",
|
||||
<%= msg.attributes.map { |attr| " #{attr.name.first_lower}: #{attr.type_js}," }.join "\n" %>
|
||||
<% $messages.select { |msg| msg.tracker || msg.replayer }.each do |msg| %>
|
||||
export interface Raw<%= msg.name.snake_case.pascal_case %> {
|
||||
tp: "<%= msg.name.snake_case %>",
|
||||
<%= msg.attributes.map { |attr| " #{attr.name.camel_case}: #{attr.type_js}," }.join "\n" %>
|
||||
}
|
||||
<% end %>
|
||||
|
||||
export type RawMessage = <%= $messages.select { |msg| msg.js || msg.replayer }.map { |msg| "Raw#{msg.name.underscore.camel_case}" }.join " | " %>;
|
||||
export type RawMessage = <%= $messages.select { |msg| msg.tracker || msg.replayer }.map { |msg| "Raw#{msg.name.snake_case.pascal_case}" }.join " | " %>;
|
||||
|
|
|
|||
|
|
@ -2,15 +2,15 @@
|
|||
import UIKit
|
||||
|
||||
enum ASMessageType: UInt64 {
|
||||
<%= $messages.map { |msg| " case #{msg.name.first_lower} = #{msg.id}" }.join "\n" %>
|
||||
<%= $messages.map { |msg| " case #{msg.name.camel_case} = #{msg.id}" }.join "\n" %>
|
||||
}
|
||||
<% $messages.each do |msg| %>
|
||||
class AS<%= msg.name.to_s.camel_case %>: ASMessage {
|
||||
class AS<%= msg.name.to_s.pascal_case %>: ASMessage {
|
||||
<%= msg.attributes[2..-1].map { |attr| " let #{attr.property}: #{attr.type_swift}" }.join "\n" %>
|
||||
|
||||
init(<%= msg.attributes[2..-1].map { |attr| "#{attr.property}: #{attr.type_swift}" }.join ", " %>) {
|
||||
<%= msg.attributes[2..-1].map { |attr| " self.#{attr.property} = #{attr.property}" }.join "\n" %>
|
||||
super.init(messageType: .<%= "#{msg.name.first_lower}" %>)
|
||||
super.init(messageType: .<%= "#{msg.name.camel_case}" %>)
|
||||
}
|
||||
|
||||
override init?(genericMessage: GenericMessage) {
|
||||
|
|
@ -30,7 +30,7 @@ class AS<%= msg.name.to_s.camel_case %>: ASMessage {
|
|||
}
|
||||
|
||||
override var description: String {
|
||||
return "-->> <%= msg.name.to_s.camel_case %>(<%= "#{msg.id}"%>): timestamp:\(timestamp) <%= msg.attributes[2..-1].map { |attr| "#{attr.property}:\\(#{attr.property})" }.join " "%>";
|
||||
return "-->> <%= msg.name.to_s.pascal_case %>(<%= "#{msg.id}"%>): timestamp:\(timestamp) <%= msg.attributes[2..-1].map { |attr| "#{attr.property}:\\(#{attr.property})" }.join " "%>";
|
||||
}
|
||||
}
|
||||
<% end %>
|
||||
|
|
|
|||
|
|
@ -0,0 +1,15 @@
|
|||
// Auto-generated, do not edit
|
||||
|
||||
export enum Type {
|
||||
<%= $messages.select { |msg| msg.tracker }.map { |msg| "#{ msg.name } = #{ msg.id }," }.join "\n " %>
|
||||
}
|
||||
|
||||
<% $messages.select { |msg| msg.tracker }.each do |msg| %>
|
||||
export type <%= msg.name %> = [
|
||||
type: Type.<%= msg.name %>,
|
||||
<%= msg.attributes.map { |attr| "#{attr.name.camel_case}: #{attr.type_js}," }.join "\n " %>
|
||||
]
|
||||
<% end %>
|
||||
|
||||
type Message = <%= $messages.select { |msg| msg.tracker }.map { |msg| "#{msg.name}" }.join " | " %>
|
||||
export default Message
|
||||
|
|
@ -1,31 +0,0 @@
|
|||
// Auto-generated, do not edit
|
||||
import type { Writer, Message }from "./types.js";
|
||||
export default Message
|
||||
|
||||
function bindNew<C extends { new(...args: A): T }, A extends any[], T>(
|
||||
Class: C & { new(...args: A): T }
|
||||
): C & ((...args: A) => T) {
|
||||
function _Class(...args: A) {
|
||||
return new Class(...args);
|
||||
}
|
||||
_Class.prototype = Class.prototype;
|
||||
return <C & ((...args: A) => T)>_Class;
|
||||
}
|
||||
|
||||
export const classes: Map<number, Function> = new Map();
|
||||
|
||||
<% $messages.select { |msg| msg.js }.each do |msg| %>
|
||||
class _<%= msg.name %> implements Message {
|
||||
readonly _id: number = <%= msg.id %>;
|
||||
constructor(
|
||||
<%= msg.attributes.map { |attr| "public #{attr.name.first_lower}: #{attr.type_js}" }.join ",\n " %>
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(<%= msg.id %>)<%= " &&" if msg.attributes.length() > 0 %>
|
||||
<%= msg.attributes.map { |attr| "writer.#{attr.type}(this.#{attr.name.first_lower})" }.join " &&\n " %>;
|
||||
}
|
||||
}
|
||||
export const <%= msg.name %> = bindNew(_<%= msg.name %>);
|
||||
classes.set(<%= msg.id %>, <%= msg.name %>);
|
||||
|
||||
<% end %>
|
||||
|
|
@ -0,0 +1,15 @@
|
|||
// Auto-generated, do not edit
|
||||
|
||||
import * as Messages from '../../common/messages.gen.js'
|
||||
export { default } from '../../common/messages.gen.js'
|
||||
|
||||
<% $messages.select { |msg| msg.tracker }.each do |msg| %>
|
||||
export function <%= msg.name %>(
|
||||
<%= msg.attributes.map { |attr| "#{attr.name.camel_case}: #{attr.type_js}," }.join "\n " %>
|
||||
): Messages.<%= msg.name %> {
|
||||
return [
|
||||
Messages.Type.<%= msg.name %>,
|
||||
<%= msg.attributes.map { |attr| "#{attr.name.camel_case}," }.join "\n " %>
|
||||
]
|
||||
}
|
||||
<% end %>
|
||||
|
|
@ -0,0 +1,20 @@
|
|||
// Auto-generated, do not edit
|
||||
|
||||
import * as Messages from '../common/messages.gen.js'
|
||||
import Message from '../common/messages.gen.js'
|
||||
import PrimitiveEncoder from './PrimitiveEncoder.js'
|
||||
|
||||
|
||||
export default class MessageEncoder extends PrimitiveEncoder {
|
||||
encode(msg: Message): boolean {
|
||||
switch(msg[0]) {
|
||||
<% $messages.select { |msg| msg.tracker }.each do |msg| %>
|
||||
case Messages.Type.<%= msg.name %>:
|
||||
return <% if msg.attributes.size == 0 %> true <% else %> <%= msg.attributes.map.with_index { |attr, index| "this.#{attr.type}(msg[#{index+1}])" }.join " && " %> <% end %>
|
||||
break
|
||||
<% end %>
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -5,4 +5,4 @@ cjs
|
|||
build
|
||||
.cache
|
||||
.eslintrc.cjs
|
||||
src/common/messages.ts
|
||||
*.gen.ts
|
||||
|
|
|
|||
1
tracker/tracker/.prettierignore
Normal file
1
tracker/tracker/.prettierignore
Normal file
|
|
@ -0,0 +1 @@
|
|||
*.gen.ts
|
||||
|
|
@ -1,5 +1,6 @@
|
|||
{
|
||||
"printWidth": 100,
|
||||
"singleQuote": true,
|
||||
"trailingComma": "all"
|
||||
"trailingComma": "all",
|
||||
"semi": false
|
||||
}
|
||||
|
|
|
|||
22
tracker/tracker/src/common/interaction.ts
Normal file
22
tracker/tracker/src/common/interaction.ts
Normal file
|
|
@ -0,0 +1,22 @@
|
|||
import Message from './messages.gen.js'
|
||||
|
||||
export interface Options {
|
||||
connAttemptCount?: number
|
||||
connAttemptGap?: number
|
||||
}
|
||||
|
||||
type Start = {
|
||||
type: 'start'
|
||||
ingestPoint: string
|
||||
pageNo: number
|
||||
timestamp: number
|
||||
url: string
|
||||
} & Options
|
||||
|
||||
type Auth = {
|
||||
type: 'auth'
|
||||
token: string
|
||||
beaconSizeLimit?: number
|
||||
}
|
||||
|
||||
export type WorkerMessageData = null | 'stop' | Start | Auth | Array<Message>
|
||||
405
tracker/tracker/src/common/messages.gen.ts
Normal file
405
tracker/tracker/src/common/messages.gen.ts
Normal file
|
|
@ -0,0 +1,405 @@
|
|||
// Auto-generated, do not edit
|
||||
|
||||
export enum Type {
|
||||
BatchMetadata = 81,
|
||||
PartitionedMessage = 82,
|
||||
Timestamp = 0,
|
||||
SetPageLocation = 4,
|
||||
SetViewportSize = 5,
|
||||
SetViewportScroll = 6,
|
||||
CreateDocument = 7,
|
||||
CreateElementNode = 8,
|
||||
CreateTextNode = 9,
|
||||
MoveNode = 10,
|
||||
RemoveNode = 11,
|
||||
SetNodeAttribute = 12,
|
||||
RemoveNodeAttribute = 13,
|
||||
SetNodeData = 14,
|
||||
SetNodeScroll = 16,
|
||||
SetInputTarget = 17,
|
||||
SetInputValue = 18,
|
||||
SetInputChecked = 19,
|
||||
MouseMove = 20,
|
||||
ConsoleLog = 22,
|
||||
PageLoadTiming = 23,
|
||||
PageRenderTiming = 24,
|
||||
JSException = 25,
|
||||
RawCustomEvent = 27,
|
||||
UserID = 28,
|
||||
UserAnonymousID = 29,
|
||||
Metadata = 30,
|
||||
CSSInsertRule = 37,
|
||||
CSSDeleteRule = 38,
|
||||
Fetch = 39,
|
||||
Profiler = 40,
|
||||
OTable = 41,
|
||||
StateAction = 42,
|
||||
Redux = 44,
|
||||
Vuex = 45,
|
||||
MobX = 46,
|
||||
NgRx = 47,
|
||||
GraphQL = 48,
|
||||
PerformanceTrack = 49,
|
||||
ResourceTiming = 53,
|
||||
ConnectionInformation = 54,
|
||||
SetPageVisibility = 55,
|
||||
LongTask = 59,
|
||||
SetNodeAttributeURLBased = 60,
|
||||
SetCSSDataURLBased = 61,
|
||||
TechnicalInfo = 63,
|
||||
CustomIssue = 64,
|
||||
CSSInsertRuleURLBased = 67,
|
||||
MouseClick = 69,
|
||||
CreateIFrameDocument = 70,
|
||||
}
|
||||
|
||||
|
||||
export type BatchMetadata = [
|
||||
type: Type.BatchMetadata,
|
||||
version: number,
|
||||
pageNo: number,
|
||||
firstIndex: number,
|
||||
timestamp: number,
|
||||
location: string,
|
||||
]
|
||||
|
||||
export type PartitionedMessage = [
|
||||
type: Type.PartitionedMessage,
|
||||
partNo: number,
|
||||
partTotal: number,
|
||||
]
|
||||
|
||||
export type Timestamp = [
|
||||
type: Type.Timestamp,
|
||||
timestamp: number,
|
||||
]
|
||||
|
||||
export type SetPageLocation = [
|
||||
type: Type.SetPageLocation,
|
||||
url: string,
|
||||
referrer: string,
|
||||
navigationStart: number,
|
||||
]
|
||||
|
||||
export type SetViewportSize = [
|
||||
type: Type.SetViewportSize,
|
||||
width: number,
|
||||
height: number,
|
||||
]
|
||||
|
||||
export type SetViewportScroll = [
|
||||
type: Type.SetViewportScroll,
|
||||
x: number,
|
||||
y: number,
|
||||
]
|
||||
|
||||
export type CreateDocument = [
|
||||
type: Type.CreateDocument,
|
||||
|
||||
]
|
||||
|
||||
export type CreateElementNode = [
|
||||
type: Type.CreateElementNode,
|
||||
id: number,
|
||||
parentID: number,
|
||||
index: number,
|
||||
tag: string,
|
||||
svg: boolean,
|
||||
]
|
||||
|
||||
export type CreateTextNode = [
|
||||
type: Type.CreateTextNode,
|
||||
id: number,
|
||||
parentID: number,
|
||||
index: number,
|
||||
]
|
||||
|
||||
export type MoveNode = [
|
||||
type: Type.MoveNode,
|
||||
id: number,
|
||||
parentID: number,
|
||||
index: number,
|
||||
]
|
||||
|
||||
export type RemoveNode = [
|
||||
type: Type.RemoveNode,
|
||||
id: number,
|
||||
]
|
||||
|
||||
export type SetNodeAttribute = [
|
||||
type: Type.SetNodeAttribute,
|
||||
id: number,
|
||||
name: string,
|
||||
value: string,
|
||||
]
|
||||
|
||||
export type RemoveNodeAttribute = [
|
||||
type: Type.RemoveNodeAttribute,
|
||||
id: number,
|
||||
name: string,
|
||||
]
|
||||
|
||||
export type SetNodeData = [
|
||||
type: Type.SetNodeData,
|
||||
id: number,
|
||||
data: string,
|
||||
]
|
||||
|
||||
export type SetNodeScroll = [
|
||||
type: Type.SetNodeScroll,
|
||||
id: number,
|
||||
x: number,
|
||||
y: number,
|
||||
]
|
||||
|
||||
export type SetInputTarget = [
|
||||
type: Type.SetInputTarget,
|
||||
id: number,
|
||||
label: string,
|
||||
]
|
||||
|
||||
export type SetInputValue = [
|
||||
type: Type.SetInputValue,
|
||||
id: number,
|
||||
value: string,
|
||||
mask: number,
|
||||
]
|
||||
|
||||
export type SetInputChecked = [
|
||||
type: Type.SetInputChecked,
|
||||
id: number,
|
||||
checked: boolean,
|
||||
]
|
||||
|
||||
export type MouseMove = [
|
||||
type: Type.MouseMove,
|
||||
x: number,
|
||||
y: number,
|
||||
]
|
||||
|
||||
export type ConsoleLog = [
|
||||
type: Type.ConsoleLog,
|
||||
level: string,
|
||||
value: string,
|
||||
]
|
||||
|
||||
export type PageLoadTiming = [
|
||||
type: Type.PageLoadTiming,
|
||||
requestStart: number,
|
||||
responseStart: number,
|
||||
responseEnd: number,
|
||||
domContentLoadedEventStart: number,
|
||||
domContentLoadedEventEnd: number,
|
||||
loadEventStart: number,
|
||||
loadEventEnd: number,
|
||||
firstPaint: number,
|
||||
firstContentfulPaint: number,
|
||||
]
|
||||
|
||||
export type PageRenderTiming = [
|
||||
type: Type.PageRenderTiming,
|
||||
speedIndex: number,
|
||||
visuallyComplete: number,
|
||||
timeToInteractive: number,
|
||||
]
|
||||
|
||||
export type JSException = [
|
||||
type: Type.JSException,
|
||||
name: string,
|
||||
message: string,
|
||||
payload: string,
|
||||
]
|
||||
|
||||
export type RawCustomEvent = [
|
||||
type: Type.RawCustomEvent,
|
||||
name: string,
|
||||
payload: string,
|
||||
]
|
||||
|
||||
export type UserID = [
|
||||
type: Type.UserID,
|
||||
id: string,
|
||||
]
|
||||
|
||||
export type UserAnonymousID = [
|
||||
type: Type.UserAnonymousID,
|
||||
id: string,
|
||||
]
|
||||
|
||||
export type Metadata = [
|
||||
type: Type.Metadata,
|
||||
key: string,
|
||||
value: string,
|
||||
]
|
||||
|
||||
export type CSSInsertRule = [
|
||||
type: Type.CSSInsertRule,
|
||||
id: number,
|
||||
rule: string,
|
||||
index: number,
|
||||
]
|
||||
|
||||
export type CSSDeleteRule = [
|
||||
type: Type.CSSDeleteRule,
|
||||
id: number,
|
||||
index: number,
|
||||
]
|
||||
|
||||
export type Fetch = [
|
||||
type: Type.Fetch,
|
||||
method: string,
|
||||
url: string,
|
||||
request: string,
|
||||
response: string,
|
||||
status: number,
|
||||
timestamp: number,
|
||||
duration: number,
|
||||
]
|
||||
|
||||
export type Profiler = [
|
||||
type: Type.Profiler,
|
||||
name: string,
|
||||
duration: number,
|
||||
args: string,
|
||||
result: string,
|
||||
]
|
||||
|
||||
export type OTable = [
|
||||
type: Type.OTable,
|
||||
key: string,
|
||||
value: string,
|
||||
]
|
||||
|
||||
export type StateAction = [
|
||||
type: Type.StateAction,
|
||||
type: string,
|
||||
]
|
||||
|
||||
export type Redux = [
|
||||
type: Type.Redux,
|
||||
action: string,
|
||||
state: string,
|
||||
duration: number,
|
||||
]
|
||||
|
||||
export type Vuex = [
|
||||
type: Type.Vuex,
|
||||
mutation: string,
|
||||
state: string,
|
||||
]
|
||||
|
||||
export type MobX = [
|
||||
type: Type.MobX,
|
||||
type: string,
|
||||
payload: string,
|
||||
]
|
||||
|
||||
export type NgRx = [
|
||||
type: Type.NgRx,
|
||||
action: string,
|
||||
state: string,
|
||||
duration: number,
|
||||
]
|
||||
|
||||
export type GraphQL = [
|
||||
type: Type.GraphQL,
|
||||
operationKind: string,
|
||||
operationName: string,
|
||||
variables: string,
|
||||
response: string,
|
||||
]
|
||||
|
||||
export type PerformanceTrack = [
|
||||
type: Type.PerformanceTrack,
|
||||
frames: number,
|
||||
ticks: number,
|
||||
totalJSHeapSize: number,
|
||||
usedJSHeapSize: number,
|
||||
]
|
||||
|
||||
export type ResourceTiming = [
|
||||
type: Type.ResourceTiming,
|
||||
timestamp: number,
|
||||
duration: number,
|
||||
ttfb: number,
|
||||
headerSize: number,
|
||||
encodedBodySize: number,
|
||||
decodedBodySize: number,
|
||||
url: string,
|
||||
initiator: string,
|
||||
]
|
||||
|
||||
export type ConnectionInformation = [
|
||||
type: Type.ConnectionInformation,
|
||||
downlink: number,
|
||||
type: string,
|
||||
]
|
||||
|
||||
export type SetPageVisibility = [
|
||||
type: Type.SetPageVisibility,
|
||||
hidden: boolean,
|
||||
]
|
||||
|
||||
export type LongTask = [
|
||||
type: Type.LongTask,
|
||||
timestamp: number,
|
||||
duration: number,
|
||||
context: number,
|
||||
containerType: number,
|
||||
containerSrc: string,
|
||||
containerId: string,
|
||||
containerName: string,
|
||||
]
|
||||
|
||||
export type SetNodeAttributeURLBased = [
|
||||
type: Type.SetNodeAttributeURLBased,
|
||||
id: number,
|
||||
name: string,
|
||||
value: string,
|
||||
baseURL: string,
|
||||
]
|
||||
|
||||
export type SetCSSDataURLBased = [
|
||||
type: Type.SetCSSDataURLBased,
|
||||
id: number,
|
||||
data: string,
|
||||
baseURL: string,
|
||||
]
|
||||
|
||||
export type TechnicalInfo = [
|
||||
type: Type.TechnicalInfo,
|
||||
type: string,
|
||||
value: string,
|
||||
]
|
||||
|
||||
export type CustomIssue = [
|
||||
type: Type.CustomIssue,
|
||||
name: string,
|
||||
payload: string,
|
||||
]
|
||||
|
||||
export type CSSInsertRuleURLBased = [
|
||||
type: Type.CSSInsertRuleURLBased,
|
||||
id: number,
|
||||
rule: string,
|
||||
index: number,
|
||||
baseURL: string,
|
||||
]
|
||||
|
||||
export type MouseClick = [
|
||||
type: Type.MouseClick,
|
||||
id: number,
|
||||
hesitationTime: number,
|
||||
label: string,
|
||||
selector: string,
|
||||
]
|
||||
|
||||
export type CreateIFrameDocument = [
|
||||
type: Type.CreateIFrameDocument,
|
||||
frameID: number,
|
||||
id: number,
|
||||
]
|
||||
|
||||
|
||||
type Message = BatchMetadata | PartitionedMessage | Timestamp | SetPageLocation | SetViewportSize | SetViewportScroll | CreateDocument | CreateElementNode | CreateTextNode | MoveNode | RemoveNode | SetNodeAttribute | RemoveNodeAttribute | SetNodeData | SetNodeScroll | SetInputTarget | SetInputValue | SetInputChecked | MouseMove | ConsoleLog | PageLoadTiming | PageRenderTiming | JSException | RawCustomEvent | UserID | UserAnonymousID | Metadata | CSSInsertRule | CSSDeleteRule | Fetch | Profiler | OTable | StateAction | Redux | Vuex | MobX | NgRx | GraphQL | PerformanceTrack | ResourceTiming | ConnectionInformation | SetPageVisibility | LongTask | SetNodeAttributeURLBased | SetCSSDataURLBased | TechnicalInfo | CustomIssue | CSSInsertRuleURLBased | MouseClick | CreateIFrameDocument
|
||||
export default Message
|
||||
|
|
@ -1,903 +0,0 @@
|
|||
// Auto-generated, do not edit
|
||||
import type { Writer, Message }from "./types.js";
|
||||
export default Message
|
||||
|
||||
function bindNew<C extends { new(...args: A): T }, A extends any[], T>(
|
||||
Class: C & { new(...args: A): T }
|
||||
): C & ((...args: A) => T) {
|
||||
function _Class(...args: A) {
|
||||
return new Class(...args);
|
||||
}
|
||||
_Class.prototype = Class.prototype;
|
||||
return <C & ((...args: A) => T)>_Class;
|
||||
}
|
||||
|
||||
export const classes: Map<number, Function> = new Map();
|
||||
|
||||
|
||||
class _BatchMeta implements Message {
|
||||
readonly _id: number = 80;
|
||||
constructor(
|
||||
public pageNo: number,
|
||||
public firstIndex: number,
|
||||
public timestamp: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(80) &&
|
||||
writer.uint(this.pageNo) &&
|
||||
writer.uint(this.firstIndex) &&
|
||||
writer.int(this.timestamp);
|
||||
}
|
||||
}
|
||||
export const BatchMeta = bindNew(_BatchMeta);
|
||||
classes.set(80, BatchMeta);
|
||||
|
||||
|
||||
class _Timestamp implements Message {
|
||||
readonly _id: number = 0;
|
||||
constructor(
|
||||
public timestamp: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(0) &&
|
||||
writer.uint(this.timestamp);
|
||||
}
|
||||
}
|
||||
export const Timestamp = bindNew(_Timestamp);
|
||||
classes.set(0, Timestamp);
|
||||
|
||||
|
||||
class _SetPageLocation implements Message {
|
||||
readonly _id: number = 4;
|
||||
constructor(
|
||||
public url: string,
|
||||
public referrer: string,
|
||||
public navigationStart: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(4) &&
|
||||
writer.string(this.url) &&
|
||||
writer.string(this.referrer) &&
|
||||
writer.uint(this.navigationStart);
|
||||
}
|
||||
}
|
||||
export const SetPageLocation = bindNew(_SetPageLocation);
|
||||
classes.set(4, SetPageLocation);
|
||||
|
||||
|
||||
class _SetViewportSize implements Message {
|
||||
readonly _id: number = 5;
|
||||
constructor(
|
||||
public width: number,
|
||||
public height: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(5) &&
|
||||
writer.uint(this.width) &&
|
||||
writer.uint(this.height);
|
||||
}
|
||||
}
|
||||
export const SetViewportSize = bindNew(_SetViewportSize);
|
||||
classes.set(5, SetViewportSize);
|
||||
|
||||
|
||||
class _SetViewportScroll implements Message {
|
||||
readonly _id: number = 6;
|
||||
constructor(
|
||||
public x: number,
|
||||
public y: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(6) &&
|
||||
writer.int(this.x) &&
|
||||
writer.int(this.y);
|
||||
}
|
||||
}
|
||||
export const SetViewportScroll = bindNew(_SetViewportScroll);
|
||||
classes.set(6, SetViewportScroll);
|
||||
|
||||
|
||||
class _CreateDocument implements Message {
|
||||
readonly _id: number = 7;
|
||||
constructor(
|
||||
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(7)
|
||||
;
|
||||
}
|
||||
}
|
||||
export const CreateDocument = bindNew(_CreateDocument);
|
||||
classes.set(7, CreateDocument);
|
||||
|
||||
|
||||
class _CreateElementNode implements Message {
|
||||
readonly _id: number = 8;
|
||||
constructor(
|
||||
public id: number,
|
||||
public parentID: number,
|
||||
public index: number,
|
||||
public tag: string,
|
||||
public svg: boolean
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(8) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.uint(this.parentID) &&
|
||||
writer.uint(this.index) &&
|
||||
writer.string(this.tag) &&
|
||||
writer.boolean(this.svg);
|
||||
}
|
||||
}
|
||||
export const CreateElementNode = bindNew(_CreateElementNode);
|
||||
classes.set(8, CreateElementNode);
|
||||
|
||||
|
||||
class _CreateTextNode implements Message {
|
||||
readonly _id: number = 9;
|
||||
constructor(
|
||||
public id: number,
|
||||
public parentID: number,
|
||||
public index: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(9) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.uint(this.parentID) &&
|
||||
writer.uint(this.index);
|
||||
}
|
||||
}
|
||||
export const CreateTextNode = bindNew(_CreateTextNode);
|
||||
classes.set(9, CreateTextNode);
|
||||
|
||||
|
||||
class _MoveNode implements Message {
|
||||
readonly _id: number = 10;
|
||||
constructor(
|
||||
public id: number,
|
||||
public parentID: number,
|
||||
public index: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(10) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.uint(this.parentID) &&
|
||||
writer.uint(this.index);
|
||||
}
|
||||
}
|
||||
export const MoveNode = bindNew(_MoveNode);
|
||||
classes.set(10, MoveNode);
|
||||
|
||||
|
||||
class _RemoveNode implements Message {
|
||||
readonly _id: number = 11;
|
||||
constructor(
|
||||
public id: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(11) &&
|
||||
writer.uint(this.id);
|
||||
}
|
||||
}
|
||||
export const RemoveNode = bindNew(_RemoveNode);
|
||||
classes.set(11, RemoveNode);
|
||||
|
||||
|
||||
class _SetNodeAttribute implements Message {
|
||||
readonly _id: number = 12;
|
||||
constructor(
|
||||
public id: number,
|
||||
public name: string,
|
||||
public value: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(12) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.string(this.name) &&
|
||||
writer.string(this.value);
|
||||
}
|
||||
}
|
||||
export const SetNodeAttribute = bindNew(_SetNodeAttribute);
|
||||
classes.set(12, SetNodeAttribute);
|
||||
|
||||
|
||||
class _RemoveNodeAttribute implements Message {
|
||||
readonly _id: number = 13;
|
||||
constructor(
|
||||
public id: number,
|
||||
public name: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(13) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.string(this.name);
|
||||
}
|
||||
}
|
||||
export const RemoveNodeAttribute = bindNew(_RemoveNodeAttribute);
|
||||
classes.set(13, RemoveNodeAttribute);
|
||||
|
||||
|
||||
class _SetNodeData implements Message {
|
||||
readonly _id: number = 14;
|
||||
constructor(
|
||||
public id: number,
|
||||
public data: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(14) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.string(this.data);
|
||||
}
|
||||
}
|
||||
export const SetNodeData = bindNew(_SetNodeData);
|
||||
classes.set(14, SetNodeData);
|
||||
|
||||
|
||||
class _SetNodeScroll implements Message {
|
||||
readonly _id: number = 16;
|
||||
constructor(
|
||||
public id: number,
|
||||
public x: number,
|
||||
public y: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(16) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.int(this.x) &&
|
||||
writer.int(this.y);
|
||||
}
|
||||
}
|
||||
export const SetNodeScroll = bindNew(_SetNodeScroll);
|
||||
classes.set(16, SetNodeScroll);
|
||||
|
||||
|
||||
class _SetInputTarget implements Message {
|
||||
readonly _id: number = 17;
|
||||
constructor(
|
||||
public id: number,
|
||||
public label: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(17) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.string(this.label);
|
||||
}
|
||||
}
|
||||
export const SetInputTarget = bindNew(_SetInputTarget);
|
||||
classes.set(17, SetInputTarget);
|
||||
|
||||
|
||||
class _SetInputValue implements Message {
|
||||
readonly _id: number = 18;
|
||||
constructor(
|
||||
public id: number,
|
||||
public value: string,
|
||||
public mask: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(18) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.string(this.value) &&
|
||||
writer.int(this.mask);
|
||||
}
|
||||
}
|
||||
export const SetInputValue = bindNew(_SetInputValue);
|
||||
classes.set(18, SetInputValue);
|
||||
|
||||
|
||||
class _SetInputChecked implements Message {
|
||||
readonly _id: number = 19;
|
||||
constructor(
|
||||
public id: number,
|
||||
public checked: boolean
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(19) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.boolean(this.checked);
|
||||
}
|
||||
}
|
||||
export const SetInputChecked = bindNew(_SetInputChecked);
|
||||
classes.set(19, SetInputChecked);
|
||||
|
||||
|
||||
class _MouseMove implements Message {
|
||||
readonly _id: number = 20;
|
||||
constructor(
|
||||
public x: number,
|
||||
public y: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(20) &&
|
||||
writer.uint(this.x) &&
|
||||
writer.uint(this.y);
|
||||
}
|
||||
}
|
||||
export const MouseMove = bindNew(_MouseMove);
|
||||
classes.set(20, MouseMove);
|
||||
|
||||
|
||||
class _ConsoleLog implements Message {
|
||||
readonly _id: number = 22;
|
||||
constructor(
|
||||
public level: string,
|
||||
public value: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(22) &&
|
||||
writer.string(this.level) &&
|
||||
writer.string(this.value);
|
||||
}
|
||||
}
|
||||
export const ConsoleLog = bindNew(_ConsoleLog);
|
||||
classes.set(22, ConsoleLog);
|
||||
|
||||
|
||||
class _PageLoadTiming implements Message {
|
||||
readonly _id: number = 23;
|
||||
constructor(
|
||||
public requestStart: number,
|
||||
public responseStart: number,
|
||||
public responseEnd: number,
|
||||
public domContentLoadedEventStart: number,
|
||||
public domContentLoadedEventEnd: number,
|
||||
public loadEventStart: number,
|
||||
public loadEventEnd: number,
|
||||
public firstPaint: number,
|
||||
public firstContentfulPaint: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(23) &&
|
||||
writer.uint(this.requestStart) &&
|
||||
writer.uint(this.responseStart) &&
|
||||
writer.uint(this.responseEnd) &&
|
||||
writer.uint(this.domContentLoadedEventStart) &&
|
||||
writer.uint(this.domContentLoadedEventEnd) &&
|
||||
writer.uint(this.loadEventStart) &&
|
||||
writer.uint(this.loadEventEnd) &&
|
||||
writer.uint(this.firstPaint) &&
|
||||
writer.uint(this.firstContentfulPaint);
|
||||
}
|
||||
}
|
||||
export const PageLoadTiming = bindNew(_PageLoadTiming);
|
||||
classes.set(23, PageLoadTiming);
|
||||
|
||||
|
||||
class _PageRenderTiming implements Message {
|
||||
readonly _id: number = 24;
|
||||
constructor(
|
||||
public speedIndex: number,
|
||||
public visuallyComplete: number,
|
||||
public timeToInteractive: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(24) &&
|
||||
writer.uint(this.speedIndex) &&
|
||||
writer.uint(this.visuallyComplete) &&
|
||||
writer.uint(this.timeToInteractive);
|
||||
}
|
||||
}
|
||||
export const PageRenderTiming = bindNew(_PageRenderTiming);
|
||||
classes.set(24, PageRenderTiming);
|
||||
|
||||
|
||||
class _JSException implements Message {
|
||||
readonly _id: number = 25;
|
||||
constructor(
|
||||
public name: string,
|
||||
public message: string,
|
||||
public payload: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(25) &&
|
||||
writer.string(this.name) &&
|
||||
writer.string(this.message) &&
|
||||
writer.string(this.payload);
|
||||
}
|
||||
}
|
||||
export const JSException = bindNew(_JSException);
|
||||
classes.set(25, JSException);
|
||||
|
||||
|
||||
class _RawCustomEvent implements Message {
|
||||
readonly _id: number = 27;
|
||||
constructor(
|
||||
public name: string,
|
||||
public payload: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(27) &&
|
||||
writer.string(this.name) &&
|
||||
writer.string(this.payload);
|
||||
}
|
||||
}
|
||||
export const RawCustomEvent = bindNew(_RawCustomEvent);
|
||||
classes.set(27, RawCustomEvent);
|
||||
|
||||
|
||||
class _UserID implements Message {
|
||||
readonly _id: number = 28;
|
||||
constructor(
|
||||
public id: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(28) &&
|
||||
writer.string(this.id);
|
||||
}
|
||||
}
|
||||
export const UserID = bindNew(_UserID);
|
||||
classes.set(28, UserID);
|
||||
|
||||
|
||||
class _UserAnonymousID implements Message {
|
||||
readonly _id: number = 29;
|
||||
constructor(
|
||||
public id: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(29) &&
|
||||
writer.string(this.id);
|
||||
}
|
||||
}
|
||||
export const UserAnonymousID = bindNew(_UserAnonymousID);
|
||||
classes.set(29, UserAnonymousID);
|
||||
|
||||
|
||||
class _Metadata implements Message {
|
||||
readonly _id: number = 30;
|
||||
constructor(
|
||||
public key: string,
|
||||
public value: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(30) &&
|
||||
writer.string(this.key) &&
|
||||
writer.string(this.value);
|
||||
}
|
||||
}
|
||||
export const Metadata = bindNew(_Metadata);
|
||||
classes.set(30, Metadata);
|
||||
|
||||
|
||||
class _CSSInsertRule implements Message {
|
||||
readonly _id: number = 37;
|
||||
constructor(
|
||||
public id: number,
|
||||
public rule: string,
|
||||
public index: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(37) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.string(this.rule) &&
|
||||
writer.uint(this.index);
|
||||
}
|
||||
}
|
||||
export const CSSInsertRule = bindNew(_CSSInsertRule);
|
||||
classes.set(37, CSSInsertRule);
|
||||
|
||||
|
||||
class _CSSDeleteRule implements Message {
|
||||
readonly _id: number = 38;
|
||||
constructor(
|
||||
public id: number,
|
||||
public index: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(38) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.uint(this.index);
|
||||
}
|
||||
}
|
||||
export const CSSDeleteRule = bindNew(_CSSDeleteRule);
|
||||
classes.set(38, CSSDeleteRule);
|
||||
|
||||
|
||||
class _Fetch implements Message {
|
||||
readonly _id: number = 39;
|
||||
constructor(
|
||||
public method: string,
|
||||
public url: string,
|
||||
public request: string,
|
||||
public response: string,
|
||||
public status: number,
|
||||
public timestamp: number,
|
||||
public duration: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(39) &&
|
||||
writer.string(this.method) &&
|
||||
writer.string(this.url) &&
|
||||
writer.string(this.request) &&
|
||||
writer.string(this.response) &&
|
||||
writer.uint(this.status) &&
|
||||
writer.uint(this.timestamp) &&
|
||||
writer.uint(this.duration);
|
||||
}
|
||||
}
|
||||
export const Fetch = bindNew(_Fetch);
|
||||
classes.set(39, Fetch);
|
||||
|
||||
|
||||
class _Profiler implements Message {
|
||||
readonly _id: number = 40;
|
||||
constructor(
|
||||
public name: string,
|
||||
public duration: number,
|
||||
public args: string,
|
||||
public result: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(40) &&
|
||||
writer.string(this.name) &&
|
||||
writer.uint(this.duration) &&
|
||||
writer.string(this.args) &&
|
||||
writer.string(this.result);
|
||||
}
|
||||
}
|
||||
export const Profiler = bindNew(_Profiler);
|
||||
classes.set(40, Profiler);
|
||||
|
||||
|
||||
class _OTable implements Message {
|
||||
readonly _id: number = 41;
|
||||
constructor(
|
||||
public key: string,
|
||||
public value: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(41) &&
|
||||
writer.string(this.key) &&
|
||||
writer.string(this.value);
|
||||
}
|
||||
}
|
||||
export const OTable = bindNew(_OTable);
|
||||
classes.set(41, OTable);
|
||||
|
||||
|
||||
class _StateAction implements Message {
|
||||
readonly _id: number = 42;
|
||||
constructor(
|
||||
public type: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(42) &&
|
||||
writer.string(this.type);
|
||||
}
|
||||
}
|
||||
export const StateAction = bindNew(_StateAction);
|
||||
classes.set(42, StateAction);
|
||||
|
||||
|
||||
class _Redux implements Message {
|
||||
readonly _id: number = 44;
|
||||
constructor(
|
||||
public action: string,
|
||||
public state: string,
|
||||
public duration: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(44) &&
|
||||
writer.string(this.action) &&
|
||||
writer.string(this.state) &&
|
||||
writer.uint(this.duration);
|
||||
}
|
||||
}
|
||||
export const Redux = bindNew(_Redux);
|
||||
classes.set(44, Redux);
|
||||
|
||||
|
||||
class _Vuex implements Message {
|
||||
readonly _id: number = 45;
|
||||
constructor(
|
||||
public mutation: string,
|
||||
public state: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(45) &&
|
||||
writer.string(this.mutation) &&
|
||||
writer.string(this.state);
|
||||
}
|
||||
}
|
||||
export const Vuex = bindNew(_Vuex);
|
||||
classes.set(45, Vuex);
|
||||
|
||||
|
||||
class _MobX implements Message {
|
||||
readonly _id: number = 46;
|
||||
constructor(
|
||||
public type: string,
|
||||
public payload: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(46) &&
|
||||
writer.string(this.type) &&
|
||||
writer.string(this.payload);
|
||||
}
|
||||
}
|
||||
export const MobX = bindNew(_MobX);
|
||||
classes.set(46, MobX);
|
||||
|
||||
|
||||
class _NgRx implements Message {
|
||||
readonly _id: number = 47;
|
||||
constructor(
|
||||
public action: string,
|
||||
public state: string,
|
||||
public duration: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(47) &&
|
||||
writer.string(this.action) &&
|
||||
writer.string(this.state) &&
|
||||
writer.uint(this.duration);
|
||||
}
|
||||
}
|
||||
export const NgRx = bindNew(_NgRx);
|
||||
classes.set(47, NgRx);
|
||||
|
||||
|
||||
class _GraphQL implements Message {
|
||||
readonly _id: number = 48;
|
||||
constructor(
|
||||
public operationKind: string,
|
||||
public operationName: string,
|
||||
public variables: string,
|
||||
public response: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(48) &&
|
||||
writer.string(this.operationKind) &&
|
||||
writer.string(this.operationName) &&
|
||||
writer.string(this.variables) &&
|
||||
writer.string(this.response);
|
||||
}
|
||||
}
|
||||
export const GraphQL = bindNew(_GraphQL);
|
||||
classes.set(48, GraphQL);
|
||||
|
||||
|
||||
class _PerformanceTrack implements Message {
|
||||
readonly _id: number = 49;
|
||||
constructor(
|
||||
public frames: number,
|
||||
public ticks: number,
|
||||
public totalJSHeapSize: number,
|
||||
public usedJSHeapSize: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(49) &&
|
||||
writer.int(this.frames) &&
|
||||
writer.int(this.ticks) &&
|
||||
writer.uint(this.totalJSHeapSize) &&
|
||||
writer.uint(this.usedJSHeapSize);
|
||||
}
|
||||
}
|
||||
export const PerformanceTrack = bindNew(_PerformanceTrack);
|
||||
classes.set(49, PerformanceTrack);
|
||||
|
||||
|
||||
class _ResourceTiming implements Message {
|
||||
readonly _id: number = 53;
|
||||
constructor(
|
||||
public timestamp: number,
|
||||
public duration: number,
|
||||
public ttfb: number,
|
||||
public headerSize: number,
|
||||
public encodedBodySize: number,
|
||||
public decodedBodySize: number,
|
||||
public url: string,
|
||||
public initiator: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(53) &&
|
||||
writer.uint(this.timestamp) &&
|
||||
writer.uint(this.duration) &&
|
||||
writer.uint(this.ttfb) &&
|
||||
writer.uint(this.headerSize) &&
|
||||
writer.uint(this.encodedBodySize) &&
|
||||
writer.uint(this.decodedBodySize) &&
|
||||
writer.string(this.url) &&
|
||||
writer.string(this.initiator);
|
||||
}
|
||||
}
|
||||
export const ResourceTiming = bindNew(_ResourceTiming);
|
||||
classes.set(53, ResourceTiming);
|
||||
|
||||
|
||||
class _ConnectionInformation implements Message {
|
||||
readonly _id: number = 54;
|
||||
constructor(
|
||||
public downlink: number,
|
||||
public type: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(54) &&
|
||||
writer.uint(this.downlink) &&
|
||||
writer.string(this.type);
|
||||
}
|
||||
}
|
||||
export const ConnectionInformation = bindNew(_ConnectionInformation);
|
||||
classes.set(54, ConnectionInformation);
|
||||
|
||||
|
||||
class _SetPageVisibility implements Message {
|
||||
readonly _id: number = 55;
|
||||
constructor(
|
||||
public hidden: boolean
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(55) &&
|
||||
writer.boolean(this.hidden);
|
||||
}
|
||||
}
|
||||
export const SetPageVisibility = bindNew(_SetPageVisibility);
|
||||
classes.set(55, SetPageVisibility);
|
||||
|
||||
|
||||
class _LongTask implements Message {
|
||||
readonly _id: number = 59;
|
||||
constructor(
|
||||
public timestamp: number,
|
||||
public duration: number,
|
||||
public context: number,
|
||||
public containerType: number,
|
||||
public containerSrc: string,
|
||||
public containerId: string,
|
||||
public containerName: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(59) &&
|
||||
writer.uint(this.timestamp) &&
|
||||
writer.uint(this.duration) &&
|
||||
writer.uint(this.context) &&
|
||||
writer.uint(this.containerType) &&
|
||||
writer.string(this.containerSrc) &&
|
||||
writer.string(this.containerId) &&
|
||||
writer.string(this.containerName);
|
||||
}
|
||||
}
|
||||
export const LongTask = bindNew(_LongTask);
|
||||
classes.set(59, LongTask);
|
||||
|
||||
|
||||
class _SetNodeAttributeURLBased implements Message {
|
||||
readonly _id: number = 60;
|
||||
constructor(
|
||||
public id: number,
|
||||
public name: string,
|
||||
public value: string,
|
||||
public baseURL: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(60) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.string(this.name) &&
|
||||
writer.string(this.value) &&
|
||||
writer.string(this.baseURL);
|
||||
}
|
||||
}
|
||||
export const SetNodeAttributeURLBased = bindNew(_SetNodeAttributeURLBased);
|
||||
classes.set(60, SetNodeAttributeURLBased);
|
||||
|
||||
|
||||
class _SetCSSDataURLBased implements Message {
|
||||
readonly _id: number = 61;
|
||||
constructor(
|
||||
public id: number,
|
||||
public data: string,
|
||||
public baseURL: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(61) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.string(this.data) &&
|
||||
writer.string(this.baseURL);
|
||||
}
|
||||
}
|
||||
export const SetCSSDataURLBased = bindNew(_SetCSSDataURLBased);
|
||||
classes.set(61, SetCSSDataURLBased);
|
||||
|
||||
|
||||
class _TechnicalInfo implements Message {
|
||||
readonly _id: number = 63;
|
||||
constructor(
|
||||
public type: string,
|
||||
public value: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(63) &&
|
||||
writer.string(this.type) &&
|
||||
writer.string(this.value);
|
||||
}
|
||||
}
|
||||
export const TechnicalInfo = bindNew(_TechnicalInfo);
|
||||
classes.set(63, TechnicalInfo);
|
||||
|
||||
|
||||
class _CustomIssue implements Message {
|
||||
readonly _id: number = 64;
|
||||
constructor(
|
||||
public name: string,
|
||||
public payload: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(64) &&
|
||||
writer.string(this.name) &&
|
||||
writer.string(this.payload);
|
||||
}
|
||||
}
|
||||
export const CustomIssue = bindNew(_CustomIssue);
|
||||
classes.set(64, CustomIssue);
|
||||
|
||||
|
||||
class _PageClose implements Message {
|
||||
readonly _id: number = 65;
|
||||
constructor(
|
||||
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(65)
|
||||
;
|
||||
}
|
||||
}
|
||||
export const PageClose = bindNew(_PageClose);
|
||||
classes.set(65, PageClose);
|
||||
|
||||
|
||||
class _CSSInsertRuleURLBased implements Message {
|
||||
readonly _id: number = 67;
|
||||
constructor(
|
||||
public id: number,
|
||||
public rule: string,
|
||||
public index: number,
|
||||
public baseURL: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(67) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.string(this.rule) &&
|
||||
writer.uint(this.index) &&
|
||||
writer.string(this.baseURL);
|
||||
}
|
||||
}
|
||||
export const CSSInsertRuleURLBased = bindNew(_CSSInsertRuleURLBased);
|
||||
classes.set(67, CSSInsertRuleURLBased);
|
||||
|
||||
|
||||
class _MouseClick implements Message {
|
||||
readonly _id: number = 69;
|
||||
constructor(
|
||||
public id: number,
|
||||
public hesitationTime: number,
|
||||
public label: string,
|
||||
public selector: string
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(69) &&
|
||||
writer.uint(this.id) &&
|
||||
writer.uint(this.hesitationTime) &&
|
||||
writer.string(this.label) &&
|
||||
writer.string(this.selector);
|
||||
}
|
||||
}
|
||||
export const MouseClick = bindNew(_MouseClick);
|
||||
classes.set(69, MouseClick);
|
||||
|
||||
|
||||
class _CreateIFrameDocument implements Message {
|
||||
readonly _id: number = 70;
|
||||
constructor(
|
||||
public frameID: number,
|
||||
public id: number
|
||||
) {}
|
||||
encode(writer: Writer): boolean {
|
||||
return writer.uint(70) &&
|
||||
writer.uint(this.frameID) &&
|
||||
writer.uint(this.id);
|
||||
}
|
||||
}
|
||||
export const CreateIFrameDocument = bindNew(_CreateIFrameDocument);
|
||||
classes.set(70, CreateIFrameDocument);
|
||||
|
||||
|
||||
|
|
@ -1,10 +0,0 @@
|
|||
export interface Writer {
|
||||
uint(n: number): boolean
|
||||
int(n: number): boolean
|
||||
string(s: string): boolean
|
||||
boolean(b: boolean): boolean
|
||||
}
|
||||
|
||||
export interface Message {
|
||||
encode(w: Writer): boolean;
|
||||
}
|
||||
|
|
@ -1,19 +0,0 @@
|
|||
export interface Options {
|
||||
connAttemptCount?: number;
|
||||
connAttemptGap?: number;
|
||||
}
|
||||
|
||||
type Start = {
|
||||
type: 'start';
|
||||
ingestPoint: string;
|
||||
pageNo: number;
|
||||
timestamp: number;
|
||||
} & Options;
|
||||
|
||||
type Auth = {
|
||||
type: 'auth';
|
||||
token: string;
|
||||
beaconSizeLimit?: number;
|
||||
};
|
||||
|
||||
export type WorkerMessageData = null | 'stop' | Start | Auth | Array<{ _id: number }>;
|
||||
|
|
@ -1,34 +1,34 @@
|
|||
export function isSVGElement(node: Element): node is SVGElement {
|
||||
return node.namespaceURI === 'http://www.w3.org/2000/svg';
|
||||
return node.namespaceURI === 'http://www.w3.org/2000/svg'
|
||||
}
|
||||
|
||||
export function isElementNode(node: Node): node is Element {
|
||||
return node.nodeType === Node.ELEMENT_NODE;
|
||||
return node.nodeType === Node.ELEMENT_NODE
|
||||
}
|
||||
|
||||
export function isTextNode(node: Node): node is Text {
|
||||
return node.nodeType === Node.TEXT_NODE;
|
||||
return node.nodeType === Node.TEXT_NODE
|
||||
}
|
||||
|
||||
export function isRootNode(node: Node): boolean {
|
||||
return node.nodeType === Node.DOCUMENT_NODE || node.nodeType === Node.DOCUMENT_FRAGMENT_NODE;
|
||||
return node.nodeType === Node.DOCUMENT_NODE || node.nodeType === Node.DOCUMENT_FRAGMENT_NODE
|
||||
}
|
||||
|
||||
type TagTypeMap = {
|
||||
HTML: HTMLHtmlElement;
|
||||
IMG: HTMLImageElement;
|
||||
INPUT: HTMLInputElement;
|
||||
TEXTAREA: HTMLTextAreaElement;
|
||||
SELECT: HTMLSelectElement;
|
||||
LABEL: HTMLLabelElement;
|
||||
IFRAME: HTMLIFrameElement;
|
||||
STYLE: HTMLStyleElement;
|
||||
style: SVGStyleElement;
|
||||
LINK: HTMLLinkElement;
|
||||
};
|
||||
HTML: HTMLHtmlElement
|
||||
IMG: HTMLImageElement
|
||||
INPUT: HTMLInputElement
|
||||
TEXTAREA: HTMLTextAreaElement
|
||||
SELECT: HTMLSelectElement
|
||||
LABEL: HTMLLabelElement
|
||||
IFRAME: HTMLIFrameElement
|
||||
STYLE: HTMLStyleElement
|
||||
style: SVGStyleElement
|
||||
LINK: HTMLLinkElement
|
||||
}
|
||||
export function hasTag<T extends keyof TagTypeMap>(
|
||||
el: Node,
|
||||
tagName: T,
|
||||
): el is TagTypeMap[typeof tagName] {
|
||||
return el.nodeName === tagName;
|
||||
return el.nodeName === tagName
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,45 +1,45 @@
|
|||
import type Message from '../../common/messages.js';
|
||||
import { Timestamp, Metadata, UserID } from '../../common/messages.js';
|
||||
import { timestamp, deprecationWarn } from '../utils.js';
|
||||
import Nodes from './nodes.js';
|
||||
import Observer from './observer/top_observer.js';
|
||||
import Sanitizer from './sanitizer.js';
|
||||
import Ticker from './ticker.js';
|
||||
import Logger, { LogLevel } from './logger.js';
|
||||
import Session from './session.js';
|
||||
import type Message from './messages.gen.js'
|
||||
import { Timestamp, Metadata, UserID } from './messages.gen.js'
|
||||
import { timestamp, deprecationWarn } from '../utils.js'
|
||||
import Nodes from './nodes.js'
|
||||
import Observer from './observer/top_observer.js'
|
||||
import Sanitizer from './sanitizer.js'
|
||||
import Ticker from './ticker.js'
|
||||
import Logger, { LogLevel } from './logger.js'
|
||||
import Session from './session.js'
|
||||
|
||||
import { deviceMemory, jsHeapSizeLimit } from '../modules/performance.js';
|
||||
import { deviceMemory, jsHeapSizeLimit } from '../modules/performance.js'
|
||||
|
||||
import type { Options as ObserverOptions } from './observer/top_observer.js';
|
||||
import type { Options as SanitizerOptions } from './sanitizer.js';
|
||||
import type { Options as LoggerOptions } from './logger.js';
|
||||
import type { Options as WebworkerOptions, WorkerMessageData } from '../../common/webworker.js';
|
||||
import type { Options as ObserverOptions } from './observer/top_observer.js'
|
||||
import type { Options as SanitizerOptions } from './sanitizer.js'
|
||||
import type { Options as LoggerOptions } from './logger.js'
|
||||
import type { Options as WebworkerOptions, WorkerMessageData } from '../../common/interaction.js'
|
||||
|
||||
// TODO: Unify and clearly describe options logic
|
||||
export interface StartOptions {
|
||||
userID?: string;
|
||||
metadata?: Record<string, string>;
|
||||
forceNew?: boolean;
|
||||
userID?: string
|
||||
metadata?: Record<string, string>
|
||||
forceNew?: boolean
|
||||
}
|
||||
|
||||
interface OnStartInfo {
|
||||
sessionID: string;
|
||||
sessionToken: string;
|
||||
userUUID: string;
|
||||
sessionID: string
|
||||
sessionToken: string
|
||||
userUUID: string
|
||||
}
|
||||
const CANCELED = 'canceled' as const;
|
||||
const START_ERROR = ':(' as const;
|
||||
type SuccessfulStart = OnStartInfo & { success: true };
|
||||
const CANCELED = 'canceled' as const
|
||||
const START_ERROR = ':(' as const
|
||||
type SuccessfulStart = OnStartInfo & { success: true }
|
||||
type UnsuccessfulStart = {
|
||||
reason: typeof CANCELED | string;
|
||||
success: false;
|
||||
};
|
||||
const UnsuccessfulStart = (reason: string): UnsuccessfulStart => ({ reason, success: false });
|
||||
const SuccessfulStart = (body: OnStartInfo): SuccessfulStart => ({ ...body, success: true });
|
||||
export type StartPromiseReturn = SuccessfulStart | UnsuccessfulStart;
|
||||
reason: typeof CANCELED | string
|
||||
success: false
|
||||
}
|
||||
const UnsuccessfulStart = (reason: string): UnsuccessfulStart => ({ reason, success: false })
|
||||
const SuccessfulStart = (body: OnStartInfo): SuccessfulStart => ({ ...body, success: true })
|
||||
export type StartPromiseReturn = SuccessfulStart | UnsuccessfulStart
|
||||
|
||||
type StartCallback = (i: OnStartInfo) => void;
|
||||
type CommitCallback = (messages: Array<Message>) => void;
|
||||
type StartCallback = (i: OnStartInfo) => void
|
||||
type CommitCallback = (messages: Array<Message>) => void
|
||||
enum ActivityState {
|
||||
NotActive,
|
||||
Starting,
|
||||
|
|
@ -47,51 +47,51 @@ enum ActivityState {
|
|||
}
|
||||
|
||||
type AppOptions = {
|
||||
revID: string;
|
||||
node_id: string;
|
||||
session_token_key: string;
|
||||
session_pageno_key: string;
|
||||
session_reset_key: string;
|
||||
local_uuid_key: string;
|
||||
ingestPoint: string;
|
||||
resourceBaseHref: string | null; // resourceHref?
|
||||
revID: string
|
||||
node_id: string
|
||||
session_token_key: string
|
||||
session_pageno_key: string
|
||||
session_reset_key: string
|
||||
local_uuid_key: string
|
||||
ingestPoint: string
|
||||
resourceBaseHref: string | null // resourceHref?
|
||||
//resourceURLRewriter: (url: string) => string | boolean,
|
||||
verbose: boolean;
|
||||
__is_snippet: boolean;
|
||||
__debug_report_edp: string | null;
|
||||
__debug__?: LoggerOptions;
|
||||
localStorage: Storage | null;
|
||||
sessionStorage: Storage | null;
|
||||
verbose: boolean
|
||||
__is_snippet: boolean
|
||||
__debug_report_edp: string | null
|
||||
__debug__?: LoggerOptions
|
||||
localStorage: Storage
|
||||
sessionStorage: Storage
|
||||
|
||||
// @deprecated
|
||||
onStart?: StartCallback;
|
||||
} & WebworkerOptions;
|
||||
onStart?: StartCallback
|
||||
} & WebworkerOptions
|
||||
|
||||
export type Options = AppOptions & ObserverOptions & SanitizerOptions;
|
||||
export type Options = AppOptions & ObserverOptions & SanitizerOptions
|
||||
|
||||
// TODO: use backendHost only
|
||||
export const DEFAULT_INGEST_POINT = 'https://api.openreplay.com/ingest';
|
||||
export const DEFAULT_INGEST_POINT = 'https://api.openreplay.com/ingest'
|
||||
|
||||
export default class App {
|
||||
readonly nodes: Nodes;
|
||||
readonly ticker: Ticker;
|
||||
readonly projectKey: string;
|
||||
readonly sanitizer: Sanitizer;
|
||||
readonly debug: Logger;
|
||||
readonly notify: Logger;
|
||||
readonly session: Session;
|
||||
readonly localStorage: Storage;
|
||||
readonly sessionStorage: Storage;
|
||||
private readonly messages: Array<Message> = [];
|
||||
private readonly observer: Observer;
|
||||
private readonly startCallbacks: Array<StartCallback> = [];
|
||||
private readonly stopCallbacks: Array<() => any> = [];
|
||||
private readonly commitCallbacks: Array<CommitCallback> = [];
|
||||
private readonly options: AppOptions;
|
||||
private readonly revID: string;
|
||||
private activityState: ActivityState = ActivityState.NotActive;
|
||||
private readonly version = 'TRACKER_VERSION'; // TODO: version compatability check inside each plugin.
|
||||
private readonly worker?: Worker;
|
||||
readonly nodes: Nodes
|
||||
readonly ticker: Ticker
|
||||
readonly projectKey: string
|
||||
readonly sanitizer: Sanitizer
|
||||
readonly debug: Logger
|
||||
readonly notify: Logger
|
||||
readonly session: Session
|
||||
readonly localStorage: Storage
|
||||
readonly sessionStorage: Storage
|
||||
private readonly messages: Array<Message> = []
|
||||
private readonly observer: Observer
|
||||
private readonly startCallbacks: Array<StartCallback> = []
|
||||
private readonly stopCallbacks: Array<() => any> = []
|
||||
private readonly commitCallbacks: Array<CommitCallback> = []
|
||||
private readonly options: AppOptions
|
||||
private readonly revID: string
|
||||
private activityState: ActivityState = ActivityState.NotActive
|
||||
private readonly version = 'TRACKER_VERSION' // TODO: version compatability check inside each plugin.
|
||||
private readonly worker?: Worker
|
||||
constructor(
|
||||
projectKey: string,
|
||||
sessionToken: string | null | undefined,
|
||||
|
|
@ -101,7 +101,7 @@ export default class App {
|
|||
// deprecationWarn("'onStart' option", "tracker.start().then(/* handle session info */)")
|
||||
// } ?? maybe onStart is good
|
||||
|
||||
this.projectKey = projectKey;
|
||||
this.projectKey = projectKey
|
||||
this.options = Object.assign(
|
||||
{
|
||||
revID: '',
|
||||
|
|
@ -119,63 +119,61 @@ export default class App {
|
|||
sessionStorage: null,
|
||||
},
|
||||
options,
|
||||
);
|
||||
)
|
||||
|
||||
this.revID = this.options.revID;
|
||||
this.sanitizer = new Sanitizer(this, options);
|
||||
this.nodes = new Nodes(this.options.node_id);
|
||||
this.observer = new Observer(this, options);
|
||||
this.ticker = new Ticker(this);
|
||||
this.ticker.attach(() => this.commit());
|
||||
this.debug = new Logger(this.options.__debug__);
|
||||
this.notify = new Logger(this.options.verbose ? LogLevel.Warnings : LogLevel.Silent);
|
||||
this.session = new Session();
|
||||
this.revID = this.options.revID
|
||||
this.sanitizer = new Sanitizer(this, options)
|
||||
this.nodes = new Nodes(this.options.node_id)
|
||||
this.observer = new Observer(this, options)
|
||||
this.ticker = new Ticker(this)
|
||||
this.ticker.attach(() => this.commit())
|
||||
this.debug = new Logger(this.options.__debug__)
|
||||
this.notify = new Logger(this.options.verbose ? LogLevel.Warnings : LogLevel.Silent)
|
||||
this.session = new Session()
|
||||
this.session.attachUpdateCallback(({ userID, metadata }) => {
|
||||
if (userID != null) {
|
||||
// TODO: nullable userID
|
||||
this.send(new UserID(userID));
|
||||
this.send(UserID(userID))
|
||||
}
|
||||
if (metadata != null) {
|
||||
Object.entries(metadata).forEach(([key, value]) => this.send(new Metadata(key, value)));
|
||||
Object.entries(metadata).forEach(([key, value]) => this.send(Metadata(key, value)))
|
||||
}
|
||||
});
|
||||
|
||||
// window.localStorage and window.sessionStorage should only be accessed if required, see #490, #637
|
||||
this.localStorage = this.options.localStorage ?? window.localStorage;
|
||||
this.sessionStorage = this.options.sessionStorage ?? window.sessionStorage;
|
||||
})
|
||||
this.localStorage = this.options.localStorage
|
||||
this.sessionStorage = this.options.sessionStorage
|
||||
|
||||
if (sessionToken != null) {
|
||||
this.sessionStorage.setItem(this.options.session_token_key, sessionToken);
|
||||
this.sessionStorage.setItem(this.options.session_token_key, sessionToken)
|
||||
}
|
||||
|
||||
try {
|
||||
this.worker = new Worker(
|
||||
URL.createObjectURL(new Blob(['WEBWORKER_BODY'], { type: 'text/javascript' })),
|
||||
);
|
||||
)
|
||||
this.worker.onerror = (e) => {
|
||||
this._debug('webworker_error', e);
|
||||
};
|
||||
this._debug('webworker_error', e)
|
||||
}
|
||||
this.worker.onmessage = ({ data }: MessageEvent) => {
|
||||
if (data === 'failed') {
|
||||
this.stop();
|
||||
this._debug('worker_failed', {}); // add context (from worker)
|
||||
this.stop()
|
||||
this._debug('worker_failed', {}) // add context (from worker)
|
||||
} else if (data === 'restart') {
|
||||
this.stop();
|
||||
this.start({ forceNew: true });
|
||||
this.stop()
|
||||
this.start({ forceNew: true })
|
||||
}
|
||||
};
|
||||
}
|
||||
const alertWorker = () => {
|
||||
if (this.worker) {
|
||||
this.worker.postMessage(null);
|
||||
this.worker.postMessage(null)
|
||||
}
|
||||
};
|
||||
}
|
||||
// keep better tactics, discard others?
|
||||
this.attachEventListener(window, 'beforeunload', alertWorker, false);
|
||||
this.attachEventListener(document.body, 'mouseleave', alertWorker, false, false);
|
||||
this.attachEventListener(window, 'beforeunload', alertWorker, false)
|
||||
this.attachEventListener(document.body, 'mouseleave', alertWorker, false, false)
|
||||
// TODO: stop session after inactivity timeout (make configurable)
|
||||
this.attachEventListener(document, 'visibilitychange', alertWorker, false);
|
||||
this.attachEventListener(document, 'visibilitychange', alertWorker, false)
|
||||
} catch (e) {
|
||||
this._debug('worker_start', e);
|
||||
this._debug('worker_start', e)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -188,56 +186,56 @@ export default class App {
|
|||
context,
|
||||
error: `${e}`,
|
||||
}),
|
||||
});
|
||||
})
|
||||
}
|
||||
this.debug.error('OpenReplay error: ', context, e);
|
||||
this.debug.error('OpenReplay error: ', context, e)
|
||||
}
|
||||
|
||||
send(message: Message, urgent = false): void {
|
||||
if (this.activityState === ActivityState.NotActive) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
this.messages.push(message);
|
||||
this.messages.push(message)
|
||||
// TODO: commit on start if there were `urgent` sends;
|
||||
// Clearify where urgent can be used for;
|
||||
// Clearify workflow for each type of message in case it was sent before start
|
||||
// (like Fetch before start; maybe add an option "preCapture: boolean" or sth alike)
|
||||
if (this.activityState === ActivityState.Active && urgent) {
|
||||
this.commit();
|
||||
this.commit()
|
||||
}
|
||||
}
|
||||
private commit(): void {
|
||||
if (this.worker && this.messages.length) {
|
||||
this.messages.unshift(new Timestamp(timestamp()));
|
||||
this.worker.postMessage(this.messages);
|
||||
this.commitCallbacks.forEach((cb) => cb(this.messages));
|
||||
this.messages.length = 0;
|
||||
this.messages.unshift(Timestamp(timestamp()))
|
||||
this.worker.postMessage(this.messages)
|
||||
this.commitCallbacks.forEach((cb) => cb(this.messages))
|
||||
this.messages.length = 0
|
||||
}
|
||||
}
|
||||
|
||||
safe<T extends (...args: any[]) => void>(fn: T): T {
|
||||
const app = this;
|
||||
const app = this
|
||||
return function (this: any, ...args: any) {
|
||||
try {
|
||||
fn.apply(this, args);
|
||||
fn.apply(this, args)
|
||||
} catch (e) {
|
||||
app._debug('safe_fn_call', e);
|
||||
app._debug('safe_fn_call', e)
|
||||
// time: timestamp(),
|
||||
// name: e.name,
|
||||
// message: e.message,
|
||||
// stack: e.stack
|
||||
}
|
||||
} as any; // TODO: correct typing
|
||||
} as any // TODO: correct typing
|
||||
}
|
||||
|
||||
attachCommitCallback(cb: CommitCallback): void {
|
||||
this.commitCallbacks.push(cb);
|
||||
this.commitCallbacks.push(cb)
|
||||
}
|
||||
attachStartCallback(cb: StartCallback): void {
|
||||
this.startCallbacks.push(cb);
|
||||
this.startCallbacks.push(cb)
|
||||
}
|
||||
attachStopCallback(cb: () => any): void {
|
||||
this.stopCallbacks.push(cb);
|
||||
this.stopCallbacks.push(cb)
|
||||
}
|
||||
attachEventListener(
|
||||
target: EventTarget,
|
||||
|
|
@ -247,22 +245,22 @@ export default class App {
|
|||
useCapture = true,
|
||||
): void {
|
||||
if (useSafe) {
|
||||
listener = this.safe(listener);
|
||||
listener = this.safe(listener)
|
||||
}
|
||||
this.attachStartCallback(() => target.addEventListener(type, listener, useCapture));
|
||||
this.attachStopCallback(() => target.removeEventListener(type, listener, useCapture));
|
||||
this.attachStartCallback(() => target.addEventListener(type, listener, useCapture))
|
||||
this.attachStopCallback(() => target.removeEventListener(type, listener, useCapture))
|
||||
}
|
||||
|
||||
// TODO: full correct semantic
|
||||
checkRequiredVersion(version: string): boolean {
|
||||
const reqVer = version.split(/[.-]/);
|
||||
const ver = this.version.split(/[.-]/);
|
||||
const reqVer = version.split(/[.-]/)
|
||||
const ver = this.version.split(/[.-]/)
|
||||
for (let i = 0; i < 3; i++) {
|
||||
if (Number(ver[i]) < Number(reqVer[i]) || isNaN(Number(ver[i])) || isNaN(Number(reqVer[i]))) {
|
||||
return false;
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true;
|
||||
return true
|
||||
}
|
||||
|
||||
private getStartInfo() {
|
||||
|
|
@ -273,97 +271,98 @@ export default class App {
|
|||
timestamp: timestamp(), // shouldn't it be set once?
|
||||
trackerVersion: this.version,
|
||||
isSnippet: this.options.__is_snippet,
|
||||
};
|
||||
}
|
||||
}
|
||||
getSessionInfo() {
|
||||
return {
|
||||
...this.session.getInfo(),
|
||||
...this.getStartInfo(),
|
||||
};
|
||||
}
|
||||
}
|
||||
getSessionToken(): string | undefined {
|
||||
const token = this.sessionStorage.getItem(this.options.session_token_key);
|
||||
const token = this.sessionStorage.getItem(this.options.session_token_key)
|
||||
if (token !== null) {
|
||||
return token;
|
||||
return token
|
||||
}
|
||||
}
|
||||
getSessionID(): string | undefined {
|
||||
return this.session.getInfo().sessionID || undefined;
|
||||
return this.session.getInfo().sessionID || undefined
|
||||
}
|
||||
getHost(): string {
|
||||
return new URL(this.options.ingestPoint).hostname;
|
||||
return new URL(this.options.ingestPoint).hostname
|
||||
}
|
||||
getProjectKey(): string {
|
||||
return this.projectKey;
|
||||
return this.projectKey
|
||||
}
|
||||
getBaseHref(): string {
|
||||
if (typeof this.options.resourceBaseHref === 'string') {
|
||||
return this.options.resourceBaseHref;
|
||||
return this.options.resourceBaseHref
|
||||
} else if (typeof this.options.resourceBaseHref === 'object') {
|
||||
//switch between types
|
||||
}
|
||||
if (document.baseURI) {
|
||||
return document.baseURI;
|
||||
return document.baseURI
|
||||
}
|
||||
// IE only
|
||||
return (
|
||||
document.head?.getElementsByTagName('base')[0]?.getAttribute('href') ||
|
||||
location.origin + location.pathname
|
||||
);
|
||||
)
|
||||
}
|
||||
resolveResourceURL(resourceURL: string): string {
|
||||
const base = new URL(this.getBaseHref());
|
||||
base.pathname += '/' + new URL(resourceURL).pathname;
|
||||
base.pathname.replace(/\/+/g, '/');
|
||||
return base.toString();
|
||||
const base = new URL(this.getBaseHref())
|
||||
base.pathname += '/' + new URL(resourceURL).pathname
|
||||
base.pathname.replace(/\/+/g, '/')
|
||||
return base.toString()
|
||||
}
|
||||
|
||||
isServiceURL(url: string): boolean {
|
||||
return url.startsWith(this.options.ingestPoint);
|
||||
return url.startsWith(this.options.ingestPoint)
|
||||
}
|
||||
|
||||
active(): boolean {
|
||||
return this.activityState === ActivityState.Active;
|
||||
return this.activityState === ActivityState.Active
|
||||
}
|
||||
|
||||
resetNextPageSession(flag: boolean) {
|
||||
if (flag) {
|
||||
this.sessionStorage.setItem(this.options.session_reset_key, 't');
|
||||
this.sessionStorage.setItem(this.options.session_reset_key, 't')
|
||||
} else {
|
||||
this.sessionStorage.removeItem(this.options.session_reset_key);
|
||||
this.sessionStorage.removeItem(this.options.session_reset_key)
|
||||
}
|
||||
}
|
||||
private _start(startOpts: StartOptions): Promise<StartPromiseReturn> {
|
||||
if (!this.worker) {
|
||||
return Promise.resolve(UnsuccessfulStart('No worker found: perhaps, CSP is not set.'));
|
||||
return Promise.resolve(UnsuccessfulStart('No worker found: perhaps, CSP is not set.'))
|
||||
}
|
||||
if (this.activityState !== ActivityState.NotActive) {
|
||||
return Promise.resolve(
|
||||
UnsuccessfulStart(
|
||||
'OpenReplay: trying to call `start()` on the instance that has been started already.',
|
||||
),
|
||||
);
|
||||
)
|
||||
}
|
||||
this.activityState = ActivityState.Starting;
|
||||
this.activityState = ActivityState.Starting
|
||||
|
||||
let pageNo = 0;
|
||||
const pageNoStr = this.sessionStorage.getItem(this.options.session_pageno_key);
|
||||
let pageNo = 0
|
||||
const pageNoStr = this.sessionStorage.getItem(this.options.session_pageno_key)
|
||||
if (pageNoStr != null) {
|
||||
pageNo = parseInt(pageNoStr);
|
||||
pageNo++;
|
||||
pageNo = parseInt(pageNoStr)
|
||||
pageNo++
|
||||
}
|
||||
this.sessionStorage.setItem(this.options.session_pageno_key, pageNo.toString());
|
||||
this.sessionStorage.setItem(this.options.session_pageno_key, pageNo.toString())
|
||||
|
||||
const startInfo = this.getStartInfo();
|
||||
const startInfo = this.getStartInfo()
|
||||
const startWorkerMsg: WorkerMessageData = {
|
||||
type: 'start',
|
||||
pageNo,
|
||||
ingestPoint: this.options.ingestPoint,
|
||||
timestamp: startInfo.timestamp,
|
||||
url: document.URL,
|
||||
connAttemptCount: this.options.connAttemptCount,
|
||||
connAttemptGap: this.options.connAttemptGap,
|
||||
};
|
||||
this.worker.postMessage(startWorkerMsg);
|
||||
}
|
||||
this.worker.postMessage(startWorkerMsg)
|
||||
|
||||
this.session.update({
|
||||
// TODO: transparent "session" module logic AND explicit internal api for plugins.
|
||||
|
|
@ -371,10 +370,10 @@ export default class App {
|
|||
// (for the case of internal .start() calls, like on "restart" webworker signal or assistent connection in tracker-assist )
|
||||
metadata: startOpts.metadata || this.session.getInfo().metadata,
|
||||
userID: startOpts.userID,
|
||||
});
|
||||
})
|
||||
|
||||
const sReset = this.sessionStorage.getItem(this.options.session_reset_key);
|
||||
this.sessionStorage.removeItem(this.options.session_reset_key);
|
||||
const sReset = this.sessionStorage.getItem(this.options.session_reset_key)
|
||||
this.sessionStorage.removeItem(this.options.session_reset_key)
|
||||
|
||||
return window
|
||||
.fetch(this.options.ingestPoint + '/v1/web/start', {
|
||||
|
|
@ -393,7 +392,7 @@ export default class App {
|
|||
})
|
||||
.then((r) => {
|
||||
if (r.status === 200) {
|
||||
return r.json();
|
||||
return r.json()
|
||||
} else {
|
||||
return r
|
||||
.text()
|
||||
|
|
@ -401,96 +400,96 @@ export default class App {
|
|||
text === CANCELED
|
||||
? Promise.reject(CANCELED)
|
||||
: Promise.reject(`Server error: ${r.status}. ${text}`),
|
||||
);
|
||||
)
|
||||
}
|
||||
})
|
||||
.then((r) => {
|
||||
if (!this.worker) {
|
||||
return Promise.reject('no worker found after start request (this might not happen)');
|
||||
return Promise.reject('no worker found after start request (this might not happen)')
|
||||
}
|
||||
const { token, userUUID, sessionID, beaconSizeLimit } = r;
|
||||
const { token, userUUID, sessionID, beaconSizeLimit } = r
|
||||
if (
|
||||
typeof token !== 'string' ||
|
||||
typeof userUUID !== 'string' ||
|
||||
(typeof beaconSizeLimit !== 'number' && typeof beaconSizeLimit !== 'undefined')
|
||||
) {
|
||||
return Promise.reject(`Incorrect server response: ${JSON.stringify(r)}`);
|
||||
return Promise.reject(`Incorrect server response: ${JSON.stringify(r)}`)
|
||||
}
|
||||
this.sessionStorage.setItem(this.options.session_token_key, token);
|
||||
this.localStorage.setItem(this.options.local_uuid_key, userUUID);
|
||||
this.session.update({ sessionID }); // TODO: no no-explicit 'any'
|
||||
this.sessionStorage.setItem(this.options.session_token_key, token)
|
||||
this.localStorage.setItem(this.options.local_uuid_key, userUUID)
|
||||
this.session.update({ sessionID }) // TODO: no no-explicit 'any'
|
||||
const startWorkerMsg: WorkerMessageData = {
|
||||
type: 'auth',
|
||||
token,
|
||||
beaconSizeLimit,
|
||||
};
|
||||
this.worker.postMessage(startWorkerMsg);
|
||||
}
|
||||
this.worker.postMessage(startWorkerMsg)
|
||||
|
||||
this.activityState = ActivityState.Active;
|
||||
this.activityState = ActivityState.Active
|
||||
|
||||
const onStartInfo = { sessionToken: token, userUUID, sessionID };
|
||||
const onStartInfo = { sessionToken: token, userUUID, sessionID }
|
||||
|
||||
this.startCallbacks.forEach((cb) => cb(onStartInfo)); // TODO: start as early as possible (before receiving the token)
|
||||
this.observer.observe();
|
||||
this.ticker.start();
|
||||
this.startCallbacks.forEach((cb) => cb(onStartInfo)) // TODO: start as early as possible (before receiving the token)
|
||||
this.observer.observe()
|
||||
this.ticker.start()
|
||||
|
||||
this.notify.log('OpenReplay tracking started.');
|
||||
this.notify.log('OpenReplay tracking started.')
|
||||
// get rid of onStart ?
|
||||
if (typeof this.options.onStart === 'function') {
|
||||
this.options.onStart(onStartInfo);
|
||||
this.options.onStart(onStartInfo)
|
||||
}
|
||||
return SuccessfulStart(onStartInfo);
|
||||
return SuccessfulStart(onStartInfo)
|
||||
})
|
||||
.catch((reason) => {
|
||||
this.sessionStorage.removeItem(this.options.session_token_key);
|
||||
this.stop();
|
||||
this.sessionStorage.removeItem(this.options.session_token_key)
|
||||
this.stop()
|
||||
if (reason === CANCELED) {
|
||||
return UnsuccessfulStart(CANCELED);
|
||||
return UnsuccessfulStart(CANCELED)
|
||||
}
|
||||
|
||||
this.notify.log('OpenReplay was unable to start. ', reason);
|
||||
this._debug('session_start', reason);
|
||||
return UnsuccessfulStart(START_ERROR);
|
||||
});
|
||||
this.notify.log('OpenReplay was unable to start. ', reason)
|
||||
this._debug('session_start', reason)
|
||||
return UnsuccessfulStart(START_ERROR)
|
||||
})
|
||||
}
|
||||
|
||||
start(options: StartOptions = {}): Promise<StartPromiseReturn> {
|
||||
if (!document.hidden) {
|
||||
return this._start(options);
|
||||
return this._start(options)
|
||||
} else {
|
||||
return new Promise((resolve) => {
|
||||
const onVisibilityChange = () => {
|
||||
if (!document.hidden) {
|
||||
document.removeEventListener('visibilitychange', onVisibilityChange);
|
||||
resolve(this._start(options));
|
||||
document.removeEventListener('visibilitychange', onVisibilityChange)
|
||||
resolve(this._start(options))
|
||||
}
|
||||
};
|
||||
document.addEventListener('visibilitychange', onVisibilityChange);
|
||||
});
|
||||
}
|
||||
document.addEventListener('visibilitychange', onVisibilityChange)
|
||||
})
|
||||
}
|
||||
}
|
||||
stop(calledFromAPI = false, restarting = false): void {
|
||||
if (this.activityState !== ActivityState.NotActive) {
|
||||
try {
|
||||
this.sanitizer.clear();
|
||||
this.observer.disconnect();
|
||||
this.nodes.clear();
|
||||
this.ticker.stop();
|
||||
this.stopCallbacks.forEach((cb) => cb());
|
||||
this.sanitizer.clear()
|
||||
this.observer.disconnect()
|
||||
this.nodes.clear()
|
||||
this.ticker.stop()
|
||||
this.stopCallbacks.forEach((cb) => cb())
|
||||
if (calledFromAPI) {
|
||||
this.session.reset();
|
||||
this.session.reset()
|
||||
}
|
||||
this.notify.log('OpenReplay tracking stopped.');
|
||||
this.notify.log('OpenReplay tracking stopped.')
|
||||
if (this.worker && !restarting) {
|
||||
this.worker.postMessage('stop');
|
||||
this.worker.postMessage('stop')
|
||||
}
|
||||
} finally {
|
||||
this.activityState = ActivityState.NotActive;
|
||||
this.activityState = ActivityState.NotActive
|
||||
}
|
||||
}
|
||||
}
|
||||
restart() {
|
||||
this.stop(false, true);
|
||||
this.start({ forceNew: false });
|
||||
this.stop(false, true)
|
||||
this.start({ forceNew: false })
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,35 +4,35 @@ export const LogLevel = {
|
|||
Warnings: 3,
|
||||
Errors: 2,
|
||||
Silent: 0,
|
||||
} as const;
|
||||
type LogLevel = typeof LogLevel[keyof typeof LogLevel];
|
||||
} as const
|
||||
type LogLevel = typeof LogLevel[keyof typeof LogLevel]
|
||||
|
||||
type CustomLevel = {
|
||||
error: boolean;
|
||||
warn: boolean;
|
||||
log: boolean;
|
||||
};
|
||||
error: boolean
|
||||
warn: boolean
|
||||
log: boolean
|
||||
}
|
||||
|
||||
function IsCustomLevel(l: LogLevel | CustomLevel): l is CustomLevel {
|
||||
return typeof l === 'object';
|
||||
return typeof l === 'object'
|
||||
}
|
||||
|
||||
interface _Options {
|
||||
level: LogLevel | CustomLevel;
|
||||
messages?: number[];
|
||||
level: LogLevel | CustomLevel
|
||||
messages?: number[]
|
||||
}
|
||||
|
||||
export type Options = true | _Options | LogLevel;
|
||||
export type Options = true | _Options | LogLevel
|
||||
|
||||
export default class Logger {
|
||||
private readonly options: _Options;
|
||||
private readonly options: _Options
|
||||
constructor(options: Options = LogLevel.Silent) {
|
||||
this.options =
|
||||
options === true
|
||||
? { level: LogLevel.Verbose }
|
||||
: typeof options === 'number'
|
||||
? { level: options }
|
||||
: options;
|
||||
: options
|
||||
}
|
||||
log(...args: any) {
|
||||
if (
|
||||
|
|
@ -40,7 +40,7 @@ export default class Logger {
|
|||
? this.options.level.log
|
||||
: this.options.level >= LogLevel.Log
|
||||
) {
|
||||
console.log(...args);
|
||||
console.log(...args)
|
||||
}
|
||||
}
|
||||
warn(...args: any) {
|
||||
|
|
@ -49,7 +49,7 @@ export default class Logger {
|
|||
? this.options.level.warn
|
||||
: this.options.level >= LogLevel.Warnings
|
||||
) {
|
||||
console.warn(...args);
|
||||
console.warn(...args)
|
||||
}
|
||||
}
|
||||
error(...args: any) {
|
||||
|
|
@ -58,7 +58,7 @@ export default class Logger {
|
|||
? this.options.level.error
|
||||
: this.options.level >= LogLevel.Errors
|
||||
) {
|
||||
console.error(...args);
|
||||
console.error(...args)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
648
tracker/tracker/src/main/app/messages.gen.ts
Normal file
648
tracker/tracker/src/main/app/messages.gen.ts
Normal file
|
|
@ -0,0 +1,648 @@
|
|||
// Auto-generated, do not edit
|
||||
|
||||
import * as Messages from '../../common/messages.gen.js'
|
||||
export { default } from '../../common/messages.gen.js'
|
||||
|
||||
|
||||
export function BatchMetadata(
|
||||
version: number,
|
||||
pageNo: number,
|
||||
firstIndex: number,
|
||||
timestamp: number,
|
||||
location: string,
|
||||
): Messages.BatchMetadata {
|
||||
return [
|
||||
Messages.Type.BatchMetadata,
|
||||
version,
|
||||
pageNo,
|
||||
firstIndex,
|
||||
timestamp,
|
||||
location,
|
||||
]
|
||||
}
|
||||
|
||||
export function PartitionedMessage(
|
||||
partNo: number,
|
||||
partTotal: number,
|
||||
): Messages.PartitionedMessage {
|
||||
return [
|
||||
Messages.Type.PartitionedMessage,
|
||||
partNo,
|
||||
partTotal,
|
||||
]
|
||||
}
|
||||
|
||||
export function Timestamp(
|
||||
timestamp: number,
|
||||
): Messages.Timestamp {
|
||||
return [
|
||||
Messages.Type.Timestamp,
|
||||
timestamp,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetPageLocation(
|
||||
url: string,
|
||||
referrer: string,
|
||||
navigationStart: number,
|
||||
): Messages.SetPageLocation {
|
||||
return [
|
||||
Messages.Type.SetPageLocation,
|
||||
url,
|
||||
referrer,
|
||||
navigationStart,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetViewportSize(
|
||||
width: number,
|
||||
height: number,
|
||||
): Messages.SetViewportSize {
|
||||
return [
|
||||
Messages.Type.SetViewportSize,
|
||||
width,
|
||||
height,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetViewportScroll(
|
||||
x: number,
|
||||
y: number,
|
||||
): Messages.SetViewportScroll {
|
||||
return [
|
||||
Messages.Type.SetViewportScroll,
|
||||
x,
|
||||
y,
|
||||
]
|
||||
}
|
||||
|
||||
export function CreateDocument(
|
||||
|
||||
): Messages.CreateDocument {
|
||||
return [
|
||||
Messages.Type.CreateDocument,
|
||||
|
||||
]
|
||||
}
|
||||
|
||||
export function CreateElementNode(
|
||||
id: number,
|
||||
parentID: number,
|
||||
index: number,
|
||||
tag: string,
|
||||
svg: boolean,
|
||||
): Messages.CreateElementNode {
|
||||
return [
|
||||
Messages.Type.CreateElementNode,
|
||||
id,
|
||||
parentID,
|
||||
index,
|
||||
tag,
|
||||
svg,
|
||||
]
|
||||
}
|
||||
|
||||
export function CreateTextNode(
|
||||
id: number,
|
||||
parentID: number,
|
||||
index: number,
|
||||
): Messages.CreateTextNode {
|
||||
return [
|
||||
Messages.Type.CreateTextNode,
|
||||
id,
|
||||
parentID,
|
||||
index,
|
||||
]
|
||||
}
|
||||
|
||||
export function MoveNode(
|
||||
id: number,
|
||||
parentID: number,
|
||||
index: number,
|
||||
): Messages.MoveNode {
|
||||
return [
|
||||
Messages.Type.MoveNode,
|
||||
id,
|
||||
parentID,
|
||||
index,
|
||||
]
|
||||
}
|
||||
|
||||
export function RemoveNode(
|
||||
id: number,
|
||||
): Messages.RemoveNode {
|
||||
return [
|
||||
Messages.Type.RemoveNode,
|
||||
id,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetNodeAttribute(
|
||||
id: number,
|
||||
name: string,
|
||||
value: string,
|
||||
): Messages.SetNodeAttribute {
|
||||
return [
|
||||
Messages.Type.SetNodeAttribute,
|
||||
id,
|
||||
name,
|
||||
value,
|
||||
]
|
||||
}
|
||||
|
||||
export function RemoveNodeAttribute(
|
||||
id: number,
|
||||
name: string,
|
||||
): Messages.RemoveNodeAttribute {
|
||||
return [
|
||||
Messages.Type.RemoveNodeAttribute,
|
||||
id,
|
||||
name,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetNodeData(
|
||||
id: number,
|
||||
data: string,
|
||||
): Messages.SetNodeData {
|
||||
return [
|
||||
Messages.Type.SetNodeData,
|
||||
id,
|
||||
data,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetNodeScroll(
|
||||
id: number,
|
||||
x: number,
|
||||
y: number,
|
||||
): Messages.SetNodeScroll {
|
||||
return [
|
||||
Messages.Type.SetNodeScroll,
|
||||
id,
|
||||
x,
|
||||
y,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetInputTarget(
|
||||
id: number,
|
||||
label: string,
|
||||
): Messages.SetInputTarget {
|
||||
return [
|
||||
Messages.Type.SetInputTarget,
|
||||
id,
|
||||
label,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetInputValue(
|
||||
id: number,
|
||||
value: string,
|
||||
mask: number,
|
||||
): Messages.SetInputValue {
|
||||
return [
|
||||
Messages.Type.SetInputValue,
|
||||
id,
|
||||
value,
|
||||
mask,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetInputChecked(
|
||||
id: number,
|
||||
checked: boolean,
|
||||
): Messages.SetInputChecked {
|
||||
return [
|
||||
Messages.Type.SetInputChecked,
|
||||
id,
|
||||
checked,
|
||||
]
|
||||
}
|
||||
|
||||
export function MouseMove(
|
||||
x: number,
|
||||
y: number,
|
||||
): Messages.MouseMove {
|
||||
return [
|
||||
Messages.Type.MouseMove,
|
||||
x,
|
||||
y,
|
||||
]
|
||||
}
|
||||
|
||||
export function ConsoleLog(
|
||||
level: string,
|
||||
value: string,
|
||||
): Messages.ConsoleLog {
|
||||
return [
|
||||
Messages.Type.ConsoleLog,
|
||||
level,
|
||||
value,
|
||||
]
|
||||
}
|
||||
|
||||
export function PageLoadTiming(
|
||||
requestStart: number,
|
||||
responseStart: number,
|
||||
responseEnd: number,
|
||||
domContentLoadedEventStart: number,
|
||||
domContentLoadedEventEnd: number,
|
||||
loadEventStart: number,
|
||||
loadEventEnd: number,
|
||||
firstPaint: number,
|
||||
firstContentfulPaint: number,
|
||||
): Messages.PageLoadTiming {
|
||||
return [
|
||||
Messages.Type.PageLoadTiming,
|
||||
requestStart,
|
||||
responseStart,
|
||||
responseEnd,
|
||||
domContentLoadedEventStart,
|
||||
domContentLoadedEventEnd,
|
||||
loadEventStart,
|
||||
loadEventEnd,
|
||||
firstPaint,
|
||||
firstContentfulPaint,
|
||||
]
|
||||
}
|
||||
|
||||
export function PageRenderTiming(
|
||||
speedIndex: number,
|
||||
visuallyComplete: number,
|
||||
timeToInteractive: number,
|
||||
): Messages.PageRenderTiming {
|
||||
return [
|
||||
Messages.Type.PageRenderTiming,
|
||||
speedIndex,
|
||||
visuallyComplete,
|
||||
timeToInteractive,
|
||||
]
|
||||
}
|
||||
|
||||
export function JSException(
|
||||
name: string,
|
||||
message: string,
|
||||
payload: string,
|
||||
): Messages.JSException {
|
||||
return [
|
||||
Messages.Type.JSException,
|
||||
name,
|
||||
message,
|
||||
payload,
|
||||
]
|
||||
}
|
||||
|
||||
export function RawCustomEvent(
|
||||
name: string,
|
||||
payload: string,
|
||||
): Messages.RawCustomEvent {
|
||||
return [
|
||||
Messages.Type.RawCustomEvent,
|
||||
name,
|
||||
payload,
|
||||
]
|
||||
}
|
||||
|
||||
export function UserID(
|
||||
id: string,
|
||||
): Messages.UserID {
|
||||
return [
|
||||
Messages.Type.UserID,
|
||||
id,
|
||||
]
|
||||
}
|
||||
|
||||
export function UserAnonymousID(
|
||||
id: string,
|
||||
): Messages.UserAnonymousID {
|
||||
return [
|
||||
Messages.Type.UserAnonymousID,
|
||||
id,
|
||||
]
|
||||
}
|
||||
|
||||
export function Metadata(
|
||||
key: string,
|
||||
value: string,
|
||||
): Messages.Metadata {
|
||||
return [
|
||||
Messages.Type.Metadata,
|
||||
key,
|
||||
value,
|
||||
]
|
||||
}
|
||||
|
||||
export function CSSInsertRule(
|
||||
id: number,
|
||||
rule: string,
|
||||
index: number,
|
||||
): Messages.CSSInsertRule {
|
||||
return [
|
||||
Messages.Type.CSSInsertRule,
|
||||
id,
|
||||
rule,
|
||||
index,
|
||||
]
|
||||
}
|
||||
|
||||
export function CSSDeleteRule(
|
||||
id: number,
|
||||
index: number,
|
||||
): Messages.CSSDeleteRule {
|
||||
return [
|
||||
Messages.Type.CSSDeleteRule,
|
||||
id,
|
||||
index,
|
||||
]
|
||||
}
|
||||
|
||||
export function Fetch(
|
||||
method: string,
|
||||
url: string,
|
||||
request: string,
|
||||
response: string,
|
||||
status: number,
|
||||
timestamp: number,
|
||||
duration: number,
|
||||
): Messages.Fetch {
|
||||
return [
|
||||
Messages.Type.Fetch,
|
||||
method,
|
||||
url,
|
||||
request,
|
||||
response,
|
||||
status,
|
||||
timestamp,
|
||||
duration,
|
||||
]
|
||||
}
|
||||
|
||||
export function Profiler(
|
||||
name: string,
|
||||
duration: number,
|
||||
args: string,
|
||||
result: string,
|
||||
): Messages.Profiler {
|
||||
return [
|
||||
Messages.Type.Profiler,
|
||||
name,
|
||||
duration,
|
||||
args,
|
||||
result,
|
||||
]
|
||||
}
|
||||
|
||||
export function OTable(
|
||||
key: string,
|
||||
value: string,
|
||||
): Messages.OTable {
|
||||
return [
|
||||
Messages.Type.OTable,
|
||||
key,
|
||||
value,
|
||||
]
|
||||
}
|
||||
|
||||
export function StateAction(
|
||||
type: string,
|
||||
): Messages.StateAction {
|
||||
return [
|
||||
Messages.Type.StateAction,
|
||||
type,
|
||||
]
|
||||
}
|
||||
|
||||
export function Redux(
|
||||
action: string,
|
||||
state: string,
|
||||
duration: number,
|
||||
): Messages.Redux {
|
||||
return [
|
||||
Messages.Type.Redux,
|
||||
action,
|
||||
state,
|
||||
duration,
|
||||
]
|
||||
}
|
||||
|
||||
export function Vuex(
|
||||
mutation: string,
|
||||
state: string,
|
||||
): Messages.Vuex {
|
||||
return [
|
||||
Messages.Type.Vuex,
|
||||
mutation,
|
||||
state,
|
||||
]
|
||||
}
|
||||
|
||||
export function MobX(
|
||||
type: string,
|
||||
payload: string,
|
||||
): Messages.MobX {
|
||||
return [
|
||||
Messages.Type.MobX,
|
||||
type,
|
||||
payload,
|
||||
]
|
||||
}
|
||||
|
||||
export function NgRx(
|
||||
action: string,
|
||||
state: string,
|
||||
duration: number,
|
||||
): Messages.NgRx {
|
||||
return [
|
||||
Messages.Type.NgRx,
|
||||
action,
|
||||
state,
|
||||
duration,
|
||||
]
|
||||
}
|
||||
|
||||
export function GraphQL(
|
||||
operationKind: string,
|
||||
operationName: string,
|
||||
variables: string,
|
||||
response: string,
|
||||
): Messages.GraphQL {
|
||||
return [
|
||||
Messages.Type.GraphQL,
|
||||
operationKind,
|
||||
operationName,
|
||||
variables,
|
||||
response,
|
||||
]
|
||||
}
|
||||
|
||||
export function PerformanceTrack(
|
||||
frames: number,
|
||||
ticks: number,
|
||||
totalJSHeapSize: number,
|
||||
usedJSHeapSize: number,
|
||||
): Messages.PerformanceTrack {
|
||||
return [
|
||||
Messages.Type.PerformanceTrack,
|
||||
frames,
|
||||
ticks,
|
||||
totalJSHeapSize,
|
||||
usedJSHeapSize,
|
||||
]
|
||||
}
|
||||
|
||||
export function ResourceTiming(
|
||||
timestamp: number,
|
||||
duration: number,
|
||||
ttfb: number,
|
||||
headerSize: number,
|
||||
encodedBodySize: number,
|
||||
decodedBodySize: number,
|
||||
url: string,
|
||||
initiator: string,
|
||||
): Messages.ResourceTiming {
|
||||
return [
|
||||
Messages.Type.ResourceTiming,
|
||||
timestamp,
|
||||
duration,
|
||||
ttfb,
|
||||
headerSize,
|
||||
encodedBodySize,
|
||||
decodedBodySize,
|
||||
url,
|
||||
initiator,
|
||||
]
|
||||
}
|
||||
|
||||
export function ConnectionInformation(
|
||||
downlink: number,
|
||||
type: string,
|
||||
): Messages.ConnectionInformation {
|
||||
return [
|
||||
Messages.Type.ConnectionInformation,
|
||||
downlink,
|
||||
type,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetPageVisibility(
|
||||
hidden: boolean,
|
||||
): Messages.SetPageVisibility {
|
||||
return [
|
||||
Messages.Type.SetPageVisibility,
|
||||
hidden,
|
||||
]
|
||||
}
|
||||
|
||||
export function LongTask(
|
||||
timestamp: number,
|
||||
duration: number,
|
||||
context: number,
|
||||
containerType: number,
|
||||
containerSrc: string,
|
||||
containerId: string,
|
||||
containerName: string,
|
||||
): Messages.LongTask {
|
||||
return [
|
||||
Messages.Type.LongTask,
|
||||
timestamp,
|
||||
duration,
|
||||
context,
|
||||
containerType,
|
||||
containerSrc,
|
||||
containerId,
|
||||
containerName,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetNodeAttributeURLBased(
|
||||
id: number,
|
||||
name: string,
|
||||
value: string,
|
||||
baseURL: string,
|
||||
): Messages.SetNodeAttributeURLBased {
|
||||
return [
|
||||
Messages.Type.SetNodeAttributeURLBased,
|
||||
id,
|
||||
name,
|
||||
value,
|
||||
baseURL,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetCSSDataURLBased(
|
||||
id: number,
|
||||
data: string,
|
||||
baseURL: string,
|
||||
): Messages.SetCSSDataURLBased {
|
||||
return [
|
||||
Messages.Type.SetCSSDataURLBased,
|
||||
id,
|
||||
data,
|
||||
baseURL,
|
||||
]
|
||||
}
|
||||
|
||||
export function TechnicalInfo(
|
||||
type: string,
|
||||
value: string,
|
||||
): Messages.TechnicalInfo {
|
||||
return [
|
||||
Messages.Type.TechnicalInfo,
|
||||
type,
|
||||
value,
|
||||
]
|
||||
}
|
||||
|
||||
export function CustomIssue(
|
||||
name: string,
|
||||
payload: string,
|
||||
): Messages.CustomIssue {
|
||||
return [
|
||||
Messages.Type.CustomIssue,
|
||||
name,
|
||||
payload,
|
||||
]
|
||||
}
|
||||
|
||||
export function CSSInsertRuleURLBased(
|
||||
id: number,
|
||||
rule: string,
|
||||
index: number,
|
||||
baseURL: string,
|
||||
): Messages.CSSInsertRuleURLBased {
|
||||
return [
|
||||
Messages.Type.CSSInsertRuleURLBased,
|
||||
id,
|
||||
rule,
|
||||
index,
|
||||
baseURL,
|
||||
]
|
||||
}
|
||||
|
||||
export function MouseClick(
|
||||
id: number,
|
||||
hesitationTime: number,
|
||||
label: string,
|
||||
selector: string,
|
||||
): Messages.MouseClick {
|
||||
return [
|
||||
Messages.Type.MouseClick,
|
||||
id,
|
||||
hesitationTime,
|
||||
label,
|
||||
selector,
|
||||
]
|
||||
}
|
||||
|
||||
export function CreateIFrameDocument(
|
||||
frameID: number,
|
||||
id: number,
|
||||
): Messages.CreateIFrameDocument {
|
||||
return [
|
||||
Messages.Type.CreateIFrameDocument,
|
||||
frameID,
|
||||
id,
|
||||
]
|
||||
}
|
||||
|
||||
334
tracker/tracker/src/main/app/messages.ts
Normal file
334
tracker/tracker/src/main/app/messages.ts
Normal file
|
|
@ -0,0 +1,334 @@
|
|||
// Auto-generated, do not edit
|
||||
|
||||
import * as Messages from '../../common/messages.gen.js'
|
||||
export { default } from '../../common/messages.gen.js'
|
||||
|
||||
export function BatchMetadata(
|
||||
version: number,
|
||||
pageNo: number,
|
||||
firstIndex: number,
|
||||
timestamp: number,
|
||||
location: string,
|
||||
): Messages.BatchMetadata {
|
||||
return [Messages.Type.BatchMetadata, version, pageNo, firstIndex, timestamp, location]
|
||||
}
|
||||
|
||||
export function PartitionedMessage(partNo: number, partTotal: number): Messages.PartitionedMessage {
|
||||
return [Messages.Type.PartitionedMessage, partNo, partTotal]
|
||||
}
|
||||
|
||||
export function Timestamp(timestamp: number): Messages.Timestamp {
|
||||
return [Messages.Type.Timestamp, timestamp]
|
||||
}
|
||||
|
||||
export function SetPageLocation(
|
||||
url: string,
|
||||
referrer: string,
|
||||
navigationStart: number,
|
||||
): Messages.SetPageLocation {
|
||||
return [Messages.Type.SetPageLocation, url, referrer, navigationStart]
|
||||
}
|
||||
|
||||
export function SetViewportSize(width: number, height: number): Messages.SetViewportSize {
|
||||
return [Messages.Type.SetViewportSize, width, height]
|
||||
}
|
||||
|
||||
export function SetViewportScroll(x: number, y: number): Messages.SetViewportScroll {
|
||||
return [Messages.Type.SetViewportScroll, x, y]
|
||||
}
|
||||
|
||||
export function CreateDocument(): Messages.CreateDocument {
|
||||
return [Messages.Type.CreateDocument]
|
||||
}
|
||||
|
||||
export function CreateElementNode(
|
||||
id: number,
|
||||
parentID: number,
|
||||
index: number,
|
||||
tag: string,
|
||||
svg: boolean,
|
||||
): Messages.CreateElementNode {
|
||||
return [Messages.Type.CreateElementNode, id, parentID, index, tag, svg]
|
||||
}
|
||||
|
||||
export function CreateTextNode(
|
||||
id: number,
|
||||
parentID: number,
|
||||
index: number,
|
||||
): Messages.CreateTextNode {
|
||||
return [Messages.Type.CreateTextNode, id, parentID, index]
|
||||
}
|
||||
|
||||
export function MoveNode(id: number, parentID: number, index: number): Messages.MoveNode {
|
||||
return [Messages.Type.MoveNode, id, parentID, index]
|
||||
}
|
||||
|
||||
export function RemoveNode(id: number): Messages.RemoveNode {
|
||||
return [Messages.Type.RemoveNode, id]
|
||||
}
|
||||
|
||||
export function SetNodeAttribute(
|
||||
id: number,
|
||||
name: string,
|
||||
value: string,
|
||||
): Messages.SetNodeAttribute {
|
||||
return [Messages.Type.SetNodeAttribute, id, name, value]
|
||||
}
|
||||
|
||||
export function RemoveNodeAttribute(id: number, name: string): Messages.RemoveNodeAttribute {
|
||||
return [Messages.Type.RemoveNodeAttribute, id, name]
|
||||
}
|
||||
|
||||
export function SetNodeData(id: number, data: string): Messages.SetNodeData {
|
||||
return [Messages.Type.SetNodeData, id, data]
|
||||
}
|
||||
|
||||
export function SetNodeScroll(id: number, x: number, y: number): Messages.SetNodeScroll {
|
||||
return [Messages.Type.SetNodeScroll, id, x, y]
|
||||
}
|
||||
|
||||
export function SetInputTarget(id: number, label: string): Messages.SetInputTarget {
|
||||
return [Messages.Type.SetInputTarget, id, label]
|
||||
}
|
||||
|
||||
export function SetInputValue(id: number, value: string, mask: number): Messages.SetInputValue {
|
||||
return [Messages.Type.SetInputValue, id, value, mask]
|
||||
}
|
||||
|
||||
export function SetInputChecked(id: number, checked: boolean): Messages.SetInputChecked {
|
||||
return [Messages.Type.SetInputChecked, id, checked]
|
||||
}
|
||||
|
||||
export function MouseMove(x: number, y: number): Messages.MouseMove {
|
||||
return [Messages.Type.MouseMove, x, y]
|
||||
}
|
||||
|
||||
export function ConsoleLog(level: string, value: string): Messages.ConsoleLog {
|
||||
return [Messages.Type.ConsoleLog, level, value]
|
||||
}
|
||||
|
||||
export function PageLoadTiming(
|
||||
requestStart: number,
|
||||
responseStart: number,
|
||||
responseEnd: number,
|
||||
domContentLoadedEventStart: number,
|
||||
domContentLoadedEventEnd: number,
|
||||
loadEventStart: number,
|
||||
loadEventEnd: number,
|
||||
firstPaint: number,
|
||||
firstContentfulPaint: number,
|
||||
): Messages.PageLoadTiming {
|
||||
return [
|
||||
Messages.Type.PageLoadTiming,
|
||||
requestStart,
|
||||
responseStart,
|
||||
responseEnd,
|
||||
domContentLoadedEventStart,
|
||||
domContentLoadedEventEnd,
|
||||
loadEventStart,
|
||||
loadEventEnd,
|
||||
firstPaint,
|
||||
firstContentfulPaint,
|
||||
]
|
||||
}
|
||||
|
||||
export function PageRenderTiming(
|
||||
speedIndex: number,
|
||||
visuallyComplete: number,
|
||||
timeToInteractive: number,
|
||||
): Messages.PageRenderTiming {
|
||||
return [Messages.Type.PageRenderTiming, speedIndex, visuallyComplete, timeToInteractive]
|
||||
}
|
||||
|
||||
export function JSException(name: string, message: string, payload: string): Messages.JSException {
|
||||
return [Messages.Type.JSException, name, message, payload]
|
||||
}
|
||||
|
||||
export function RawCustomEvent(name: string, payload: string): Messages.RawCustomEvent {
|
||||
return [Messages.Type.RawCustomEvent, name, payload]
|
||||
}
|
||||
|
||||
export function UserID(id: string): Messages.UserID {
|
||||
return [Messages.Type.UserID, id]
|
||||
}
|
||||
|
||||
export function UserAnonymousID(id: string): Messages.UserAnonymousID {
|
||||
return [Messages.Type.UserAnonymousID, id]
|
||||
}
|
||||
|
||||
export function Metadata(key: string, value: string): Messages.Metadata {
|
||||
return [Messages.Type.Metadata, key, value]
|
||||
}
|
||||
|
||||
export function CSSInsertRule(id: number, rule: string, index: number): Messages.CSSInsertRule {
|
||||
return [Messages.Type.CSSInsertRule, id, rule, index]
|
||||
}
|
||||
|
||||
export function CSSDeleteRule(id: number, index: number): Messages.CSSDeleteRule {
|
||||
return [Messages.Type.CSSDeleteRule, id, index]
|
||||
}
|
||||
|
||||
export function Fetch(
|
||||
method: string,
|
||||
url: string,
|
||||
request: string,
|
||||
response: string,
|
||||
status: number,
|
||||
timestamp: number,
|
||||
duration: number,
|
||||
): Messages.Fetch {
|
||||
return [Messages.Type.Fetch, method, url, request, response, status, timestamp, duration]
|
||||
}
|
||||
|
||||
export function Profiler(
|
||||
name: string,
|
||||
duration: number,
|
||||
args: string,
|
||||
result: string,
|
||||
): Messages.Profiler {
|
||||
return [Messages.Type.Profiler, name, duration, args, result]
|
||||
}
|
||||
|
||||
export function OTable(key: string, value: string): Messages.OTable {
|
||||
return [Messages.Type.OTable, key, value]
|
||||
}
|
||||
|
||||
export function StateAction(type: string): Messages.StateAction {
|
||||
return [Messages.Type.StateAction, type]
|
||||
}
|
||||
|
||||
export function Redux(action: string, state: string, duration: number): Messages.Redux {
|
||||
return [Messages.Type.Redux, action, state, duration]
|
||||
}
|
||||
|
||||
export function Vuex(mutation: string, state: string): Messages.Vuex {
|
||||
return [Messages.Type.Vuex, mutation, state]
|
||||
}
|
||||
|
||||
export function MobX(type: string, payload: string): Messages.MobX {
|
||||
return [Messages.Type.MobX, type, payload]
|
||||
}
|
||||
|
||||
export function NgRx(action: string, state: string, duration: number): Messages.NgRx {
|
||||
return [Messages.Type.NgRx, action, state, duration]
|
||||
}
|
||||
|
||||
export function GraphQL(
|
||||
operationKind: string,
|
||||
operationName: string,
|
||||
variables: string,
|
||||
response: string,
|
||||
): Messages.GraphQL {
|
||||
return [Messages.Type.GraphQL, operationKind, operationName, variables, response]
|
||||
}
|
||||
|
||||
export function PerformanceTrack(
|
||||
frames: number,
|
||||
ticks: number,
|
||||
totalJSHeapSize: number,
|
||||
usedJSHeapSize: number,
|
||||
): Messages.PerformanceTrack {
|
||||
return [Messages.Type.PerformanceTrack, frames, ticks, totalJSHeapSize, usedJSHeapSize]
|
||||
}
|
||||
|
||||
export function ResourceTiming(
|
||||
timestamp: number,
|
||||
duration: number,
|
||||
ttfb: number,
|
||||
headerSize: number,
|
||||
encodedBodySize: number,
|
||||
decodedBodySize: number,
|
||||
url: string,
|
||||
initiator: string,
|
||||
): Messages.ResourceTiming {
|
||||
return [
|
||||
Messages.Type.ResourceTiming,
|
||||
timestamp,
|
||||
duration,
|
||||
ttfb,
|
||||
headerSize,
|
||||
encodedBodySize,
|
||||
decodedBodySize,
|
||||
url,
|
||||
initiator,
|
||||
]
|
||||
}
|
||||
|
||||
export function ConnectionInformation(
|
||||
downlink: number,
|
||||
type: string,
|
||||
): Messages.ConnectionInformation {
|
||||
return [Messages.Type.ConnectionInformation, downlink, type]
|
||||
}
|
||||
|
||||
export function SetPageVisibility(hidden: boolean): Messages.SetPageVisibility {
|
||||
return [Messages.Type.SetPageVisibility, hidden]
|
||||
}
|
||||
|
||||
export function LongTask(
|
||||
timestamp: number,
|
||||
duration: number,
|
||||
context: number,
|
||||
containerType: number,
|
||||
containerSrc: string,
|
||||
containerId: string,
|
||||
containerName: string,
|
||||
): Messages.LongTask {
|
||||
return [
|
||||
Messages.Type.LongTask,
|
||||
timestamp,
|
||||
duration,
|
||||
context,
|
||||
containerType,
|
||||
containerSrc,
|
||||
containerId,
|
||||
containerName,
|
||||
]
|
||||
}
|
||||
|
||||
export function SetNodeAttributeURLBased(
|
||||
id: number,
|
||||
name: string,
|
||||
value: string,
|
||||
baseURL: string,
|
||||
): Messages.SetNodeAttributeURLBased {
|
||||
return [Messages.Type.SetNodeAttributeURLBased, id, name, value, baseURL]
|
||||
}
|
||||
|
||||
export function SetCSSDataURLBased(
|
||||
id: number,
|
||||
data: string,
|
||||
baseURL: string,
|
||||
): Messages.SetCSSDataURLBased {
|
||||
return [Messages.Type.SetCSSDataURLBased, id, data, baseURL]
|
||||
}
|
||||
|
||||
export function TechnicalInfo(type: string, value: string): Messages.TechnicalInfo {
|
||||
return [Messages.Type.TechnicalInfo, type, value]
|
||||
}
|
||||
|
||||
export function CustomIssue(name: string, payload: string): Messages.CustomIssue {
|
||||
return [Messages.Type.CustomIssue, name, payload]
|
||||
}
|
||||
|
||||
export function CSSInsertRuleURLBased(
|
||||
id: number,
|
||||
rule: string,
|
||||
index: number,
|
||||
baseURL: string,
|
||||
): Messages.CSSInsertRuleURLBased {
|
||||
return [Messages.Type.CSSInsertRuleURLBased, id, rule, index, baseURL]
|
||||
}
|
||||
|
||||
export function MouseClick(
|
||||
id: number,
|
||||
hesitationTime: number,
|
||||
label: string,
|
||||
selector: string,
|
||||
): Messages.MouseClick {
|
||||
return [Messages.Type.MouseClick, id, hesitationTime, label, selector]
|
||||
}
|
||||
|
||||
export function CreateIFrameDocument(frameID: number, id: number): Messages.CreateIFrameDocument {
|
||||
return [Messages.Type.CreateIFrameDocument, frameID, id]
|
||||
}
|
||||
|
|
@ -1,53 +1,53 @@
|
|||
type NodeCallback = (node: Node, isStart: boolean) => void;
|
||||
type ElementListener = [string, EventListener];
|
||||
type NodeCallback = (node: Node, isStart: boolean) => void
|
||||
type ElementListener = [string, EventListener]
|
||||
|
||||
export default class Nodes {
|
||||
private nodes: Array<Node | void> = [];
|
||||
private readonly nodeCallbacks: Array<NodeCallback> = [];
|
||||
private readonly elementListeners: Map<number, Array<ElementListener>> = new Map();
|
||||
private nodes: Array<Node | void> = []
|
||||
private readonly nodeCallbacks: Array<NodeCallback> = []
|
||||
private readonly elementListeners: Map<number, Array<ElementListener>> = new Map()
|
||||
|
||||
constructor(private readonly node_id: string) {}
|
||||
|
||||
attachNodeCallback(nodeCallback: NodeCallback): void {
|
||||
this.nodeCallbacks.push(nodeCallback);
|
||||
this.nodeCallbacks.push(nodeCallback)
|
||||
}
|
||||
attachElementListener(type: string, node: Element, elementListener: EventListener): void {
|
||||
const id = this.getID(node);
|
||||
const id = this.getID(node)
|
||||
if (id === undefined) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
node.addEventListener(type, elementListener);
|
||||
let listeners = this.elementListeners.get(id);
|
||||
node.addEventListener(type, elementListener)
|
||||
let listeners = this.elementListeners.get(id)
|
||||
if (listeners === undefined) {
|
||||
listeners = [];
|
||||
this.elementListeners.set(id, listeners);
|
||||
return;
|
||||
listeners = []
|
||||
this.elementListeners.set(id, listeners)
|
||||
return
|
||||
}
|
||||
listeners.push([type, elementListener]);
|
||||
listeners.push([type, elementListener])
|
||||
}
|
||||
|
||||
registerNode(node: Node): [id: number, isNew: boolean] {
|
||||
let id: number = (node as any)[this.node_id];
|
||||
const isNew = id === undefined;
|
||||
let id: number = (node as any)[this.node_id]
|
||||
const isNew = id === undefined
|
||||
if (isNew) {
|
||||
id = this.nodes.length;
|
||||
this.nodes[id] = node;
|
||||
(node as any)[this.node_id] = id;
|
||||
id = this.nodes.length
|
||||
this.nodes[id] = node
|
||||
;(node as any)[this.node_id] = id
|
||||
}
|
||||
return [id, isNew];
|
||||
return [id, isNew]
|
||||
}
|
||||
unregisterNode(node: Node): number | undefined {
|
||||
const id = (node as any)[this.node_id];
|
||||
const id = (node as any)[this.node_id]
|
||||
if (id !== undefined) {
|
||||
delete (node as any)[this.node_id];
|
||||
delete this.nodes[id];
|
||||
const listeners = this.elementListeners.get(id);
|
||||
delete (node as any)[this.node_id]
|
||||
delete this.nodes[id]
|
||||
const listeners = this.elementListeners.get(id)
|
||||
if (listeners !== undefined) {
|
||||
this.elementListeners.delete(id);
|
||||
listeners.forEach((listener) => node.removeEventListener(listener[0], listener[1]));
|
||||
this.elementListeners.delete(id)
|
||||
listeners.forEach((listener) => node.removeEventListener(listener[0], listener[1]))
|
||||
}
|
||||
}
|
||||
return id;
|
||||
return id
|
||||
}
|
||||
cleanTree() {
|
||||
// sadly we keep empty items in array here resulting in some memory still being used
|
||||
|
|
@ -55,30 +55,30 @@ export default class Nodes {
|
|||
// plus we keep our index positions for new/alive nodes
|
||||
// performance test: 3ms for 30k nodes with 17k dead ones
|
||||
for (let i = 0; i < this.nodes.length; i++) {
|
||||
const node = this.nodes[i];
|
||||
const node = this.nodes[i]
|
||||
if (node && !document.contains(node)) {
|
||||
this.unregisterNode(node);
|
||||
this.unregisterNode(node)
|
||||
}
|
||||
}
|
||||
}
|
||||
callNodeCallbacks(node: Node, isStart: boolean): void {
|
||||
this.nodeCallbacks.forEach((cb) => cb(node, isStart));
|
||||
this.nodeCallbacks.forEach((cb) => cb(node, isStart))
|
||||
}
|
||||
getID(node: Node): number | undefined {
|
||||
return (node as any)[this.node_id];
|
||||
return (node as any)[this.node_id]
|
||||
}
|
||||
getNode(id: number) {
|
||||
return this.nodes[id];
|
||||
return this.nodes[id]
|
||||
}
|
||||
|
||||
clear(): void {
|
||||
for (let id = 0; id < this.nodes.length; id++) {
|
||||
const node = this.nodes[id];
|
||||
const node = this.nodes[id]
|
||||
if (node === undefined) {
|
||||
continue;
|
||||
continue
|
||||
}
|
||||
this.unregisterNode(node);
|
||||
this.unregisterNode(node)
|
||||
}
|
||||
this.nodes.length = 0;
|
||||
this.nodes.length = 0
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,20 +1,20 @@
|
|||
import Observer from './observer.js';
|
||||
import { CreateIFrameDocument } from '../../../common/messages.js';
|
||||
import Observer from './observer.js'
|
||||
import { CreateIFrameDocument } from '../messages.gen.js'
|
||||
|
||||
export default class IFrameObserver extends Observer {
|
||||
observe(iframe: HTMLIFrameElement) {
|
||||
const doc = iframe.contentDocument;
|
||||
const hostID = this.app.nodes.getID(iframe);
|
||||
const doc = iframe.contentDocument
|
||||
const hostID = this.app.nodes.getID(iframe)
|
||||
if (!doc || hostID === undefined) {
|
||||
return;
|
||||
return
|
||||
} //log TODO common app.logger
|
||||
// Have to observe document, because the inner <html> might be changed
|
||||
this.observeRoot(doc, (docID) => {
|
||||
if (docID === undefined) {
|
||||
console.log('OpenReplay: Iframe document not bound');
|
||||
return;
|
||||
console.log('OpenReplay: Iframe document not bound')
|
||||
return
|
||||
}
|
||||
this.app.send(CreateIFrameDocument(hostID, docID));
|
||||
});
|
||||
this.app.send(CreateIFrameDocument(hostID, docID))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,33 +8,33 @@ import {
|
|||
CreateElementNode,
|
||||
MoveNode,
|
||||
RemoveNode,
|
||||
} from '../../../common/messages.js';
|
||||
import App from '../index.js';
|
||||
import { isRootNode, isTextNode, isElementNode, isSVGElement, hasTag } from '../guards.js';
|
||||
} from '../messages.gen.js'
|
||||
import App from '../index.js'
|
||||
import { isRootNode, isTextNode, isElementNode, isSVGElement, hasTag } from '../guards.js'
|
||||
|
||||
function isIgnored(node: Node): boolean {
|
||||
if (isTextNode(node)) {
|
||||
return false;
|
||||
return false
|
||||
}
|
||||
if (!isElementNode(node)) {
|
||||
return true;
|
||||
return true
|
||||
}
|
||||
const tag = node.tagName.toUpperCase();
|
||||
const tag = node.tagName.toUpperCase()
|
||||
if (tag === 'LINK') {
|
||||
const rel = node.getAttribute('rel');
|
||||
const as = node.getAttribute('as');
|
||||
return !(rel?.includes('stylesheet') || as === 'style' || as === 'font');
|
||||
const rel = node.getAttribute('rel')
|
||||
const as = node.getAttribute('as')
|
||||
return !(rel?.includes('stylesheet') || as === 'style' || as === 'font')
|
||||
}
|
||||
return (
|
||||
tag === 'SCRIPT' || tag === 'NOSCRIPT' || tag === 'META' || tag === 'TITLE' || tag === 'BASE'
|
||||
);
|
||||
)
|
||||
}
|
||||
|
||||
function isObservable(node: Node): boolean {
|
||||
if (isRootNode(node)) {
|
||||
return true;
|
||||
return true
|
||||
}
|
||||
return !isIgnored(node);
|
||||
return !isIgnored(node)
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
@ -57,84 +57,84 @@ enum RecentsType {
|
|||
}
|
||||
|
||||
export default abstract class Observer {
|
||||
private readonly observer: MutationObserver;
|
||||
private readonly commited: Array<boolean | undefined> = [];
|
||||
private readonly recents: Map<number, RecentsType> = new Map();
|
||||
private readonly indexes: Array<number> = [];
|
||||
private readonly attributesMap: Map<number, Set<string>> = new Map();
|
||||
private readonly textSet: Set<number> = new Set();
|
||||
private readonly observer: MutationObserver
|
||||
private readonly commited: Array<boolean | undefined> = []
|
||||
private readonly recents: Map<number, RecentsType> = new Map()
|
||||
private readonly indexes: Array<number> = []
|
||||
private readonly attributesMap: Map<number, Set<string>> = new Map()
|
||||
private readonly textSet: Set<number> = new Set()
|
||||
constructor(protected readonly app: App, protected readonly isTopContext = false) {
|
||||
this.observer = new MutationObserver(
|
||||
this.app.safe((mutations) => {
|
||||
for (const mutation of mutations) {
|
||||
// mutations order is sequential
|
||||
const target = mutation.target;
|
||||
const type = mutation.type;
|
||||
const target = mutation.target
|
||||
const type = mutation.type
|
||||
|
||||
if (!isObservable(target)) {
|
||||
continue;
|
||||
continue
|
||||
}
|
||||
if (type === 'childList') {
|
||||
for (let i = 0; i < mutation.removedNodes.length; i++) {
|
||||
this.bindTree(mutation.removedNodes[i], true);
|
||||
this.bindTree(mutation.removedNodes[i], true)
|
||||
}
|
||||
for (let i = 0; i < mutation.addedNodes.length; i++) {
|
||||
this.bindTree(mutation.addedNodes[i]);
|
||||
this.bindTree(mutation.addedNodes[i])
|
||||
}
|
||||
continue;
|
||||
continue
|
||||
}
|
||||
const id = this.app.nodes.getID(target);
|
||||
const id = this.app.nodes.getID(target)
|
||||
if (id === undefined) {
|
||||
continue;
|
||||
continue
|
||||
}
|
||||
if (!this.recents.has(id)) {
|
||||
this.recents.set(id, RecentsType.Changed); // TODO only when altered
|
||||
this.recents.set(id, RecentsType.Changed) // TODO only when altered
|
||||
}
|
||||
if (type === 'attributes') {
|
||||
const name = mutation.attributeName;
|
||||
const name = mutation.attributeName
|
||||
if (name === null) {
|
||||
continue;
|
||||
continue
|
||||
}
|
||||
let attr = this.attributesMap.get(id);
|
||||
let attr = this.attributesMap.get(id)
|
||||
if (attr === undefined) {
|
||||
this.attributesMap.set(id, (attr = new Set()));
|
||||
this.attributesMap.set(id, (attr = new Set()))
|
||||
}
|
||||
attr.add(name);
|
||||
continue;
|
||||
attr.add(name)
|
||||
continue
|
||||
}
|
||||
if (type === 'characterData') {
|
||||
this.textSet.add(id);
|
||||
continue;
|
||||
this.textSet.add(id)
|
||||
continue
|
||||
}
|
||||
}
|
||||
this.commitNodes();
|
||||
this.commitNodes()
|
||||
}),
|
||||
);
|
||||
)
|
||||
}
|
||||
private clear(): void {
|
||||
this.commited.length = 0;
|
||||
this.recents.clear();
|
||||
this.indexes.length = 1;
|
||||
this.attributesMap.clear();
|
||||
this.textSet.clear();
|
||||
this.commited.length = 0
|
||||
this.recents.clear()
|
||||
this.indexes.length = 1
|
||||
this.attributesMap.clear()
|
||||
this.textSet.clear()
|
||||
}
|
||||
|
||||
private sendNodeAttribute(id: number, node: Element, name: string, value: string | null): void {
|
||||
if (isSVGElement(node)) {
|
||||
if (name.substr(0, 6) === 'xlink:') {
|
||||
name = name.substr(6);
|
||||
name = name.substr(6)
|
||||
}
|
||||
if (value === null) {
|
||||
this.app.send(new RemoveNodeAttribute(id, name));
|
||||
this.app.send(RemoveNodeAttribute(id, name))
|
||||
} else if (name === 'href') {
|
||||
if (value.length > 1e5) {
|
||||
value = '';
|
||||
value = ''
|
||||
}
|
||||
this.app.send(new SetNodeAttributeURLBased(id, name, value, this.app.getBaseHref()));
|
||||
this.app.send(SetNodeAttributeURLBased(id, name, value, this.app.getBaseHref()))
|
||||
} else {
|
||||
this.app.send(new SetNodeAttribute(id, name, value));
|
||||
this.app.send(SetNodeAttribute(id, name, value))
|
||||
}
|
||||
return;
|
||||
return
|
||||
}
|
||||
if (
|
||||
name === 'src' ||
|
||||
|
|
@ -144,7 +144,7 @@ export default abstract class Observer {
|
|||
name === 'autocomplete' ||
|
||||
name.substr(0, 2) === 'on'
|
||||
) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
if (
|
||||
name === 'value' &&
|
||||
|
|
@ -153,50 +153,50 @@ export default abstract class Observer {
|
|||
node.type !== 'reset' &&
|
||||
node.type !== 'submit'
|
||||
) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
if (value === null) {
|
||||
this.app.send(new RemoveNodeAttribute(id, name));
|
||||
return;
|
||||
this.app.send(RemoveNodeAttribute(id, name))
|
||||
return
|
||||
}
|
||||
if (name === 'style' || (name === 'href' && hasTag(node, 'LINK'))) {
|
||||
this.app.send(new SetNodeAttributeURLBased(id, name, value, this.app.getBaseHref()));
|
||||
return;
|
||||
this.app.send(SetNodeAttributeURLBased(id, name, value, this.app.getBaseHref()))
|
||||
return
|
||||
}
|
||||
if (name === 'href' || value.length > 1e5) {
|
||||
value = '';
|
||||
value = ''
|
||||
}
|
||||
this.app.send(new SetNodeAttribute(id, name, value));
|
||||
this.app.send(SetNodeAttribute(id, name, value))
|
||||
}
|
||||
|
||||
private sendNodeData(id: number, parentElement: Element, data: string): void {
|
||||
if (hasTag(parentElement, 'STYLE') || hasTag(parentElement, 'style')) {
|
||||
this.app.send(new SetCSSDataURLBased(id, data, this.app.getBaseHref()));
|
||||
return;
|
||||
this.app.send(SetCSSDataURLBased(id, data, this.app.getBaseHref()))
|
||||
return
|
||||
}
|
||||
data = this.app.sanitizer.sanitize(id, data);
|
||||
this.app.send(new SetNodeData(id, data));
|
||||
data = this.app.sanitizer.sanitize(id, data)
|
||||
this.app.send(SetNodeData(id, data))
|
||||
}
|
||||
|
||||
private bindNode(node: Node): void {
|
||||
const [id, isNew] = this.app.nodes.registerNode(node);
|
||||
const [id, isNew] = this.app.nodes.registerNode(node)
|
||||
if (isNew) {
|
||||
this.recents.set(id, RecentsType.New);
|
||||
this.recents.set(id, RecentsType.New)
|
||||
} else if (this.recents.get(id) !== RecentsType.New) {
|
||||
// can we do just `else` here?
|
||||
this.recents.set(id, RecentsType.Removed);
|
||||
this.recents.set(id, RecentsType.Removed)
|
||||
}
|
||||
}
|
||||
private unbindChildNode(node: Node): void {
|
||||
const [id] = this.app.nodes.registerNode(node);
|
||||
this.recents.set(id, RecentsType.RemovedChild);
|
||||
const [id] = this.app.nodes.registerNode(node)
|
||||
this.recents.set(id, RecentsType.RemovedChild)
|
||||
}
|
||||
|
||||
private bindTree(node: Node, isChildUnbinding = false): void {
|
||||
if (!isObservable(node)) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
this.bindNode(node);
|
||||
this.bindNode(node)
|
||||
const walker = document.createTreeWalker(
|
||||
node,
|
||||
NodeFilter.SHOW_ELEMENT + NodeFilter.SHOW_TEXT,
|
||||
|
|
@ -208,30 +208,30 @@ export default abstract class Observer {
|
|||
},
|
||||
// @ts-ignore
|
||||
false,
|
||||
);
|
||||
)
|
||||
while (walker.nextNode()) {
|
||||
if (isChildUnbinding) {
|
||||
this.unbindChildNode(walker.currentNode);
|
||||
this.unbindChildNode(walker.currentNode)
|
||||
} else {
|
||||
this.bindNode(walker.currentNode);
|
||||
this.bindNode(walker.currentNode)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private unbindNode(node: Node) {
|
||||
const id = this.app.nodes.unregisterNode(node);
|
||||
const id = this.app.nodes.unregisterNode(node)
|
||||
if (id !== undefined && this.recents.get(id) === RecentsType.Removed) {
|
||||
this.app.send(new RemoveNode(id));
|
||||
this.app.send(RemoveNode(id))
|
||||
}
|
||||
}
|
||||
|
||||
// A top-consumption function on the infinite lists test. (~1% of performance resources)
|
||||
private _commitNode(id: number, node: Node): boolean {
|
||||
if (isRootNode(node)) {
|
||||
return true;
|
||||
return true
|
||||
}
|
||||
const parent = node.parentNode;
|
||||
let parentID: number | undefined;
|
||||
const parent = node.parentNode
|
||||
let parentID: number | undefined
|
||||
|
||||
// Disable parent check for the upper context HTMLHtmlElement, because it is root there... (before)
|
||||
// TODO: get rid of "special" cases (there is an issue with CreateDocument altered behaviour though)
|
||||
|
|
@ -240,109 +240,109 @@ export default abstract class Observer {
|
|||
if (parent === null) {
|
||||
// Sometimes one observation contains attribute mutations for the removimg node, which gets ignored here.
|
||||
// That shouldn't affect the visual rendering ( should it? )
|
||||
this.unbindNode(node);
|
||||
return false;
|
||||
this.unbindNode(node)
|
||||
return false
|
||||
}
|
||||
parentID = this.app.nodes.getID(parent);
|
||||
parentID = this.app.nodes.getID(parent)
|
||||
if (parentID === undefined) {
|
||||
this.unbindNode(node);
|
||||
return false;
|
||||
this.unbindNode(node)
|
||||
return false
|
||||
}
|
||||
if (!this.commitNode(parentID)) {
|
||||
this.unbindNode(node);
|
||||
return false;
|
||||
this.unbindNode(node)
|
||||
return false
|
||||
}
|
||||
this.app.sanitizer.handleNode(id, parentID, node);
|
||||
this.app.sanitizer.handleNode(id, parentID, node)
|
||||
if (this.app.sanitizer.isMaskedContainer(parentID)) {
|
||||
return false;
|
||||
return false
|
||||
}
|
||||
}
|
||||
// From here parentID === undefined if node is top context HTML node
|
||||
let sibling = node.previousSibling;
|
||||
let sibling = node.previousSibling
|
||||
while (sibling !== null) {
|
||||
const siblingID = this.app.nodes.getID(sibling);
|
||||
const siblingID = this.app.nodes.getID(sibling)
|
||||
if (siblingID !== undefined) {
|
||||
this.commitNode(siblingID);
|
||||
this.indexes[id] = this.indexes[siblingID] + 1;
|
||||
break;
|
||||
this.commitNode(siblingID)
|
||||
this.indexes[id] = this.indexes[siblingID] + 1
|
||||
break
|
||||
}
|
||||
sibling = sibling.previousSibling;
|
||||
sibling = sibling.previousSibling
|
||||
}
|
||||
if (sibling === null) {
|
||||
this.indexes[id] = 0;
|
||||
this.indexes[id] = 0
|
||||
}
|
||||
const recentsType = this.recents.get(id);
|
||||
const isNew = recentsType === RecentsType.New;
|
||||
const index = this.indexes[id];
|
||||
const recentsType = this.recents.get(id)
|
||||
const isNew = recentsType === RecentsType.New
|
||||
const index = this.indexes[id]
|
||||
if (index === undefined) {
|
||||
throw 'commitNode: missing node index';
|
||||
throw 'commitNode: missing node index'
|
||||
}
|
||||
if (isNew) {
|
||||
if (isElementNode(node)) {
|
||||
let el: Element = node;
|
||||
let el: Element = node
|
||||
if (parentID !== undefined) {
|
||||
if (this.app.sanitizer.isMaskedContainer(id)) {
|
||||
const width = el.clientWidth;
|
||||
const height = el.clientHeight;
|
||||
el = node.cloneNode() as Element;
|
||||
(el as HTMLElement | SVGElement).style.width = width + 'px';
|
||||
(el as HTMLElement | SVGElement).style.height = height + 'px';
|
||||
const width = el.clientWidth
|
||||
const height = el.clientHeight
|
||||
el = node.cloneNode() as Element
|
||||
;(el as HTMLElement | SVGElement).style.width = width + 'px'
|
||||
;(el as HTMLElement | SVGElement).style.height = height + 'px'
|
||||
}
|
||||
|
||||
this.app.send(new CreateElementNode(id, parentID, index, el.tagName, isSVGElement(node)));
|
||||
this.app.send(CreateElementNode(id, parentID, index, el.tagName, isSVGElement(node)))
|
||||
}
|
||||
for (let i = 0; i < el.attributes.length; i++) {
|
||||
const attr = el.attributes[i];
|
||||
this.sendNodeAttribute(id, el, attr.nodeName, attr.value);
|
||||
const attr = el.attributes[i]
|
||||
this.sendNodeAttribute(id, el, attr.nodeName, attr.value)
|
||||
}
|
||||
} else if (isTextNode(node)) {
|
||||
// for text node id != 0, hence parentID !== undefined and parent is Element
|
||||
this.app.send(new CreateTextNode(id, parentID as number, index));
|
||||
this.sendNodeData(id, parent as Element, node.data);
|
||||
this.app.send(CreateTextNode(id, parentID as number, index))
|
||||
this.sendNodeData(id, parent as Element, node.data)
|
||||
}
|
||||
return true;
|
||||
return true
|
||||
}
|
||||
if (recentsType === RecentsType.Removed && parentID !== undefined) {
|
||||
this.app.send(new MoveNode(id, parentID, index));
|
||||
this.app.send(MoveNode(id, parentID, index))
|
||||
}
|
||||
const attr = this.attributesMap.get(id);
|
||||
const attr = this.attributesMap.get(id)
|
||||
if (attr !== undefined) {
|
||||
if (!isElementNode(node)) {
|
||||
throw 'commitNode: node is not an element';
|
||||
throw 'commitNode: node is not an element'
|
||||
}
|
||||
for (const name of attr) {
|
||||
this.sendNodeAttribute(id, node, name, node.getAttribute(name));
|
||||
this.sendNodeAttribute(id, node, name, node.getAttribute(name))
|
||||
}
|
||||
}
|
||||
if (this.textSet.has(id)) {
|
||||
if (!isTextNode(node)) {
|
||||
throw 'commitNode: node is not a text';
|
||||
throw 'commitNode: node is not a text'
|
||||
}
|
||||
// for text node id != 0, hence parent is Element
|
||||
this.sendNodeData(id, parent as Element, node.data);
|
||||
this.sendNodeData(id, parent as Element, node.data)
|
||||
}
|
||||
return true;
|
||||
return true
|
||||
}
|
||||
private commitNode(id: number): boolean {
|
||||
const node = this.app.nodes.getNode(id);
|
||||
const node = this.app.nodes.getNode(id)
|
||||
if (node === undefined) {
|
||||
return false;
|
||||
return false
|
||||
}
|
||||
const cmt = this.commited[id];
|
||||
const cmt = this.commited[id]
|
||||
if (cmt !== undefined) {
|
||||
return cmt;
|
||||
return cmt
|
||||
}
|
||||
return (this.commited[id] = this._commitNode(id, node));
|
||||
return (this.commited[id] = this._commitNode(id, node))
|
||||
}
|
||||
private commitNodes(isStart = false): void {
|
||||
let node;
|
||||
let node
|
||||
this.recents.forEach((type, id) => {
|
||||
this.commitNode(id);
|
||||
this.commitNode(id)
|
||||
if (type === RecentsType.New && (node = this.app.nodes.getNode(id))) {
|
||||
this.app.nodes.callNodeCallbacks(node, isStart);
|
||||
this.app.nodes.callNodeCallbacks(node, isStart)
|
||||
}
|
||||
});
|
||||
this.clear();
|
||||
})
|
||||
this.clear()
|
||||
}
|
||||
|
||||
// ISSSUE
|
||||
|
|
@ -358,14 +358,14 @@ export default abstract class Observer {
|
|||
subtree: true,
|
||||
attributeOldValue: false,
|
||||
characterDataOldValue: false,
|
||||
});
|
||||
this.bindTree(nodeToBind);
|
||||
beforeCommit(this.app.nodes.getID(node));
|
||||
this.commitNodes(true);
|
||||
})
|
||||
this.bindTree(nodeToBind)
|
||||
beforeCommit(this.app.nodes.getID(node))
|
||||
this.commitNodes(true)
|
||||
}
|
||||
|
||||
disconnect(): void {
|
||||
this.observer.disconnect();
|
||||
this.clear();
|
||||
this.observer.disconnect()
|
||||
this.clear()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,19 +1,19 @@
|
|||
import Observer from './observer.js';
|
||||
import { CreateIFrameDocument } from '../../../common/messages.js';
|
||||
import Observer from './observer.js'
|
||||
import { CreateIFrameDocument } from '../messages.gen.js'
|
||||
|
||||
export default class ShadowRootObserver extends Observer {
|
||||
observe(el: Element) {
|
||||
const shRoot = el.shadowRoot;
|
||||
const hostID = this.app.nodes.getID(el);
|
||||
const shRoot = el.shadowRoot
|
||||
const hostID = this.app.nodes.getID(el)
|
||||
if (!shRoot || hostID === undefined) {
|
||||
return;
|
||||
return
|
||||
} // log
|
||||
this.observeRoot(shRoot, (rootID) => {
|
||||
if (rootID === undefined) {
|
||||
console.log('OpenReplay: Shadow Root was not bound');
|
||||
return;
|
||||
console.log('OpenReplay: Shadow Root was not bound')
|
||||
return
|
||||
}
|
||||
this.app.send(CreateIFrameDocument(hostID, rootID));
|
||||
});
|
||||
this.app.send(CreateIFrameDocument(hostID, rootID))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,29 +1,29 @@
|
|||
import Observer from './observer.js';
|
||||
import { isElementNode, hasTag } from '../guards.js';
|
||||
import Observer from './observer.js'
|
||||
import { isElementNode, hasTag } from '../guards.js'
|
||||
|
||||
import IFrameObserver from './iframe_observer.js';
|
||||
import ShadowRootObserver from './shadow_root_observer.js';
|
||||
import IFrameObserver from './iframe_observer.js'
|
||||
import ShadowRootObserver from './shadow_root_observer.js'
|
||||
|
||||
import { CreateDocument } from '../../../common/messages.js';
|
||||
import App from '../index.js';
|
||||
import { IN_BROWSER, hasOpenreplayAttribute } from '../../utils.js';
|
||||
import { CreateDocument } from '../messages.gen.js'
|
||||
import App from '../index.js'
|
||||
import { IN_BROWSER, hasOpenreplayAttribute } from '../../utils.js'
|
||||
|
||||
export interface Options {
|
||||
captureIFrames: boolean;
|
||||
captureIFrames: boolean
|
||||
}
|
||||
|
||||
const attachShadowNativeFn = IN_BROWSER ? Element.prototype.attachShadow : () => new ShadowRoot();
|
||||
const attachShadowNativeFn = IN_BROWSER ? Element.prototype.attachShadow : () => new ShadowRoot()
|
||||
|
||||
export default class TopObserver extends Observer {
|
||||
private readonly options: Options;
|
||||
private readonly options: Options
|
||||
constructor(app: App, options: Partial<Options>) {
|
||||
super(app, true);
|
||||
super(app, true)
|
||||
this.options = Object.assign(
|
||||
{
|
||||
captureIFrames: true,
|
||||
},
|
||||
options,
|
||||
);
|
||||
)
|
||||
|
||||
// IFrames
|
||||
this.app.nodes.attachNodeCallback((node) => {
|
||||
|
|
@ -32,59 +32,59 @@ export default class TopObserver extends Observer {
|
|||
((this.options.captureIFrames && !hasOpenreplayAttribute(node, 'obscured')) ||
|
||||
hasOpenreplayAttribute(node, 'capture'))
|
||||
) {
|
||||
this.handleIframe(node);
|
||||
this.handleIframe(node)
|
||||
}
|
||||
});
|
||||
})
|
||||
|
||||
// ShadowDOM
|
||||
this.app.nodes.attachNodeCallback((node) => {
|
||||
if (isElementNode(node) && node.shadowRoot !== null) {
|
||||
this.handleShadowRoot(node.shadowRoot);
|
||||
this.handleShadowRoot(node.shadowRoot)
|
||||
}
|
||||
});
|
||||
})
|
||||
}
|
||||
|
||||
private iframeObservers: IFrameObserver[] = [];
|
||||
private iframeObservers: IFrameObserver[] = []
|
||||
private handleIframe(iframe: HTMLIFrameElement): void {
|
||||
let doc: Document | null = null;
|
||||
let doc: Document | null = null
|
||||
const handle = this.app.safe(() => {
|
||||
const id = this.app.nodes.getID(iframe);
|
||||
const id = this.app.nodes.getID(iframe)
|
||||
if (id === undefined) {
|
||||
return;
|
||||
return
|
||||
} //log
|
||||
if (iframe.contentDocument === doc) {
|
||||
return;
|
||||
return
|
||||
} // How frequently can it happen?
|
||||
doc = iframe.contentDocument;
|
||||
doc = iframe.contentDocument
|
||||
if (!doc || !iframe.contentWindow) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
const observer = new IFrameObserver(this.app);
|
||||
const observer = new IFrameObserver(this.app)
|
||||
|
||||
this.iframeObservers.push(observer);
|
||||
observer.observe(iframe);
|
||||
});
|
||||
iframe.addEventListener('load', handle); // why app.attachEventListener not working?
|
||||
handle();
|
||||
this.iframeObservers.push(observer)
|
||||
observer.observe(iframe)
|
||||
})
|
||||
iframe.addEventListener('load', handle) // why app.attachEventListener not working?
|
||||
handle()
|
||||
}
|
||||
|
||||
private shadowRootObservers: ShadowRootObserver[] = [];
|
||||
private shadowRootObservers: ShadowRootObserver[] = []
|
||||
private handleShadowRoot(shRoot: ShadowRoot) {
|
||||
const observer = new ShadowRootObserver(this.app);
|
||||
this.shadowRootObservers.push(observer);
|
||||
observer.observe(shRoot.host);
|
||||
const observer = new ShadowRootObserver(this.app)
|
||||
this.shadowRootObservers.push(observer)
|
||||
observer.observe(shRoot.host)
|
||||
}
|
||||
|
||||
observe(): void {
|
||||
// Protection from several subsequent calls?
|
||||
|
||||
const observer = this;
|
||||
const observer = this
|
||||
Element.prototype.attachShadow = function () {
|
||||
// eslint-disable-next-line
|
||||
const shadow = attachShadowNativeFn.apply(this, arguments);
|
||||
observer.handleShadowRoot(shadow);
|
||||
return shadow;
|
||||
};
|
||||
const shadow = attachShadowNativeFn.apply(this, arguments)
|
||||
observer.handleShadowRoot(shadow)
|
||||
return shadow
|
||||
}
|
||||
|
||||
// Can observe documentElement (<html>) here, because it is not supposed to be changing.
|
||||
// However, it is possible in some exotic cases and may cause an ignorance of the newly created <html>
|
||||
|
|
@ -95,18 +95,18 @@ export default class TopObserver extends Observer {
|
|||
this.observeRoot(
|
||||
window.document,
|
||||
() => {
|
||||
this.app.send(new CreateDocument());
|
||||
this.app.send(CreateDocument())
|
||||
},
|
||||
window.document.documentElement,
|
||||
);
|
||||
)
|
||||
}
|
||||
|
||||
disconnect() {
|
||||
Element.prototype.attachShadow = attachShadowNativeFn;
|
||||
this.iframeObservers.forEach((o) => o.disconnect());
|
||||
this.iframeObservers = [];
|
||||
this.shadowRootObservers.forEach((o) => o.disconnect());
|
||||
this.shadowRootObservers = [];
|
||||
super.disconnect();
|
||||
Element.prototype.attachShadow = attachShadowNativeFn
|
||||
this.iframeObservers.forEach((o) => o.disconnect())
|
||||
this.iframeObservers = []
|
||||
this.shadowRootObservers.forEach((o) => o.disconnect())
|
||||
this.shadowRootObservers = []
|
||||
super.disconnect()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,16 +1,16 @@
|
|||
import type App from './index.js';
|
||||
import { stars, hasOpenreplayAttribute } from '../utils.js';
|
||||
import { isElementNode } from './guards.js';
|
||||
import type App from './index.js'
|
||||
import { stars, hasOpenreplayAttribute } from '../utils.js'
|
||||
import { isElementNode } from './guards.js'
|
||||
|
||||
export interface Options {
|
||||
obscureTextEmails: boolean;
|
||||
obscureTextNumbers: boolean;
|
||||
obscureTextEmails: boolean
|
||||
obscureTextNumbers: boolean
|
||||
}
|
||||
|
||||
export default class Sanitizer {
|
||||
private readonly masked: Set<number> = new Set();
|
||||
private readonly maskedContainers: Set<number> = new Set();
|
||||
private readonly options: Options;
|
||||
private readonly masked: Set<number> = new Set()
|
||||
private readonly maskedContainers: Set<number> = new Set()
|
||||
private readonly options: Options
|
||||
|
||||
constructor(private readonly app: App, options: Partial<Options>) {
|
||||
this.options = Object.assign(
|
||||
|
|
@ -19,7 +19,7 @@ export default class Sanitizer {
|
|||
obscureTextNumbers: false,
|
||||
},
|
||||
options,
|
||||
);
|
||||
)
|
||||
}
|
||||
|
||||
handleNode(id: number, parentID: number, node: Node) {
|
||||
|
|
@ -27,13 +27,13 @@ export default class Sanitizer {
|
|||
this.masked.has(parentID) ||
|
||||
(isElementNode(node) && hasOpenreplayAttribute(node, 'masked'))
|
||||
) {
|
||||
this.masked.add(id);
|
||||
this.masked.add(id)
|
||||
}
|
||||
if (
|
||||
this.maskedContainers.has(parentID) ||
|
||||
(isElementNode(node) && hasOpenreplayAttribute(node, 'htmlmasked'))
|
||||
) {
|
||||
this.maskedContainers.add(id);
|
||||
this.maskedContainers.add(id)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -42,40 +42,37 @@ export default class Sanitizer {
|
|||
// TODO: is it the best place to put trim() ? Might trimmed spaces be considered in layout in certain cases?
|
||||
return data
|
||||
.trim()
|
||||
.replace(
|
||||
/[^\f\n\r\t\v\u00a0\u1680\u2000-\u200a\u2028\u2029\u202f\u205f\u3000\ufeff]/g,
|
||||
'█',
|
||||
);
|
||||
.replace(/[^\f\n\r\t\v\u00a0\u1680\u2000-\u200a\u2028\u2029\u202f\u205f\u3000\ufeff]/g, '█')
|
||||
}
|
||||
if (this.options.obscureTextNumbers) {
|
||||
data = data.replace(/\d/g, '0');
|
||||
data = data.replace(/\d/g, '0')
|
||||
}
|
||||
if (this.options.obscureTextEmails) {
|
||||
data = data.replace(
|
||||
/([^\s]+)@([^\s]+)\.([^\s]+)/g,
|
||||
(...f: Array<string>) => stars(f[1]) + '@' + stars(f[2]) + '.' + stars(f[3]),
|
||||
);
|
||||
)
|
||||
}
|
||||
return data;
|
||||
return data
|
||||
}
|
||||
|
||||
isMasked(id: number): boolean {
|
||||
return this.masked.has(id);
|
||||
return this.masked.has(id)
|
||||
}
|
||||
isMaskedContainer(id: number) {
|
||||
return this.maskedContainers.has(id);
|
||||
return this.maskedContainers.has(id)
|
||||
}
|
||||
|
||||
getInnerTextSecure(el: HTMLElement): string {
|
||||
const id = this.app.nodes.getID(el);
|
||||
const id = this.app.nodes.getID(el)
|
||||
if (!id) {
|
||||
return '';
|
||||
return ''
|
||||
}
|
||||
return this.sanitize(id, el.innerText);
|
||||
return this.sanitize(id, el.innerText)
|
||||
}
|
||||
|
||||
clear(): void {
|
||||
this.masked.clear();
|
||||
this.maskedContainers.clear();
|
||||
this.masked.clear()
|
||||
this.maskedContainers.clear()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,52 +1,50 @@
|
|||
import { UserID, UserAnonymousID, Metadata } from '../../common/messages.js';
|
||||
|
||||
interface SessionInfo {
|
||||
sessionID: string | null;
|
||||
metadata: Record<string, string>;
|
||||
userID: string | null;
|
||||
sessionID: string | null
|
||||
metadata: Record<string, string>
|
||||
userID: string | null
|
||||
}
|
||||
type OnUpdateCallback = (i: Partial<SessionInfo>) => void;
|
||||
type OnUpdateCallback = (i: Partial<SessionInfo>) => void
|
||||
|
||||
export default class Session {
|
||||
private metadata: Record<string, string> = {};
|
||||
private userID: string | null = null;
|
||||
private sessionID: string | null = null;
|
||||
private readonly callbacks: OnUpdateCallback[] = [];
|
||||
private metadata: Record<string, string> = {}
|
||||
private userID: string | null = null
|
||||
private sessionID: string | null = null
|
||||
private readonly callbacks: OnUpdateCallback[] = []
|
||||
|
||||
attachUpdateCallback(cb: OnUpdateCallback) {
|
||||
this.callbacks.push(cb);
|
||||
this.callbacks.push(cb)
|
||||
}
|
||||
private handleUpdate(newInfo: Partial<SessionInfo>) {
|
||||
if (newInfo.userID == null) {
|
||||
delete newInfo.userID;
|
||||
delete newInfo.userID
|
||||
}
|
||||
if (newInfo.sessionID == null) {
|
||||
delete newInfo.sessionID;
|
||||
delete newInfo.sessionID
|
||||
}
|
||||
this.callbacks.forEach((cb) => cb(newInfo));
|
||||
this.callbacks.forEach((cb) => cb(newInfo))
|
||||
}
|
||||
|
||||
update(newInfo: Partial<SessionInfo>) {
|
||||
if (newInfo.userID !== undefined) {
|
||||
// TODO clear nullable/undefinable types
|
||||
this.userID = newInfo.userID;
|
||||
this.userID = newInfo.userID
|
||||
}
|
||||
if (newInfo.metadata !== undefined) {
|
||||
Object.entries(newInfo.metadata).forEach(([k, v]) => (this.metadata[k] = v));
|
||||
Object.entries(newInfo.metadata).forEach(([k, v]) => (this.metadata[k] = v))
|
||||
}
|
||||
if (newInfo.sessionID !== undefined) {
|
||||
this.sessionID = newInfo.sessionID;
|
||||
this.sessionID = newInfo.sessionID
|
||||
}
|
||||
this.handleUpdate(newInfo);
|
||||
this.handleUpdate(newInfo)
|
||||
}
|
||||
|
||||
setMetadata(key: string, value: string) {
|
||||
this.metadata[key] = value;
|
||||
this.handleUpdate({ metadata: { [key]: value } });
|
||||
this.metadata[key] = value
|
||||
this.handleUpdate({ metadata: { [key]: value } })
|
||||
}
|
||||
setUserID(userID: string) {
|
||||
this.userID = userID;
|
||||
this.handleUpdate({ userID });
|
||||
this.userID = userID
|
||||
this.handleUpdate({ userID })
|
||||
}
|
||||
|
||||
getInfo(): SessionInfo {
|
||||
|
|
@ -54,12 +52,12 @@ export default class Session {
|
|||
sessionID: this.sessionID,
|
||||
metadata: this.metadata,
|
||||
userID: this.userID,
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
reset(): void {
|
||||
this.metadata = {};
|
||||
this.userID = null;
|
||||
this.sessionID = null;
|
||||
this.metadata = {}
|
||||
this.userID = null
|
||||
this.sessionID = null
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,31 +1,31 @@
|
|||
import App from './index.js';
|
||||
import App from './index.js'
|
||||
|
||||
type Callback = () => void;
|
||||
type Callback = () => void
|
||||
function wrap(callback: Callback, n: number): Callback {
|
||||
let t = 0;
|
||||
let t = 0
|
||||
return (): void => {
|
||||
if (t++ >= n) {
|
||||
t = 0;
|
||||
callback();
|
||||
t = 0
|
||||
callback()
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
export default class Ticker {
|
||||
private timer: ReturnType<typeof setInterval> | null = null;
|
||||
private readonly callbacks: Array<Callback | undefined>;
|
||||
private timer: ReturnType<typeof setInterval> | null = null
|
||||
private readonly callbacks: Array<Callback | undefined>
|
||||
constructor(private readonly app: App) {
|
||||
this.callbacks = [];
|
||||
this.callbacks = []
|
||||
}
|
||||
|
||||
attach(callback: Callback, n = 0, useSafe = true, thisArg?: any) {
|
||||
if (thisArg) {
|
||||
callback = callback.bind(thisArg);
|
||||
callback = callback.bind(thisArg)
|
||||
}
|
||||
if (useSafe) {
|
||||
callback = this.app.safe(callback);
|
||||
callback = this.app.safe(callback)
|
||||
}
|
||||
this.callbacks.unshift(n ? wrap(callback, n) : callback) - 1;
|
||||
this.callbacks.unshift(n ? wrap(callback, n) : callback) - 1
|
||||
}
|
||||
|
||||
start(): void {
|
||||
|
|
@ -33,17 +33,17 @@ export default class Ticker {
|
|||
this.timer = setInterval(
|
||||
() =>
|
||||
this.callbacks.forEach((cb) => {
|
||||
if (cb) cb();
|
||||
if (cb) cb()
|
||||
}),
|
||||
30,
|
||||
);
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
stop(): void {
|
||||
if (this.timer !== null) {
|
||||
clearInterval(this.timer);
|
||||
this.timer = null;
|
||||
clearInterval(this.timer)
|
||||
this.timer = null
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,62 +1,56 @@
|
|||
import App, { DEFAULT_INGEST_POINT } from './app/index.js';
|
||||
export { default as App } from './app/index.js';
|
||||
import App, { DEFAULT_INGEST_POINT } from './app/index.js'
|
||||
export { default as App } from './app/index.js'
|
||||
|
||||
import {
|
||||
UserID,
|
||||
UserAnonymousID,
|
||||
Metadata,
|
||||
RawCustomEvent,
|
||||
CustomIssue,
|
||||
} from '../common/messages.js';
|
||||
import * as _Messages from '../common/messages.js';
|
||||
export const Messages = _Messages;
|
||||
import { UserID, UserAnonymousID, RawCustomEvent, CustomIssue } from './app/messages.gen.js'
|
||||
import * as _Messages from './app/messages.gen.js'
|
||||
export const Messages = _Messages
|
||||
|
||||
import Connection from './modules/connection.js';
|
||||
import Console from './modules/console.js';
|
||||
import Connection from './modules/connection.js'
|
||||
import Console from './modules/console.js'
|
||||
import Exception, {
|
||||
getExceptionMessageFromEvent,
|
||||
getExceptionMessage,
|
||||
} from './modules/exception.js';
|
||||
import Img from './modules/img.js';
|
||||
import Input from './modules/input.js';
|
||||
import Mouse from './modules/mouse.js';
|
||||
import Timing from './modules/timing.js';
|
||||
import Performance from './modules/performance.js';
|
||||
import Scroll from './modules/scroll.js';
|
||||
import Viewport from './modules/viewport.js';
|
||||
import CSSRules from './modules/cssrules.js';
|
||||
import { IN_BROWSER, deprecationWarn, DOCS_HOST } from './utils.js';
|
||||
} from './modules/exception.js'
|
||||
import Img from './modules/img.js'
|
||||
import Input from './modules/input.js'
|
||||
import Mouse from './modules/mouse.js'
|
||||
import Timing from './modules/timing.js'
|
||||
import Performance from './modules/performance.js'
|
||||
import Scroll from './modules/scroll.js'
|
||||
import Viewport from './modules/viewport.js'
|
||||
import CSSRules from './modules/cssrules.js'
|
||||
import { IN_BROWSER, deprecationWarn, DOCS_HOST } from './utils.js'
|
||||
|
||||
import type { Options as AppOptions } from './app/index.js';
|
||||
import type { Options as ConsoleOptions } from './modules/console.js';
|
||||
import type { Options as ExceptionOptions } from './modules/exception.js';
|
||||
import type { Options as InputOptions } from './modules/input.js';
|
||||
import type { Options as PerformanceOptions } from './modules/performance.js';
|
||||
import type { Options as TimingOptions } from './modules/timing.js';
|
||||
import type { StartOptions } from './app/index.js';
|
||||
import type { Options as AppOptions } from './app/index.js'
|
||||
import type { Options as ConsoleOptions } from './modules/console.js'
|
||||
import type { Options as ExceptionOptions } from './modules/exception.js'
|
||||
import type { Options as InputOptions } from './modules/input.js'
|
||||
import type { Options as PerformanceOptions } from './modules/performance.js'
|
||||
import type { Options as TimingOptions } from './modules/timing.js'
|
||||
import type { StartOptions } from './app/index.js'
|
||||
//TODO: unique options init
|
||||
import type { StartPromiseReturn } from './app/index.js';
|
||||
import type { StartPromiseReturn } from './app/index.js'
|
||||
|
||||
export type Options = Partial<
|
||||
AppOptions & ConsoleOptions & ExceptionOptions & InputOptions & PerformanceOptions & TimingOptions
|
||||
> & {
|
||||
projectID?: number; // For the back compatibility only (deprecated)
|
||||
projectKey: string;
|
||||
sessionToken?: string;
|
||||
respectDoNotTrack?: boolean;
|
||||
autoResetOnWindowOpen?: boolean;
|
||||
projectID?: number // For the back compatibility only (deprecated)
|
||||
projectKey: string
|
||||
sessionToken?: string
|
||||
respectDoNotTrack?: boolean
|
||||
autoResetOnWindowOpen?: boolean
|
||||
// dev only
|
||||
__DISABLE_SECURE_MODE?: boolean;
|
||||
};
|
||||
__DISABLE_SECURE_MODE?: boolean
|
||||
}
|
||||
|
||||
const DOCS_SETUP = '/installation/setup-or';
|
||||
const DOCS_SETUP = '/installation/setup-or'
|
||||
|
||||
function processOptions(obj: any): obj is Options {
|
||||
if (obj == null) {
|
||||
console.error(
|
||||
`OpenReplay: invalid options argument type. Please, check documentation on ${DOCS_HOST}${DOCS_SETUP}`,
|
||||
);
|
||||
return false;
|
||||
)
|
||||
return false
|
||||
}
|
||||
if (typeof obj.projectKey !== 'string') {
|
||||
if (typeof obj.projectKey !== 'number') {
|
||||
|
|
@ -64,46 +58,46 @@ function processOptions(obj: any): obj is Options {
|
|||
// Back compatability
|
||||
console.error(
|
||||
`OpenReplay: projectKey is missing or wrong type (string is expected). Please, check ${DOCS_HOST}${DOCS_SETUP} for more information.`,
|
||||
);
|
||||
return false;
|
||||
)
|
||||
return false
|
||||
} else {
|
||||
obj.projectKey = obj.projectID.toString();
|
||||
deprecationWarn('`projectID` option', '`projectKey` option', DOCS_SETUP);
|
||||
obj.projectKey = obj.projectID.toString()
|
||||
deprecationWarn('`projectID` option', '`projectKey` option', DOCS_SETUP)
|
||||
}
|
||||
} else {
|
||||
console.warn('OpenReplay: projectKey is expected to have a string type.');
|
||||
obj.projectKey = obj.projectKey.toString();
|
||||
console.warn('OpenReplay: projectKey is expected to have a string type.')
|
||||
obj.projectKey = obj.projectKey.toString()
|
||||
}
|
||||
}
|
||||
if (typeof obj.sessionToken !== 'string' && obj.sessionToken != null) {
|
||||
console.warn(
|
||||
`OpenReplay: invalid options argument type. Please, check documentation on ${DOCS_HOST}${DOCS_SETUP}`,
|
||||
);
|
||||
)
|
||||
}
|
||||
return true;
|
||||
return true
|
||||
}
|
||||
|
||||
export default class API {
|
||||
private readonly app: App | null = null;
|
||||
private readonly app: App | null = null
|
||||
constructor(private readonly options: Options) {
|
||||
if (!IN_BROWSER || !processOptions(options)) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
if ((window as any).__OPENREPLAY__) {
|
||||
console.error('OpenReplay: one tracker instance has been initialised already');
|
||||
return;
|
||||
console.error('OpenReplay: one tracker instance has been initialised already')
|
||||
return
|
||||
}
|
||||
if (!options.__DISABLE_SECURE_MODE && location.protocol !== 'https:') {
|
||||
console.error(
|
||||
'OpenReplay: Your website must be publicly accessible and running on SSL in order for OpenReplay to properly capture and replay the user session. You can disable this check by setting `__DISABLE_SECURE_MODE` option to `true` if you are testing in localhost. Keep in mind, that asset files on a local machine are not available to the outside world. This might affect tracking if you use css files.',
|
||||
);
|
||||
return;
|
||||
)
|
||||
return
|
||||
}
|
||||
const doNotTrack =
|
||||
options.respectDoNotTrack &&
|
||||
(navigator.doNotTrack == '1' ||
|
||||
// @ts-ignore
|
||||
window.doNotTrack == '1');
|
||||
window.doNotTrack == '1')
|
||||
const app = (this.app =
|
||||
doNotTrack ||
|
||||
!('Map' in window) ||
|
||||
|
|
@ -115,42 +109,42 @@ export default class API {
|
|||
!('Blob' in window) ||
|
||||
!('Worker' in window)
|
||||
? null
|
||||
: new App(options.projectKey, options.sessionToken, options));
|
||||
: new App(options.projectKey, options.sessionToken, options))
|
||||
if (app !== null) {
|
||||
Viewport(app);
|
||||
CSSRules(app);
|
||||
Connection(app);
|
||||
Console(app, options);
|
||||
Exception(app, options);
|
||||
Img(app);
|
||||
Input(app, options);
|
||||
Mouse(app);
|
||||
Timing(app, options);
|
||||
Performance(app, options);
|
||||
Scroll(app);
|
||||
(window as any).__OPENREPLAY__ = this;
|
||||
Viewport(app)
|
||||
CSSRules(app)
|
||||
Connection(app)
|
||||
Console(app, options)
|
||||
Exception(app, options)
|
||||
Img(app)
|
||||
Input(app, options)
|
||||
Mouse(app)
|
||||
Timing(app, options)
|
||||
Performance(app, options)
|
||||
Scroll(app)
|
||||
;(window as any).__OPENREPLAY__ = this
|
||||
|
||||
if (options.autoResetOnWindowOpen) {
|
||||
const wOpen = window.open;
|
||||
const wOpen = window.open
|
||||
app.attachStartCallback(() => {
|
||||
// @ts-ignore ?
|
||||
window.open = function (...args) {
|
||||
app.resetNextPageSession(true);
|
||||
wOpen.call(window, ...args);
|
||||
app.resetNextPageSession(false);
|
||||
};
|
||||
});
|
||||
app.resetNextPageSession(true)
|
||||
wOpen.call(window, ...args)
|
||||
app.resetNextPageSession(false)
|
||||
}
|
||||
})
|
||||
app.attachStopCallback(() => {
|
||||
window.open = wOpen;
|
||||
});
|
||||
window.open = wOpen
|
||||
})
|
||||
}
|
||||
} else {
|
||||
console.log(
|
||||
"OpenReplay: browser doesn't support API required for tracking or doNotTrack is set to 1.",
|
||||
);
|
||||
const req = new XMLHttpRequest();
|
||||
const orig = options.ingestPoint || DEFAULT_INGEST_POINT;
|
||||
req.open('POST', orig + '/v1/web/not-started');
|
||||
)
|
||||
const req = new XMLHttpRequest()
|
||||
const orig = options.ingestPoint || DEFAULT_INGEST_POINT
|
||||
req.open('POST', orig + '/v1/web/not-started')
|
||||
// no-cors issue only with text/plain or not-set Content-Type
|
||||
// req.setRequestHeader("Content-Type", "application/json;charset=UTF-8");
|
||||
req.send(
|
||||
|
|
@ -160,99 +154,99 @@ export default class API {
|
|||
doNotTrack,
|
||||
// TODO: add precise reason (an exact API missing)
|
||||
}),
|
||||
);
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
use<T>(fn: (app: App | null, options?: Options) => T): T {
|
||||
return fn(this.app, this.options);
|
||||
return fn(this.app, this.options)
|
||||
}
|
||||
|
||||
isActive(): boolean {
|
||||
if (this.app === null) {
|
||||
return false;
|
||||
return false
|
||||
}
|
||||
return this.app.active();
|
||||
return this.app.active()
|
||||
}
|
||||
|
||||
start(startOpts?: Partial<StartOptions>): Promise<StartPromiseReturn> {
|
||||
if (!IN_BROWSER) {
|
||||
console.error(
|
||||
`OpenReplay: you are trying to start Tracker on a node.js environment. If you want to use OpenReplay with SSR, please, use componentDidMount or useEffect API for placing the \`tracker.start()\` line. Check documentation on ${DOCS_HOST}${DOCS_SETUP}`,
|
||||
);
|
||||
return Promise.reject('Trying to start not in browser.');
|
||||
)
|
||||
return Promise.reject('Trying to start not in browser.')
|
||||
}
|
||||
if (this.app === null) {
|
||||
return Promise.reject("Browser doesn't support required api, or doNotTrack is active.");
|
||||
return Promise.reject("Browser doesn't support required api, or doNotTrack is active.")
|
||||
}
|
||||
// TODO: check argument type
|
||||
return this.app.start(startOpts);
|
||||
return this.app.start(startOpts)
|
||||
}
|
||||
stop(): void {
|
||||
if (this.app === null) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
this.app.stop(true);
|
||||
this.app.stop(true)
|
||||
}
|
||||
|
||||
getSessionToken(): string | null | undefined {
|
||||
if (this.app === null) {
|
||||
return null;
|
||||
return null
|
||||
}
|
||||
return this.app.getSessionToken();
|
||||
return this.app.getSessionToken()
|
||||
}
|
||||
getSessionID(): string | null | undefined {
|
||||
if (this.app === null) {
|
||||
return null;
|
||||
return null
|
||||
}
|
||||
return this.app.getSessionID();
|
||||
return this.app.getSessionID()
|
||||
}
|
||||
sessionID(): string | null | undefined {
|
||||
deprecationWarn("'sessionID' method", "'getSessionID' method", '/');
|
||||
return this.getSessionID();
|
||||
deprecationWarn("'sessionID' method", "'getSessionID' method", '/')
|
||||
return this.getSessionID()
|
||||
}
|
||||
|
||||
setUserID(id: string): void {
|
||||
if (typeof id === 'string' && this.app !== null) {
|
||||
this.app.session.setUserID(id);
|
||||
this.app.session.setUserID(id)
|
||||
}
|
||||
}
|
||||
userID(id: string): void {
|
||||
deprecationWarn("'userID' method", "'setUserID' method", '/');
|
||||
this.setUserID(id);
|
||||
deprecationWarn("'userID' method", "'setUserID' method", '/')
|
||||
this.setUserID(id)
|
||||
}
|
||||
|
||||
setUserAnonymousID(id: string): void {
|
||||
if (typeof id === 'string' && this.app !== null) {
|
||||
this.app.send(new UserAnonymousID(id));
|
||||
this.app.send(UserAnonymousID(id))
|
||||
}
|
||||
}
|
||||
userAnonymousID(id: string): void {
|
||||
deprecationWarn("'userAnonymousID' method", "'setUserAnonymousID' method", '/');
|
||||
this.setUserAnonymousID(id);
|
||||
deprecationWarn("'userAnonymousID' method", "'setUserAnonymousID' method", '/')
|
||||
this.setUserAnonymousID(id)
|
||||
}
|
||||
|
||||
setMetadata(key: string, value: string): void {
|
||||
if (typeof key === 'string' && typeof value === 'string' && this.app !== null) {
|
||||
this.app.session.setMetadata(key, value);
|
||||
this.app.session.setMetadata(key, value)
|
||||
}
|
||||
}
|
||||
metadata(key: string, value: string): void {
|
||||
deprecationWarn("'metadata' method", "'setMetadata' method", '/');
|
||||
this.setMetadata(key, value);
|
||||
deprecationWarn("'metadata' method", "'setMetadata' method", '/')
|
||||
this.setMetadata(key, value)
|
||||
}
|
||||
|
||||
event(key: string, payload: any, issue = false): void {
|
||||
if (typeof key === 'string' && this.app !== null) {
|
||||
if (issue) {
|
||||
return this.issue(key, payload);
|
||||
return this.issue(key, payload)
|
||||
} else {
|
||||
try {
|
||||
payload = JSON.stringify(payload);
|
||||
payload = JSON.stringify(payload)
|
||||
} catch (e) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
this.app.send(new RawCustomEvent(key, payload));
|
||||
this.app.send(RawCustomEvent(key, payload))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -260,28 +254,28 @@ export default class API {
|
|||
issue(key: string, payload: any): void {
|
||||
if (typeof key === 'string' && this.app !== null) {
|
||||
try {
|
||||
payload = JSON.stringify(payload);
|
||||
payload = JSON.stringify(payload)
|
||||
} catch (e) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
this.app.send(new CustomIssue(key, payload));
|
||||
this.app.send(CustomIssue(key, payload))
|
||||
}
|
||||
}
|
||||
|
||||
handleError = (e: Error | ErrorEvent | PromiseRejectionEvent) => {
|
||||
if (this.app === null) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
if (e instanceof Error) {
|
||||
this.app.send(getExceptionMessage(e, []));
|
||||
this.app.send(getExceptionMessage(e, []))
|
||||
} else if (
|
||||
e instanceof ErrorEvent ||
|
||||
('PromiseRejectionEvent' in window && e instanceof PromiseRejectionEvent)
|
||||
) {
|
||||
const msg = getExceptionMessageFromEvent(e);
|
||||
const msg = getExceptionMessageFromEvent(e)
|
||||
if (msg != null) {
|
||||
this.app.send(msg);
|
||||
this.app.send(msg)
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,28 +1,25 @@
|
|||
import App from '../app/index.js';
|
||||
import { ConnectionInformation } from '../../common/messages.js';
|
||||
import App from '../app/index.js'
|
||||
import { ConnectionInformation } from '../app/messages.gen.js'
|
||||
|
||||
export default function (app: App): void {
|
||||
const connection:
|
||||
| {
|
||||
downlink: number;
|
||||
type?: string;
|
||||
addEventListener: (type: 'change', cb: () => void) => void;
|
||||
downlink: number
|
||||
type?: string
|
||||
addEventListener: (type: 'change', cb: () => void) => void
|
||||
}
|
||||
| undefined =
|
||||
(navigator as any).connection ||
|
||||
(navigator as any).mozConnection ||
|
||||
(navigator as any).webkitConnection;
|
||||
(navigator as any).webkitConnection
|
||||
if (connection === undefined) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
|
||||
const sendConnectionInformation = (): void =>
|
||||
app.send(
|
||||
new ConnectionInformation(
|
||||
Math.round(connection.downlink * 1000),
|
||||
connection.type || 'unknown',
|
||||
),
|
||||
);
|
||||
sendConnectionInformation();
|
||||
connection.addEventListener('change', sendConnectionInformation);
|
||||
ConnectionInformation(Math.round(connection.downlink * 1000), connection.type || 'unknown'),
|
||||
)
|
||||
sendConnectionInformation()
|
||||
connection.addEventListener('change', sendConnectionInformation)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,100 +1,100 @@
|
|||
import type App from '../app/index.js';
|
||||
import { hasTag } from '../app/guards.js';
|
||||
import { IN_BROWSER } from '../utils.js';
|
||||
import { ConsoleLog } from '../../common/messages.js';
|
||||
import type App from '../app/index.js'
|
||||
import { hasTag } from '../app/guards.js'
|
||||
import { IN_BROWSER } from '../utils.js'
|
||||
import { ConsoleLog } from '../app/messages.gen.js'
|
||||
|
||||
const printError: (e: Error) => string =
|
||||
IN_BROWSER && 'InstallTrigger' in window // detect Firefox
|
||||
? (e: Error): string => e.message + '\n' + e.stack
|
||||
: (e: Error): string => e.stack || e.message;
|
||||
: (e: Error): string => e.stack || e.message
|
||||
|
||||
function printString(arg: any): string {
|
||||
if (arg === undefined) {
|
||||
return 'undefined';
|
||||
return 'undefined'
|
||||
}
|
||||
if (arg === null) {
|
||||
return 'null';
|
||||
return 'null'
|
||||
}
|
||||
if (arg instanceof Error) {
|
||||
return printError(arg);
|
||||
return printError(arg)
|
||||
}
|
||||
if (Array.isArray(arg)) {
|
||||
return `Array(${arg.length})`;
|
||||
return `Array(${arg.length})`
|
||||
}
|
||||
return String(arg);
|
||||
return String(arg)
|
||||
}
|
||||
|
||||
function printFloat(arg: any): string {
|
||||
if (typeof arg !== 'number') return 'NaN';
|
||||
return arg.toString();
|
||||
if (typeof arg !== 'number') return 'NaN'
|
||||
return arg.toString()
|
||||
}
|
||||
|
||||
function printInt(arg: any): string {
|
||||
if (typeof arg !== 'number') return 'NaN';
|
||||
return Math.floor(arg).toString();
|
||||
if (typeof arg !== 'number') return 'NaN'
|
||||
return Math.floor(arg).toString()
|
||||
}
|
||||
|
||||
function printObject(arg: any): string {
|
||||
if (arg === undefined) {
|
||||
return 'undefined';
|
||||
return 'undefined'
|
||||
}
|
||||
if (arg === null) {
|
||||
return 'null';
|
||||
return 'null'
|
||||
}
|
||||
if (arg instanceof Error) {
|
||||
return printError(arg);
|
||||
return printError(arg)
|
||||
}
|
||||
if (Array.isArray(arg)) {
|
||||
const length = arg.length;
|
||||
const values = arg.slice(0, 10).map(printString).join(', ');
|
||||
return `Array(${length})[${values}]`;
|
||||
const length = arg.length
|
||||
const values = arg.slice(0, 10).map(printString).join(', ')
|
||||
return `Array(${length})[${values}]`
|
||||
}
|
||||
if (typeof arg === 'object') {
|
||||
const res = [];
|
||||
let i = 0;
|
||||
const res = []
|
||||
let i = 0
|
||||
for (const k in arg) {
|
||||
if (++i === 10) {
|
||||
break;
|
||||
break
|
||||
}
|
||||
const v = arg[k];
|
||||
res.push(k + ': ' + printString(v));
|
||||
const v = arg[k]
|
||||
res.push(k + ': ' + printString(v))
|
||||
}
|
||||
return '{' + res.join(', ') + '}';
|
||||
return '{' + res.join(', ') + '}'
|
||||
}
|
||||
return arg.toString();
|
||||
return arg.toString()
|
||||
}
|
||||
|
||||
function printf(args: any[]): string {
|
||||
if (typeof args[0] === 'string') {
|
||||
args.unshift(
|
||||
args.shift().replace(/%(o|s|f|d|i)/g, (s: string, t: string): string => {
|
||||
const arg = args.shift();
|
||||
if (arg === undefined) return s;
|
||||
const arg = args.shift()
|
||||
if (arg === undefined) return s
|
||||
switch (t) {
|
||||
case 'o':
|
||||
return printObject(arg);
|
||||
return printObject(arg)
|
||||
case 's':
|
||||
return printString(arg);
|
||||
return printString(arg)
|
||||
case 'f':
|
||||
return printFloat(arg);
|
||||
return printFloat(arg)
|
||||
case 'd':
|
||||
case 'i':
|
||||
return printInt(arg);
|
||||
return printInt(arg)
|
||||
default:
|
||||
return s;
|
||||
return s
|
||||
}
|
||||
}),
|
||||
);
|
||||
)
|
||||
}
|
||||
return args.map(printObject).join(' ');
|
||||
return args.map(printObject).join(' ')
|
||||
}
|
||||
|
||||
export interface Options {
|
||||
consoleMethods: Array<string> | null;
|
||||
consoleThrottling: number;
|
||||
consoleMethods: Array<string> | null
|
||||
consoleThrottling: number
|
||||
}
|
||||
|
||||
const consoleMethods = ['log', 'info', 'warn', 'error', 'debug', 'assert'];
|
||||
const consoleMethods = ['log', 'info', 'warn', 'error', 'debug', 'assert']
|
||||
|
||||
export default function (app: App, opts: Partial<Options>): void {
|
||||
const options: Options = Object.assign(
|
||||
|
|
@ -103,54 +103,54 @@ export default function (app: App, opts: Partial<Options>): void {
|
|||
consoleThrottling: 30,
|
||||
},
|
||||
opts,
|
||||
);
|
||||
)
|
||||
if (!Array.isArray(options.consoleMethods) || options.consoleMethods.length === 0) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
|
||||
const sendConsoleLog = app.safe((level: string, args: unknown[]): void =>
|
||||
app.send(new ConsoleLog(level, printf(args))),
|
||||
);
|
||||
app.send(ConsoleLog(level, printf(args))),
|
||||
)
|
||||
|
||||
let n: number;
|
||||
let n: number
|
||||
const reset = (): void => {
|
||||
n = 0;
|
||||
};
|
||||
app.attachStartCallback(reset);
|
||||
app.ticker.attach(reset, 33, false);
|
||||
n = 0
|
||||
}
|
||||
app.attachStartCallback(reset)
|
||||
app.ticker.attach(reset, 33, false)
|
||||
|
||||
const patchConsole = (console: Console) =>
|
||||
options.consoleMethods!.forEach((method) => {
|
||||
if (consoleMethods.indexOf(method) === -1) {
|
||||
console.error(`OpenReplay: unsupported console method "${method}"`);
|
||||
return;
|
||||
console.error(`OpenReplay: unsupported console method "${method}"`)
|
||||
return
|
||||
}
|
||||
const fn = (console as any)[method];
|
||||
(console as any)[method] = function (...args: unknown[]): void {
|
||||
fn.apply(this, args);
|
||||
const fn = (console as any)[method]
|
||||
;(console as any)[method] = function (...args: unknown[]): void {
|
||||
fn.apply(this, args)
|
||||
if (n++ > options.consoleThrottling) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
sendConsoleLog(method, args);
|
||||
};
|
||||
});
|
||||
patchConsole(window.console);
|
||||
sendConsoleLog(method, args)
|
||||
}
|
||||
})
|
||||
patchConsole(window.console)
|
||||
|
||||
app.nodes.attachNodeCallback(
|
||||
app.safe((node) => {
|
||||
if (hasTag(node, 'IFRAME')) {
|
||||
// TODO: newContextCallback
|
||||
let context = node.contentWindow;
|
||||
let context = node.contentWindow
|
||||
if (context) {
|
||||
patchConsole((context as Window & typeof globalThis).console);
|
||||
patchConsole((context as Window & typeof globalThis).console)
|
||||
}
|
||||
app.attachEventListener(node, 'load', () => {
|
||||
if (node.contentWindow !== context) {
|
||||
context = node.contentWindow;
|
||||
patchConsole((context as Window & typeof globalThis).console);
|
||||
context = node.contentWindow
|
||||
patchConsole((context as Window & typeof globalThis).console)
|
||||
}
|
||||
});
|
||||
})
|
||||
}
|
||||
}),
|
||||
);
|
||||
)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,53 +1,53 @@
|
|||
import type App from '../app/index.js';
|
||||
import { CSSInsertRuleURLBased, CSSDeleteRule, TechnicalInfo } from '../../common/messages.js';
|
||||
import { hasTag } from '../app/guards.js';
|
||||
import type App from '../app/index.js'
|
||||
import { CSSInsertRuleURLBased, CSSDeleteRule, TechnicalInfo } from '../app/messages.gen.js'
|
||||
import { hasTag } from '../app/guards.js'
|
||||
|
||||
export default function (app: App | null) {
|
||||
if (app === null) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
if (!window.CSSStyleSheet) {
|
||||
app.send(new TechnicalInfo('no_stylesheet_prototype_in_window', ''));
|
||||
return;
|
||||
app.send(TechnicalInfo('no_stylesheet_prototype_in_window', ''))
|
||||
return
|
||||
}
|
||||
|
||||
const processOperation = app.safe((stylesheet: CSSStyleSheet, index: number, rule?: string) => {
|
||||
const sendMessage =
|
||||
typeof rule === 'string'
|
||||
? (nodeID: number) =>
|
||||
app.send(new CSSInsertRuleURLBased(nodeID, rule, index, app.getBaseHref()))
|
||||
: (nodeID: number) => app.send(new CSSDeleteRule(nodeID, index));
|
||||
app.send(CSSInsertRuleURLBased(nodeID, rule, index, app.getBaseHref()))
|
||||
: (nodeID: number) => app.send(CSSDeleteRule(nodeID, index))
|
||||
// TODO: Extend messages to maintain nested rules (CSSGroupingRule prototype, as well as CSSKeyframesRule)
|
||||
if (stylesheet.ownerNode == null) {
|
||||
throw new Error('Owner Node not found');
|
||||
throw new Error('Owner Node not found')
|
||||
}
|
||||
const nodeID = app.nodes.getID(stylesheet.ownerNode);
|
||||
const nodeID = app.nodes.getID(stylesheet.ownerNode)
|
||||
if (nodeID !== undefined) {
|
||||
sendMessage(nodeID);
|
||||
sendMessage(nodeID)
|
||||
} // else error?
|
||||
});
|
||||
})
|
||||
|
||||
const { insertRule, deleteRule } = CSSStyleSheet.prototype;
|
||||
const { insertRule, deleteRule } = CSSStyleSheet.prototype
|
||||
|
||||
CSSStyleSheet.prototype.insertRule = function (rule: string, index = 0) {
|
||||
processOperation(this, index, rule);
|
||||
return insertRule.call(this, rule, index);
|
||||
};
|
||||
processOperation(this, index, rule)
|
||||
return insertRule.call(this, rule, index)
|
||||
}
|
||||
CSSStyleSheet.prototype.deleteRule = function (index: number) {
|
||||
processOperation(this, index);
|
||||
return deleteRule.call(this, index);
|
||||
};
|
||||
processOperation(this, index)
|
||||
return deleteRule.call(this, index)
|
||||
}
|
||||
|
||||
app.nodes.attachNodeCallback((node: Node): void => {
|
||||
if (!hasTag(node, 'STYLE') || !node.sheet) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
if (node.textContent !== null && node.textContent.trim().length > 0) {
|
||||
return; // Only fully virtual sheets maintained so far
|
||||
return // Only fully virtual sheets maintained so far
|
||||
}
|
||||
const rules = node.sheet.cssRules;
|
||||
const rules = node.sheet.cssRules
|
||||
for (let i = 0; i < rules.length; i++) {
|
||||
processOperation(node.sheet, i, rules[i].cssText);
|
||||
processOperation(node.sheet, i, rules[i].cssText)
|
||||
}
|
||||
});
|
||||
})
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,18 +1,18 @@
|
|||
import type App from '../app/index.js';
|
||||
import type Message from '../../common/messages.js';
|
||||
import { JSException } from '../../common/messages.js';
|
||||
import ErrorStackParser from 'error-stack-parser';
|
||||
import type App from '../app/index.js'
|
||||
import type Message from '../app/messages.gen.js'
|
||||
import { JSException } from '../app/messages.gen.js'
|
||||
import ErrorStackParser from 'error-stack-parser'
|
||||
|
||||
export interface Options {
|
||||
captureExceptions: boolean;
|
||||
captureExceptions: boolean
|
||||
}
|
||||
|
||||
interface StackFrame {
|
||||
columnNumber?: number;
|
||||
lineNumber?: number;
|
||||
fileName?: string;
|
||||
functionName?: string;
|
||||
source?: string;
|
||||
columnNumber?: number
|
||||
lineNumber?: number
|
||||
fileName?: string
|
||||
functionName?: string
|
||||
source?: string
|
||||
}
|
||||
|
||||
function getDefaultStack(e: ErrorEvent): Array<StackFrame> {
|
||||
|
|
@ -24,15 +24,15 @@ function getDefaultStack(e: ErrorEvent): Array<StackFrame> {
|
|||
functionName: '',
|
||||
source: '',
|
||||
},
|
||||
];
|
||||
]
|
||||
}
|
||||
|
||||
export function getExceptionMessage(error: Error, fallbackStack: Array<StackFrame>): Message {
|
||||
let stack = fallbackStack;
|
||||
let stack = fallbackStack
|
||||
try {
|
||||
stack = ErrorStackParser.parse(error);
|
||||
stack = ErrorStackParser.parse(error)
|
||||
} catch (e) {}
|
||||
return new JSException(error.name, error.message, JSON.stringify(stack));
|
||||
return JSException(error.name, error.message, JSON.stringify(stack))
|
||||
}
|
||||
|
||||
export function getExceptionMessageFromEvent(
|
||||
|
|
@ -40,29 +40,29 @@ export function getExceptionMessageFromEvent(
|
|||
): Message | null {
|
||||
if (e instanceof ErrorEvent) {
|
||||
if (e.error instanceof Error) {
|
||||
return getExceptionMessage(e.error, getDefaultStack(e));
|
||||
return getExceptionMessage(e.error, getDefaultStack(e))
|
||||
} else {
|
||||
let [name, message] = e.message.split(':');
|
||||
let [name, message] = e.message.split(':')
|
||||
if (!message) {
|
||||
name = 'Error';
|
||||
message = e.message;
|
||||
name = 'Error'
|
||||
message = e.message
|
||||
}
|
||||
return new JSException(name, message, JSON.stringify(getDefaultStack(e)));
|
||||
return JSException(name, message, JSON.stringify(getDefaultStack(e)))
|
||||
}
|
||||
} else if ('PromiseRejectionEvent' in window && e instanceof PromiseRejectionEvent) {
|
||||
if (e.reason instanceof Error) {
|
||||
return getExceptionMessage(e.reason, []);
|
||||
return getExceptionMessage(e.reason, [])
|
||||
} else {
|
||||
let message: string;
|
||||
let message: string
|
||||
try {
|
||||
message = JSON.stringify(e.reason);
|
||||
message = JSON.stringify(e.reason)
|
||||
} catch (_) {
|
||||
message = String(e.reason);
|
||||
message = String(e.reason)
|
||||
}
|
||||
return new JSException('Unhandled Promise Rejection', message, '[]');
|
||||
return JSException('Unhandled Promise Rejection', message, '[]')
|
||||
}
|
||||
}
|
||||
return null;
|
||||
return null
|
||||
}
|
||||
|
||||
export default function (app: App, opts: Partial<Options>): void {
|
||||
|
|
@ -71,18 +71,18 @@ export default function (app: App, opts: Partial<Options>): void {
|
|||
captureExceptions: true,
|
||||
},
|
||||
opts,
|
||||
);
|
||||
)
|
||||
if (options.captureExceptions) {
|
||||
const handler = (e: ErrorEvent | PromiseRejectionEvent): void => {
|
||||
const msg = getExceptionMessageFromEvent(e);
|
||||
const msg = getExceptionMessageFromEvent(e)
|
||||
if (msg != null) {
|
||||
app.send(msg);
|
||||
app.send(msg)
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
app.attachEventListener(window, 'unhandledrejection', (e: PromiseRejectionEvent): void =>
|
||||
handler(e),
|
||||
);
|
||||
app.attachEventListener(window, 'error', (e: ErrorEvent): void => handler(e));
|
||||
)
|
||||
app.attachEventListener(window, 'error', (e: ErrorEvent): void => handler(e))
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,96 +1,92 @@
|
|||
import type App from '../app/index.js';
|
||||
import { timestamp, isURL } from '../utils.js';
|
||||
import {
|
||||
ResourceTiming,
|
||||
SetNodeAttributeURLBased,
|
||||
SetNodeAttribute,
|
||||
} from '../../common/messages.js';
|
||||
import { hasTag } from '../app/guards.js';
|
||||
import type App from '../app/index.js'
|
||||
import { timestamp, isURL } from '../utils.js'
|
||||
import { ResourceTiming, SetNodeAttributeURLBased, SetNodeAttribute } from '../app/messages.gen.js'
|
||||
import { hasTag } from '../app/guards.js'
|
||||
|
||||
function resolveURL(url: string, location: Location = document.location) {
|
||||
url = url.trim();
|
||||
url = url.trim()
|
||||
if (url.startsWith('/')) {
|
||||
return location.origin + url;
|
||||
return location.origin + url
|
||||
} else if (
|
||||
url.startsWith('http://') ||
|
||||
url.startsWith('https://') ||
|
||||
url.startsWith('data:') // any other possible value here?
|
||||
) {
|
||||
return url;
|
||||
return url
|
||||
} else {
|
||||
return location.origin + location.pathname + url;
|
||||
return location.origin + location.pathname + url
|
||||
}
|
||||
}
|
||||
|
||||
const PLACEHOLDER_SRC = 'https://static.openreplay.com/tracker/placeholder.jpeg';
|
||||
const PLACEHOLDER_SRC = 'https://static.openreplay.com/tracker/placeholder.jpeg'
|
||||
|
||||
export default function (app: App): void {
|
||||
function sendPlaceholder(id: number, node: HTMLImageElement): void {
|
||||
app.send(new SetNodeAttribute(id, 'src', PLACEHOLDER_SRC));
|
||||
const { width, height } = node.getBoundingClientRect();
|
||||
app.send(SetNodeAttribute(id, 'src', PLACEHOLDER_SRC))
|
||||
const { width, height } = node.getBoundingClientRect()
|
||||
if (!node.hasAttribute('width')) {
|
||||
app.send(new SetNodeAttribute(id, 'width', String(width)));
|
||||
app.send(SetNodeAttribute(id, 'width', String(width)))
|
||||
}
|
||||
if (!node.hasAttribute('height')) {
|
||||
app.send(new SetNodeAttribute(id, 'height', String(height)));
|
||||
app.send(SetNodeAttribute(id, 'height', String(height)))
|
||||
}
|
||||
}
|
||||
|
||||
const sendImgSrc = app.safe(function (this: HTMLImageElement): void {
|
||||
const id = app.nodes.getID(this);
|
||||
const id = app.nodes.getID(this)
|
||||
if (id === undefined) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
const { src, complete, naturalWidth, naturalHeight, srcset } = this;
|
||||
const { src, complete, naturalWidth, naturalHeight, srcset } = this
|
||||
if (!complete) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
const resolvedSrc = resolveURL(src || ''); // Src type is null sometimes. - is it true?
|
||||
const resolvedSrc = resolveURL(src || '') // Src type is null sometimes. - is it true?
|
||||
if (naturalWidth === 0 && naturalHeight === 0) {
|
||||
if (isURL(resolvedSrc)) {
|
||||
app.send(new ResourceTiming(timestamp(), 0, 0, 0, 0, 0, resolvedSrc, 'img'));
|
||||
app.send(ResourceTiming(timestamp(), 0, 0, 0, 0, 0, resolvedSrc, 'img'))
|
||||
}
|
||||
} else if (resolvedSrc.length >= 1e5 || app.sanitizer.isMasked(id)) {
|
||||
sendPlaceholder(id, this);
|
||||
sendPlaceholder(id, this)
|
||||
} else {
|
||||
app.send(new SetNodeAttribute(id, 'src', resolvedSrc));
|
||||
app.send(SetNodeAttribute(id, 'src', resolvedSrc))
|
||||
if (srcset) {
|
||||
const resolvedSrcset = srcset
|
||||
.split(',')
|
||||
.map((str) => resolveURL(str))
|
||||
.join(',');
|
||||
app.send(new SetNodeAttribute(id, 'srcset', resolvedSrcset));
|
||||
.join(',')
|
||||
app.send(SetNodeAttribute(id, 'srcset', resolvedSrcset))
|
||||
}
|
||||
}
|
||||
});
|
||||
})
|
||||
|
||||
const observer = new MutationObserver((mutations) => {
|
||||
for (const mutation of mutations) {
|
||||
if (mutation.type === 'attributes') {
|
||||
const target = mutation.target as HTMLImageElement;
|
||||
const id = app.nodes.getID(target);
|
||||
const target = mutation.target as HTMLImageElement
|
||||
const id = app.nodes.getID(target)
|
||||
if (id === undefined) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
if (mutation.attributeName === 'src') {
|
||||
const src = target.src;
|
||||
app.send(new SetNodeAttributeURLBased(id, 'src', src, app.getBaseHref()));
|
||||
const src = target.src
|
||||
app.send(SetNodeAttributeURLBased(id, 'src', src, app.getBaseHref()))
|
||||
}
|
||||
if (mutation.attributeName === 'srcset') {
|
||||
const srcset = target.srcset;
|
||||
app.send(new SetNodeAttribute(id, 'srcset', srcset));
|
||||
const srcset = target.srcset
|
||||
app.send(SetNodeAttribute(id, 'srcset', srcset))
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
})
|
||||
|
||||
app.nodes.attachNodeCallback((node: Node): void => {
|
||||
if (!hasTag(node, 'IMG')) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
app.nodes.attachElementListener('error', node, sendImgSrc);
|
||||
app.nodes.attachElementListener('load', node, sendImgSrc);
|
||||
sendImgSrc.call(node);
|
||||
observer.observe(node, { attributes: true, attributeFilter: ['src', 'srcset'] });
|
||||
});
|
||||
app.nodes.attachElementListener('error', node, sendImgSrc)
|
||||
app.nodes.attachElementListener('load', node, sendImgSrc)
|
||||
sendImgSrc.call(node)
|
||||
observer.observe(node, { attributes: true, attributeFilter: ['src', 'srcset'] })
|
||||
})
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,74 +1,74 @@
|
|||
import type App from '../app/index.js';
|
||||
import { normSpaces, IN_BROWSER, getLabelAttribute, hasOpenreplayAttribute } from '../utils.js';
|
||||
import { hasTag } from '../app/guards.js';
|
||||
import { SetInputTarget, SetInputValue, SetInputChecked } from '../../common/messages.js';
|
||||
import type App from '../app/index.js'
|
||||
import { normSpaces, IN_BROWSER, getLabelAttribute, hasOpenreplayAttribute } from '../utils.js'
|
||||
import { hasTag } from '../app/guards.js'
|
||||
import { SetInputTarget, SetInputValue, SetInputChecked } from '../app/messages.gen.js'
|
||||
|
||||
const INPUT_TYPES = ['text', 'password', 'email', 'search', 'number', 'range', 'date'];
|
||||
const INPUT_TYPES = ['text', 'password', 'email', 'search', 'number', 'range', 'date']
|
||||
|
||||
// TODO: take into consideration "contenteditable" attribute
|
||||
type TextEditableElement = HTMLInputElement | HTMLTextAreaElement;
|
||||
type TextEditableElement = HTMLInputElement | HTMLTextAreaElement
|
||||
function isTextEditable(node: any): node is TextEditableElement {
|
||||
if (hasTag(node, 'TEXTAREA')) {
|
||||
return true;
|
||||
return true
|
||||
}
|
||||
if (!hasTag(node, 'INPUT')) {
|
||||
return false;
|
||||
return false
|
||||
}
|
||||
|
||||
return INPUT_TYPES.includes(node.type);
|
||||
return INPUT_TYPES.includes(node.type)
|
||||
}
|
||||
|
||||
function isCheckable(node: any): node is HTMLInputElement {
|
||||
if (!hasTag(node, 'INPUT')) {
|
||||
return false;
|
||||
return false
|
||||
}
|
||||
const type = node.type;
|
||||
return type === 'checkbox' || type === 'radio';
|
||||
const type = node.type
|
||||
return type === 'checkbox' || type === 'radio'
|
||||
}
|
||||
|
||||
const labelElementFor: (element: TextEditableElement) => HTMLLabelElement | undefined =
|
||||
IN_BROWSER && 'labels' in HTMLInputElement.prototype
|
||||
? (node) => {
|
||||
let p: Node | null = node;
|
||||
let p: Node | null = node
|
||||
while ((p = p.parentNode) !== null) {
|
||||
if (hasTag(p, 'LABEL')) {
|
||||
return p;
|
||||
return p
|
||||
}
|
||||
}
|
||||
const labels = node.labels;
|
||||
const labels = node.labels
|
||||
if (labels !== null && labels.length === 1) {
|
||||
return labels[0];
|
||||
return labels[0]
|
||||
}
|
||||
}
|
||||
: (node) => {
|
||||
let p: Node | null = node;
|
||||
let p: Node | null = node
|
||||
while ((p = p.parentNode) !== null) {
|
||||
if (hasTag(p, 'LABEL')) {
|
||||
return p;
|
||||
return p
|
||||
}
|
||||
}
|
||||
const id = node.id;
|
||||
const id = node.id
|
||||
if (id) {
|
||||
const labels = document.querySelectorAll('label[for="' + id + '"]');
|
||||
const labels = document.querySelectorAll('label[for="' + id + '"]')
|
||||
if (labels !== null && labels.length === 1) {
|
||||
return labels[0] as HTMLLabelElement;
|
||||
return labels[0] as HTMLLabelElement
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
export function getInputLabel(node: TextEditableElement): string {
|
||||
let label = getLabelAttribute(node);
|
||||
let label = getLabelAttribute(node)
|
||||
if (label === null) {
|
||||
const labelElement = labelElementFor(node);
|
||||
const labelElement = labelElementFor(node)
|
||||
label =
|
||||
(labelElement && labelElement.innerText) ||
|
||||
node.placeholder ||
|
||||
node.name ||
|
||||
node.id ||
|
||||
node.className ||
|
||||
node.type;
|
||||
node.type
|
||||
}
|
||||
return normSpaces(label).slice(0, 100);
|
||||
return normSpaces(label).slice(0, 100)
|
||||
}
|
||||
|
||||
export declare const enum InputMode {
|
||||
|
|
@ -78,10 +78,10 @@ export declare const enum InputMode {
|
|||
}
|
||||
|
||||
export interface Options {
|
||||
obscureInputNumbers: boolean;
|
||||
obscureInputEmails: boolean;
|
||||
defaultInputMode: InputMode;
|
||||
obscureInputDates: boolean;
|
||||
obscureInputNumbers: boolean
|
||||
obscureInputEmails: boolean
|
||||
defaultInputMode: InputMode
|
||||
obscureInputDates: boolean
|
||||
}
|
||||
|
||||
export default function (app: App, opts: Partial<Options>): void {
|
||||
|
|
@ -93,18 +93,18 @@ export default function (app: App, opts: Partial<Options>): void {
|
|||
obscureInputDates: false,
|
||||
},
|
||||
opts,
|
||||
);
|
||||
)
|
||||
function sendInputTarget(id: number, node: TextEditableElement): void {
|
||||
const label = getInputLabel(node);
|
||||
const label = getInputLabel(node)
|
||||
if (label !== '') {
|
||||
app.send(new SetInputTarget(id, label));
|
||||
app.send(SetInputTarget(id, label))
|
||||
}
|
||||
}
|
||||
function sendInputValue(id: number, node: TextEditableElement | HTMLSelectElement): void {
|
||||
let value = node.value;
|
||||
let inputMode: InputMode = options.defaultInputMode;
|
||||
let value = node.value
|
||||
let inputMode: InputMode = options.defaultInputMode
|
||||
if (node.type === 'password' || hasOpenreplayAttribute(node, 'hidden')) {
|
||||
inputMode = InputMode.Hidden;
|
||||
inputMode = InputMode.Hidden
|
||||
} else if (
|
||||
hasOpenreplayAttribute(node, 'obscured') ||
|
||||
(inputMode === InputMode.Plain &&
|
||||
|
|
@ -112,88 +112,88 @@ export default function (app: App, opts: Partial<Options>): void {
|
|||
(options.obscureInputDates && node.type === 'date') ||
|
||||
(options.obscureInputEmails && (node.type === 'email' || !!~value.indexOf('@')))))
|
||||
) {
|
||||
inputMode = InputMode.Obscured;
|
||||
inputMode = InputMode.Obscured
|
||||
}
|
||||
let mask = 0;
|
||||
let mask = 0
|
||||
switch (inputMode) {
|
||||
case InputMode.Hidden:
|
||||
mask = -1;
|
||||
value = '';
|
||||
break;
|
||||
mask = -1
|
||||
value = ''
|
||||
break
|
||||
case InputMode.Obscured:
|
||||
mask = value.length;
|
||||
value = '';
|
||||
break;
|
||||
mask = value.length
|
||||
value = ''
|
||||
break
|
||||
}
|
||||
|
||||
app.send(new SetInputValue(id, value, mask));
|
||||
app.send(SetInputValue(id, value, mask))
|
||||
}
|
||||
|
||||
const inputValues: Map<number, string> = new Map();
|
||||
const checkableValues: Map<number, boolean> = new Map();
|
||||
const registeredTargets: Set<number> = new Set();
|
||||
const inputValues: Map<number, string> = new Map()
|
||||
const checkableValues: Map<number, boolean> = new Map()
|
||||
const registeredTargets: Set<number> = new Set()
|
||||
|
||||
app.attachStopCallback(() => {
|
||||
inputValues.clear();
|
||||
checkableValues.clear();
|
||||
registeredTargets.clear();
|
||||
});
|
||||
inputValues.clear()
|
||||
checkableValues.clear()
|
||||
registeredTargets.clear()
|
||||
})
|
||||
|
||||
app.ticker.attach((): void => {
|
||||
inputValues.forEach((value, id) => {
|
||||
const node = app.nodes.getNode(id);
|
||||
if (!node) return;
|
||||
const node = app.nodes.getNode(id)
|
||||
if (!node) return
|
||||
if (!isTextEditable(node)) {
|
||||
inputValues.delete(id);
|
||||
return;
|
||||
inputValues.delete(id)
|
||||
return
|
||||
}
|
||||
if (value !== node.value) {
|
||||
inputValues.set(id, node.value);
|
||||
inputValues.set(id, node.value)
|
||||
if (!registeredTargets.has(id)) {
|
||||
registeredTargets.add(id);
|
||||
sendInputTarget(id, node);
|
||||
registeredTargets.add(id)
|
||||
sendInputTarget(id, node)
|
||||
}
|
||||
sendInputValue(id, node);
|
||||
sendInputValue(id, node)
|
||||
}
|
||||
});
|
||||
})
|
||||
checkableValues.forEach((checked, id) => {
|
||||
const node = app.nodes.getNode(id);
|
||||
if (!node) return;
|
||||
const node = app.nodes.getNode(id)
|
||||
if (!node) return
|
||||
if (!isCheckable(node)) {
|
||||
checkableValues.delete(id);
|
||||
return;
|
||||
checkableValues.delete(id)
|
||||
return
|
||||
}
|
||||
if (checked !== node.checked) {
|
||||
checkableValues.set(id, node.checked);
|
||||
app.send(new SetInputChecked(id, node.checked));
|
||||
checkableValues.set(id, node.checked)
|
||||
app.send(SetInputChecked(id, node.checked))
|
||||
}
|
||||
});
|
||||
});
|
||||
app.ticker.attach(Set.prototype.clear, 100, false, registeredTargets);
|
||||
})
|
||||
})
|
||||
app.ticker.attach(Set.prototype.clear, 100, false, registeredTargets)
|
||||
|
||||
app.nodes.attachNodeCallback(
|
||||
app.safe((node: Node): void => {
|
||||
const id = app.nodes.getID(node);
|
||||
const id = app.nodes.getID(node)
|
||||
if (id === undefined) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
// TODO: support multiple select (?): use selectedOptions; Need send target?
|
||||
if (hasTag(node, 'SELECT')) {
|
||||
sendInputValue(id, node);
|
||||
sendInputValue(id, node)
|
||||
app.attachEventListener(node, 'change', () => {
|
||||
sendInputValue(id, node);
|
||||
});
|
||||
sendInputValue(id, node)
|
||||
})
|
||||
}
|
||||
if (isTextEditable(node)) {
|
||||
inputValues.set(id, node.value);
|
||||
sendInputValue(id, node);
|
||||
return;
|
||||
inputValues.set(id, node.value)
|
||||
sendInputValue(id, node)
|
||||
return
|
||||
}
|
||||
if (isCheckable(node)) {
|
||||
checkableValues.set(id, node.checked);
|
||||
app.send(new SetInputChecked(id, node.checked));
|
||||
return;
|
||||
checkableValues.set(id, node.checked)
|
||||
app.send(SetInputChecked(id, node.checked))
|
||||
return
|
||||
}
|
||||
}),
|
||||
);
|
||||
)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,22 +1,22 @@
|
|||
import type App from '../app/index.js';
|
||||
import { LongTask } from '../../common/messages.js';
|
||||
import type App from '../app/index.js'
|
||||
import { LongTask } from '../app/messages.gen.js'
|
||||
|
||||
// https://w3c.github.io/performance-timeline/#the-performanceentry-interface
|
||||
interface TaskAttributionTiming extends PerformanceEntry {
|
||||
readonly containerType: string;
|
||||
readonly containerSrc: string;
|
||||
readonly containerId: string;
|
||||
readonly containerName: string;
|
||||
readonly containerType: string
|
||||
readonly containerSrc: string
|
||||
readonly containerId: string
|
||||
readonly containerName: string
|
||||
}
|
||||
|
||||
// https://www.w3.org/TR/longtasks/#performancelongtasktiming
|
||||
interface PerformanceLongTaskTiming extends PerformanceEntry {
|
||||
readonly attribution: ReadonlyArray<TaskAttributionTiming>;
|
||||
readonly attribution: ReadonlyArray<TaskAttributionTiming>
|
||||
}
|
||||
|
||||
export default function (app: App): void {
|
||||
if (!('PerformanceObserver' in window) || !('PerformanceLongTaskTiming' in window)) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
|
||||
const contexts: string[] = [
|
||||
|
|
@ -29,23 +29,23 @@ export default function (app: App): void {
|
|||
'cross-origin-descendant',
|
||||
'cross-origin-unreachable',
|
||||
'multiple-contexts',
|
||||
];
|
||||
const containerTypes: string[] = ['window', 'iframe', 'embed', 'object'];
|
||||
]
|
||||
const containerTypes: string[] = ['window', 'iframe', 'embed', 'object']
|
||||
function longTask(entry: PerformanceLongTaskTiming): void {
|
||||
let type = '',
|
||||
src = '',
|
||||
id = '',
|
||||
name = '';
|
||||
const container = entry.attribution[0];
|
||||
name = ''
|
||||
const container = entry.attribution[0]
|
||||
if (container != null) {
|
||||
type = container.containerType;
|
||||
name = container.containerName;
|
||||
id = container.containerId;
|
||||
src = container.containerSrc;
|
||||
type = container.containerType
|
||||
name = container.containerName
|
||||
id = container.containerId
|
||||
src = container.containerSrc
|
||||
}
|
||||
|
||||
app.send(
|
||||
new LongTask(
|
||||
LongTask(
|
||||
entry.startTime + performance.timing.navigationStart,
|
||||
entry.duration,
|
||||
Math.max(contexts.indexOf(entry.name), 0),
|
||||
|
|
@ -54,11 +54,11 @@ export default function (app: App): void {
|
|||
id,
|
||||
src,
|
||||
),
|
||||
);
|
||||
)
|
||||
}
|
||||
|
||||
const observer: PerformanceObserver = new PerformanceObserver((list) =>
|
||||
list.getEntries().forEach(longTask),
|
||||
);
|
||||
observer.observe({ entryTypes: ['longtask'] });
|
||||
)
|
||||
observer.observe({ entryTypes: ['longtask'] })
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,15 +1,15 @@
|
|||
import type App from '../app/index.js';
|
||||
import { hasTag, isSVGElement } from '../app/guards.js';
|
||||
import { normSpaces, hasOpenreplayAttribute, getLabelAttribute } from '../utils.js';
|
||||
import { MouseMove, MouseClick } from '../../common/messages.js';
|
||||
import { getInputLabel } from './input.js';
|
||||
import type App from '../app/index.js'
|
||||
import { hasTag, isSVGElement } from '../app/guards.js'
|
||||
import { normSpaces, hasOpenreplayAttribute, getLabelAttribute } from '../utils.js'
|
||||
import { MouseMove, MouseClick } from '../app/messages.gen.js'
|
||||
import { getInputLabel } from './input.js'
|
||||
|
||||
function _getSelector(target: Element): string {
|
||||
let el: Element | null = target;
|
||||
let selector: string | null = null;
|
||||
let el: Element | null = target
|
||||
let selector: string | null = null
|
||||
do {
|
||||
if (el.id) {
|
||||
return `#${el.id}` + (selector ? ` > ${selector}` : '');
|
||||
return `#${el.id}` + (selector ? ` > ${selector}` : '')
|
||||
}
|
||||
selector =
|
||||
el.className
|
||||
|
|
@ -17,17 +17,17 @@ function _getSelector(target: Element): string {
|
|||
.map((cn) => cn.trim())
|
||||
.filter((cn) => cn !== '')
|
||||
.reduce((sel, cn) => `${sel}.${cn}`, el.tagName.toLowerCase()) +
|
||||
(selector ? ` > ${selector}` : '');
|
||||
(selector ? ` > ${selector}` : '')
|
||||
if (el === document.body) {
|
||||
return selector;
|
||||
return selector
|
||||
}
|
||||
el = el.parentElement;
|
||||
} while (el !== document.body && el !== null);
|
||||
return selector;
|
||||
el = el.parentElement
|
||||
} while (el !== document.body && el !== null)
|
||||
return selector
|
||||
}
|
||||
|
||||
function isClickable(element: Element): boolean {
|
||||
const tag = element.tagName.toUpperCase();
|
||||
const tag = element.tagName.toUpperCase()
|
||||
return (
|
||||
tag === 'BUTTON' ||
|
||||
tag === 'A' ||
|
||||
|
|
@ -35,7 +35,7 @@ function isClickable(element: Element): boolean {
|
|||
tag === 'SELECT' ||
|
||||
(element as HTMLElement).onclick != null ||
|
||||
element.getAttribute('role') === 'button'
|
||||
);
|
||||
)
|
||||
//|| element.className.includes("btn")
|
||||
// MBTODO: intersect addEventListener
|
||||
}
|
||||
|
|
@ -43,125 +43,125 @@ function isClickable(element: Element): boolean {
|
|||
//TODO: fix (typescript doesn't allow work when the guard is inside the function)
|
||||
function getTarget(target: EventTarget | null): Element | null {
|
||||
if (target instanceof Element) {
|
||||
return _getTarget(target);
|
||||
return _getTarget(target)
|
||||
}
|
||||
return null;
|
||||
return null
|
||||
}
|
||||
|
||||
function _getTarget(target: Element): Element | null {
|
||||
let element: Element | null = target;
|
||||
let element: Element | null = target
|
||||
while (element !== null && element !== document.documentElement) {
|
||||
if (hasOpenreplayAttribute(element, 'masked')) {
|
||||
return null;
|
||||
return null
|
||||
}
|
||||
element = element.parentElement;
|
||||
element = element.parentElement
|
||||
}
|
||||
if (isSVGElement(target)) {
|
||||
let owner = target.ownerSVGElement;
|
||||
let owner = target.ownerSVGElement
|
||||
while (owner !== null) {
|
||||
target = owner;
|
||||
owner = owner.ownerSVGElement;
|
||||
target = owner
|
||||
owner = owner.ownerSVGElement
|
||||
}
|
||||
}
|
||||
element = target;
|
||||
element = target
|
||||
while (element !== null && element !== document.documentElement) {
|
||||
const tag = element.tagName.toUpperCase();
|
||||
const tag = element.tagName.toUpperCase()
|
||||
if (tag === 'LABEL') {
|
||||
return null;
|
||||
return null
|
||||
}
|
||||
if (tag === 'INPUT') {
|
||||
return element;
|
||||
return element
|
||||
}
|
||||
if (isClickable(element) || getLabelAttribute(element) !== null) {
|
||||
return element;
|
||||
return element
|
||||
}
|
||||
element = element.parentElement;
|
||||
element = element.parentElement
|
||||
}
|
||||
return target === document.documentElement ? null : target;
|
||||
return target === document.documentElement ? null : target
|
||||
}
|
||||
|
||||
export default function (app: App): void {
|
||||
function getTargetLabel(target: Element): string {
|
||||
const dl = getLabelAttribute(target);
|
||||
const dl = getLabelAttribute(target)
|
||||
if (dl !== null) {
|
||||
return dl;
|
||||
return dl
|
||||
}
|
||||
if (hasTag(target, 'INPUT')) {
|
||||
return getInputLabel(target);
|
||||
return getInputLabel(target)
|
||||
}
|
||||
if (isClickable(target)) {
|
||||
let label = '';
|
||||
let label = ''
|
||||
if (target instanceof HTMLElement) {
|
||||
label = app.sanitizer.getInnerTextSecure(target);
|
||||
label = app.sanitizer.getInnerTextSecure(target)
|
||||
}
|
||||
label = label || target.id || target.className;
|
||||
return normSpaces(label).slice(0, 100);
|
||||
label = label || target.id || target.className
|
||||
return normSpaces(label).slice(0, 100)
|
||||
}
|
||||
return '';
|
||||
return ''
|
||||
}
|
||||
|
||||
let mousePositionX = -1;
|
||||
let mousePositionY = -1;
|
||||
let mousePositionChanged = false;
|
||||
let mouseTarget: Element | null = null;
|
||||
let mouseTargetTime = 0;
|
||||
let mousePositionX = -1
|
||||
let mousePositionY = -1
|
||||
let mousePositionChanged = false
|
||||
let mouseTarget: Element | null = null
|
||||
let mouseTargetTime = 0
|
||||
|
||||
app.attachStopCallback(() => {
|
||||
mousePositionX = -1;
|
||||
mousePositionY = -1;
|
||||
mousePositionChanged = false;
|
||||
mouseTarget = null;
|
||||
});
|
||||
mousePositionX = -1
|
||||
mousePositionY = -1
|
||||
mousePositionChanged = false
|
||||
mouseTarget = null
|
||||
})
|
||||
|
||||
const sendMouseMove = (): void => {
|
||||
if (mousePositionChanged) {
|
||||
app.send(new MouseMove(mousePositionX, mousePositionY));
|
||||
mousePositionChanged = false;
|
||||
app.send(MouseMove(mousePositionX, mousePositionY))
|
||||
mousePositionChanged = false
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
const selectorMap: { [id: number]: string } = {};
|
||||
const selectorMap: { [id: number]: string } = {}
|
||||
function getSelector(id: number, target: Element): string {
|
||||
return (selectorMap[id] = selectorMap[id] || _getSelector(target));
|
||||
return (selectorMap[id] = selectorMap[id] || _getSelector(target))
|
||||
}
|
||||
|
||||
app.attachEventListener(document.documentElement, 'mouseover', (e: MouseEvent): void => {
|
||||
const target = getTarget(e.target);
|
||||
const target = getTarget(e.target)
|
||||
if (target !== mouseTarget) {
|
||||
mouseTarget = target;
|
||||
mouseTargetTime = performance.now();
|
||||
mouseTarget = target
|
||||
mouseTargetTime = performance.now()
|
||||
}
|
||||
});
|
||||
})
|
||||
app.attachEventListener(
|
||||
document,
|
||||
'mousemove',
|
||||
(e: MouseEvent): void => {
|
||||
mousePositionX = e.clientX;
|
||||
mousePositionY = e.clientY;
|
||||
mousePositionChanged = true;
|
||||
mousePositionX = e.clientX
|
||||
mousePositionY = e.clientY
|
||||
mousePositionChanged = true
|
||||
},
|
||||
false,
|
||||
);
|
||||
)
|
||||
app.attachEventListener(document, 'click', (e: MouseEvent): void => {
|
||||
const target = getTarget(e.target);
|
||||
const target = getTarget(e.target)
|
||||
if ((!e.clientX && !e.clientY) || target === null) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
const id = app.nodes.getID(target);
|
||||
const id = app.nodes.getID(target)
|
||||
if (id !== undefined) {
|
||||
sendMouseMove();
|
||||
sendMouseMove()
|
||||
app.send(
|
||||
new MouseClick(
|
||||
MouseClick(
|
||||
id,
|
||||
mouseTarget === target ? Math.round(performance.now() - mouseTargetTime) : 0,
|
||||
getTargetLabel(target),
|
||||
getSelector(id, target),
|
||||
),
|
||||
true,
|
||||
);
|
||||
)
|
||||
}
|
||||
mouseTarget = null;
|
||||
});
|
||||
mouseTarget = null
|
||||
})
|
||||
|
||||
app.ticker.attach(sendMouseMove, 10);
|
||||
app.ticker.attach(sendMouseMove, 10)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,25 +1,25 @@
|
|||
import type App from '../app/index.js';
|
||||
import { IN_BROWSER } from '../utils.js';
|
||||
import { PerformanceTrack } from '../../common/messages.js';
|
||||
import type App from '../app/index.js'
|
||||
import { IN_BROWSER } from '../utils.js'
|
||||
import { PerformanceTrack } from '../app/messages.gen.js'
|
||||
|
||||
type Perf = {
|
||||
memory: {
|
||||
totalJSHeapSize?: number;
|
||||
usedJSHeapSize?: number;
|
||||
jsHeapSizeLimit?: number;
|
||||
};
|
||||
};
|
||||
totalJSHeapSize?: number
|
||||
usedJSHeapSize?: number
|
||||
jsHeapSizeLimit?: number
|
||||
}
|
||||
}
|
||||
|
||||
const perf: Perf =
|
||||
IN_BROWSER && 'performance' in window && 'memory' in performance // works in Chrome only
|
||||
? (performance as any)
|
||||
: { memory: {} };
|
||||
: { memory: {} }
|
||||
|
||||
export const deviceMemory = IN_BROWSER ? ((navigator as any).deviceMemory || 0) * 1024 : 0;
|
||||
export const jsHeapSizeLimit = perf.memory.jsHeapSizeLimit || 0;
|
||||
export const deviceMemory = IN_BROWSER ? ((navigator as any).deviceMemory || 0) * 1024 : 0
|
||||
export const jsHeapSizeLimit = perf.memory.jsHeapSizeLimit || 0
|
||||
|
||||
export interface Options {
|
||||
capturePerformance: boolean;
|
||||
capturePerformance: boolean
|
||||
}
|
||||
|
||||
export default function (app: App, opts: Partial<Options>): void {
|
||||
|
|
@ -28,59 +28,59 @@ export default function (app: App, opts: Partial<Options>): void {
|
|||
capturePerformance: true,
|
||||
},
|
||||
opts,
|
||||
);
|
||||
)
|
||||
if (!options.capturePerformance) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
|
||||
let frames: number | undefined;
|
||||
let ticks: number | undefined;
|
||||
let frames: number | undefined
|
||||
let ticks: number | undefined
|
||||
|
||||
const nextFrame = (): void => {
|
||||
if (frames === undefined || frames === -1) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
frames++;
|
||||
requestAnimationFrame(nextFrame);
|
||||
};
|
||||
frames++
|
||||
requestAnimationFrame(nextFrame)
|
||||
}
|
||||
|
||||
app.ticker.attach(
|
||||
(): void => {
|
||||
if (ticks === undefined || ticks === -1) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
ticks++;
|
||||
ticks++
|
||||
},
|
||||
0,
|
||||
false,
|
||||
);
|
||||
)
|
||||
|
||||
const sendPerformanceTrack = (): void => {
|
||||
if (frames === undefined || ticks === undefined) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
app.send(
|
||||
new PerformanceTrack(
|
||||
PerformanceTrack(
|
||||
frames,
|
||||
ticks,
|
||||
perf.memory.totalJSHeapSize || 0,
|
||||
perf.memory.usedJSHeapSize || 0,
|
||||
),
|
||||
);
|
||||
ticks = frames = document.hidden ? -1 : 0;
|
||||
};
|
||||
)
|
||||
ticks = frames = document.hidden ? -1 : 0
|
||||
}
|
||||
|
||||
app.attachStartCallback((): void => {
|
||||
ticks = frames = -1;
|
||||
sendPerformanceTrack();
|
||||
nextFrame();
|
||||
});
|
||||
ticks = frames = -1
|
||||
sendPerformanceTrack()
|
||||
nextFrame()
|
||||
})
|
||||
|
||||
app.attachStopCallback((): void => {
|
||||
ticks = frames = undefined;
|
||||
});
|
||||
ticks = frames = undefined
|
||||
})
|
||||
|
||||
app.ticker.attach(sendPerformanceTrack, 40, false);
|
||||
app.ticker.attach(sendPerformanceTrack, 40, false)
|
||||
|
||||
if (document.hidden !== undefined) {
|
||||
app.attachEventListener(
|
||||
|
|
@ -89,6 +89,6 @@ export default function (app: App, opts: Partial<Options>): void {
|
|||
sendPerformanceTrack as EventListener,
|
||||
false,
|
||||
false,
|
||||
);
|
||||
)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,14 +1,14 @@
|
|||
import type App from '../app/index.js';
|
||||
import { SetViewportScroll, SetNodeScroll } from '../../common/messages.js';
|
||||
import { isElementNode } from '../app/guards.js';
|
||||
import type App from '../app/index.js'
|
||||
import { SetViewportScroll, SetNodeScroll } from '../app/messages.gen.js'
|
||||
import { isElementNode } from '../app/guards.js'
|
||||
|
||||
export default function (app: App): void {
|
||||
let documentScroll = false;
|
||||
const nodeScroll: Map<Element, [number, number]> = new Map();
|
||||
let documentScroll = false
|
||||
const nodeScroll: Map<Element, [number, number]> = new Map()
|
||||
|
||||
const sendSetViewportScroll = app.safe((): void =>
|
||||
app.send(
|
||||
new SetViewportScroll(
|
||||
SetViewportScroll(
|
||||
window.pageXOffset ||
|
||||
(document.documentElement && document.documentElement.scrollLeft) ||
|
||||
(document.body && document.body.scrollLeft) ||
|
||||
|
|
@ -19,49 +19,49 @@ export default function (app: App): void {
|
|||
0,
|
||||
),
|
||||
),
|
||||
);
|
||||
)
|
||||
|
||||
const sendSetNodeScroll = app.safe((s: [number, number], node: Node): void => {
|
||||
const id = app.nodes.getID(node);
|
||||
const id = app.nodes.getID(node)
|
||||
if (id !== undefined) {
|
||||
app.send(new SetNodeScroll(id, s[0], s[1]));
|
||||
app.send(SetNodeScroll(id, s[0], s[1]))
|
||||
}
|
||||
});
|
||||
})
|
||||
|
||||
app.attachStartCallback(sendSetViewportScroll);
|
||||
app.attachStartCallback(sendSetViewportScroll)
|
||||
|
||||
app.attachStopCallback(() => {
|
||||
documentScroll = false;
|
||||
nodeScroll.clear();
|
||||
});
|
||||
documentScroll = false
|
||||
nodeScroll.clear()
|
||||
})
|
||||
|
||||
app.nodes.attachNodeCallback((node, isStart) => {
|
||||
if (isStart && isElementNode(node) && node.scrollLeft + node.scrollTop > 0) {
|
||||
nodeScroll.set(node, [node.scrollLeft, node.scrollTop]);
|
||||
nodeScroll.set(node, [node.scrollLeft, node.scrollTop])
|
||||
}
|
||||
});
|
||||
})
|
||||
|
||||
app.attachEventListener(window, 'scroll', (e: Event): void => {
|
||||
const target = e.target;
|
||||
const target = e.target
|
||||
if (target === document) {
|
||||
documentScroll = true;
|
||||
return;
|
||||
documentScroll = true
|
||||
return
|
||||
}
|
||||
if (target instanceof Element) {
|
||||
nodeScroll.set(target, [target.scrollLeft, target.scrollTop]);
|
||||
nodeScroll.set(target, [target.scrollLeft, target.scrollTop])
|
||||
}
|
||||
});
|
||||
})
|
||||
|
||||
app.ticker.attach(
|
||||
(): void => {
|
||||
if (documentScroll) {
|
||||
sendSetViewportScroll();
|
||||
documentScroll = false;
|
||||
sendSetViewportScroll()
|
||||
documentScroll = false
|
||||
}
|
||||
nodeScroll.forEach(sendSetNodeScroll);
|
||||
nodeScroll.clear();
|
||||
nodeScroll.forEach(sendSetNodeScroll)
|
||||
nodeScroll.clear()
|
||||
},
|
||||
5,
|
||||
false,
|
||||
);
|
||||
)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,62 +1,62 @@
|
|||
import type App from '../app/index.js';
|
||||
import { hasTag } from '../app/guards.js';
|
||||
import { isURL } from '../utils.js';
|
||||
import { ResourceTiming, PageLoadTiming, PageRenderTiming } from '../../common/messages.js';
|
||||
import type App from '../app/index.js'
|
||||
import { hasTag } from '../app/guards.js'
|
||||
import { isURL } from '../utils.js'
|
||||
import { ResourceTiming, PageLoadTiming, PageRenderTiming } from '../app/messages.gen.js'
|
||||
|
||||
// Inspired by https://github.com/WPO-Foundation/RUM-SpeedIndex/blob/master/src/rum-speedindex.js
|
||||
|
||||
interface ResourcesTimeMap {
|
||||
[k: string]: number;
|
||||
[k: string]: number
|
||||
}
|
||||
|
||||
interface PaintBlock {
|
||||
time: number;
|
||||
area: number;
|
||||
time: number
|
||||
area: number
|
||||
}
|
||||
|
||||
function getPaintBlocks(resources: ResourcesTimeMap): Array<PaintBlock> {
|
||||
const paintBlocks: Array<PaintBlock> = [];
|
||||
const elements = document.getElementsByTagName('*');
|
||||
const styleURL = /url\(("[^"]*"|'[^']*'|[^)]*)\)/i;
|
||||
const paintBlocks: Array<PaintBlock> = []
|
||||
const elements = document.getElementsByTagName('*')
|
||||
const styleURL = /url\(("[^"]*"|'[^']*'|[^)]*)\)/i
|
||||
for (let i = 0; i < elements.length; i++) {
|
||||
const element = elements[i];
|
||||
let src = '';
|
||||
const element = elements[i]
|
||||
let src = ''
|
||||
if (hasTag(element, 'IMG')) {
|
||||
src = element.currentSrc || element.src;
|
||||
src = element.currentSrc || element.src
|
||||
}
|
||||
if (!src) {
|
||||
const backgroundImage = getComputedStyle(element).getPropertyValue('background-image');
|
||||
const backgroundImage = getComputedStyle(element).getPropertyValue('background-image')
|
||||
if (backgroundImage) {
|
||||
const matches = styleURL.exec(backgroundImage);
|
||||
const matches = styleURL.exec(backgroundImage)
|
||||
if (matches !== null) {
|
||||
src = matches[1];
|
||||
src = matches[1]
|
||||
if (src.startsWith('"') || src.startsWith("'")) {
|
||||
src = src.substr(1, src.length - 2);
|
||||
src = src.substr(1, src.length - 2)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!src) continue;
|
||||
const time = src.substr(0, 10) === 'data:image' ? 0 : resources[src];
|
||||
if (time === undefined) continue;
|
||||
const rect = element.getBoundingClientRect();
|
||||
const top = Math.max(rect.top, 0);
|
||||
const left = Math.max(rect.left, 0);
|
||||
if (!src) continue
|
||||
const time = src.substr(0, 10) === 'data:image' ? 0 : resources[src]
|
||||
if (time === undefined) continue
|
||||
const rect = element.getBoundingClientRect()
|
||||
const top = Math.max(rect.top, 0)
|
||||
const left = Math.max(rect.left, 0)
|
||||
const bottom = Math.min(
|
||||
rect.bottom,
|
||||
window.innerHeight ||
|
||||
(document.documentElement && document.documentElement.clientHeight) ||
|
||||
0,
|
||||
);
|
||||
)
|
||||
const right = Math.min(
|
||||
rect.right,
|
||||
window.innerWidth || (document.documentElement && document.documentElement.clientWidth) || 0,
|
||||
);
|
||||
if (bottom <= top || right <= left) continue;
|
||||
const area = (bottom - top) * (right - left);
|
||||
paintBlocks.push({ time, area });
|
||||
)
|
||||
if (bottom <= top || right <= left) continue
|
||||
const area = (bottom - top) * (right - left)
|
||||
paintBlocks.push({ time, area })
|
||||
}
|
||||
return paintBlocks;
|
||||
return paintBlocks
|
||||
}
|
||||
|
||||
function calculateSpeedIndex(firstContentfulPaint: number, paintBlocks: Array<PaintBlock>): number {
|
||||
|
|
@ -69,20 +69,20 @@ function calculateSpeedIndex(firstContentfulPaint: number, paintBlocks: Array<Pa
|
|||
(document.documentElement && document.documentElement.clientHeight) || 0,
|
||||
window.innerHeight || 0,
|
||||
)) /
|
||||
10;
|
||||
let s = a * firstContentfulPaint;
|
||||
10
|
||||
let s = a * firstContentfulPaint
|
||||
for (let i = 0; i < paintBlocks.length; i++) {
|
||||
const { time, area } = paintBlocks[i];
|
||||
a += area;
|
||||
s += area * (time > firstContentfulPaint ? time : firstContentfulPaint);
|
||||
const { time, area } = paintBlocks[i]
|
||||
a += area
|
||||
s += area * (time > firstContentfulPaint ? time : firstContentfulPaint)
|
||||
}
|
||||
return a === 0 ? 0 : s / a;
|
||||
return a === 0 ? 0 : s / a
|
||||
}
|
||||
|
||||
export interface Options {
|
||||
captureResourceTimings: boolean;
|
||||
capturePageLoadTimings: boolean;
|
||||
capturePageRenderTimings: boolean;
|
||||
captureResourceTimings: boolean
|
||||
capturePageLoadTimings: boolean
|
||||
capturePageRenderTimings: boolean
|
||||
}
|
||||
|
||||
export default function (app: App, opts: Partial<Options>): void {
|
||||
|
|
@ -93,23 +93,23 @@ export default function (app: App, opts: Partial<Options>): void {
|
|||
capturePageRenderTimings: true,
|
||||
},
|
||||
opts,
|
||||
);
|
||||
)
|
||||
if (!('PerformanceObserver' in window)) {
|
||||
options.captureResourceTimings = false;
|
||||
options.captureResourceTimings = false
|
||||
}
|
||||
if (!options.captureResourceTimings) {
|
||||
return;
|
||||
return
|
||||
} // Resources are necessary for all timings
|
||||
|
||||
let resources: ResourcesTimeMap | null = {};
|
||||
let resources: ResourcesTimeMap | null = {}
|
||||
|
||||
function resourceTiming(entry: PerformanceResourceTiming): void {
|
||||
if (entry.duration < 0 || !isURL(entry.name) || app.isServiceURL(entry.name)) return;
|
||||
if (entry.duration < 0 || !isURL(entry.name) || app.isServiceURL(entry.name)) return
|
||||
if (resources !== null) {
|
||||
resources[entry.name] = entry.startTime + entry.duration;
|
||||
resources[entry.name] = entry.startTime + entry.duration
|
||||
}
|
||||
app.send(
|
||||
new ResourceTiming(
|
||||
ResourceTiming(
|
||||
entry.startTime + performance.timing.navigationStart,
|
||||
entry.duration,
|
||||
entry.responseStart && entry.startTime ? entry.responseStart - entry.startTime : 0,
|
||||
|
|
@ -119,51 +119,51 @@ export default function (app: App, opts: Partial<Options>): void {
|
|||
entry.name,
|
||||
entry.initiatorType,
|
||||
),
|
||||
);
|
||||
)
|
||||
}
|
||||
|
||||
const observer: PerformanceObserver = new PerformanceObserver((list) =>
|
||||
list.getEntries().forEach(resourceTiming),
|
||||
);
|
||||
)
|
||||
|
||||
let prevSessionID: string | undefined;
|
||||
let prevSessionID: string | undefined
|
||||
app.attachStartCallback(function ({ sessionID }) {
|
||||
if (sessionID !== prevSessionID) {
|
||||
// Send past page resources on a newly started session
|
||||
performance.getEntriesByType('resource').forEach(resourceTiming);
|
||||
prevSessionID = sessionID;
|
||||
performance.getEntriesByType('resource').forEach(resourceTiming)
|
||||
prevSessionID = sessionID
|
||||
}
|
||||
observer.observe({ entryTypes: ['resource'] });
|
||||
});
|
||||
observer.observe({ entryTypes: ['resource'] })
|
||||
})
|
||||
|
||||
app.attachStopCallback(function () {
|
||||
observer.disconnect();
|
||||
});
|
||||
observer.disconnect()
|
||||
})
|
||||
|
||||
let firstPaint = 0,
|
||||
firstContentfulPaint = 0;
|
||||
firstContentfulPaint = 0
|
||||
|
||||
if (options.capturePageLoadTimings) {
|
||||
let pageLoadTimingSent = false;
|
||||
let pageLoadTimingSent = false
|
||||
app.ticker.attach(() => {
|
||||
if (pageLoadTimingSent) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
if (firstPaint === 0 || firstContentfulPaint === 0) {
|
||||
performance.getEntriesByType('paint').forEach((entry: PerformanceEntry) => {
|
||||
const { name, startTime } = entry;
|
||||
const { name, startTime } = entry
|
||||
switch (name) {
|
||||
case 'first-paint':
|
||||
firstPaint = startTime;
|
||||
break;
|
||||
firstPaint = startTime
|
||||
break
|
||||
case 'first-contentful-paint':
|
||||
firstContentfulPaint = startTime;
|
||||
break;
|
||||
firstContentfulPaint = startTime
|
||||
break
|
||||
}
|
||||
});
|
||||
})
|
||||
}
|
||||
if (performance.timing.loadEventEnd || performance.now() > 30000) {
|
||||
pageLoadTimingSent = true;
|
||||
pageLoadTimingSent = true
|
||||
const {
|
||||
navigationStart,
|
||||
requestStart,
|
||||
|
|
@ -173,9 +173,9 @@ export default function (app: App, opts: Partial<Options>): void {
|
|||
domContentLoadedEventEnd,
|
||||
loadEventStart,
|
||||
loadEventEnd,
|
||||
} = performance.timing;
|
||||
} = performance.timing
|
||||
app.send(
|
||||
new PageLoadTiming(
|
||||
PageLoadTiming(
|
||||
requestStart - navigationStart || 0,
|
||||
responseStart - navigationStart || 0,
|
||||
responseEnd - navigationStart || 0,
|
||||
|
|
@ -186,46 +186,46 @@ export default function (app: App, opts: Partial<Options>): void {
|
|||
firstPaint,
|
||||
firstContentfulPaint,
|
||||
),
|
||||
);
|
||||
)
|
||||
}
|
||||
}, 30);
|
||||
}, 30)
|
||||
}
|
||||
|
||||
if (options.capturePageRenderTimings) {
|
||||
let visuallyComplete = 0,
|
||||
interactiveWindowStartTime = 0,
|
||||
interactiveWindowTickTime: number | null = 0,
|
||||
paintBlocks: Array<PaintBlock> | null = null;
|
||||
paintBlocks: Array<PaintBlock> | null = null
|
||||
|
||||
let pageRenderTimingSent = false;
|
||||
let pageRenderTimingSent = false
|
||||
app.ticker.attach(() => {
|
||||
if (pageRenderTimingSent) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
const time = performance.now();
|
||||
const time = performance.now()
|
||||
if (resources !== null) {
|
||||
visuallyComplete = Math.max.apply(
|
||||
null,
|
||||
Object.keys(resources).map((k) => (resources as any)[k]),
|
||||
);
|
||||
)
|
||||
if (time - visuallyComplete > 1000) {
|
||||
paintBlocks = getPaintBlocks(resources);
|
||||
resources = null;
|
||||
paintBlocks = getPaintBlocks(resources)
|
||||
resources = null
|
||||
}
|
||||
}
|
||||
if (interactiveWindowTickTime !== null) {
|
||||
if (time - interactiveWindowTickTime > 50) {
|
||||
interactiveWindowStartTime = time;
|
||||
interactiveWindowStartTime = time
|
||||
}
|
||||
interactiveWindowTickTime = time - interactiveWindowStartTime > 5000 ? null : time;
|
||||
interactiveWindowTickTime = time - interactiveWindowStartTime > 5000 ? null : time
|
||||
}
|
||||
if ((paintBlocks !== null && interactiveWindowTickTime === null) || time > 30000) {
|
||||
pageRenderTimingSent = true;
|
||||
resources = null;
|
||||
pageRenderTimingSent = true
|
||||
resources = null
|
||||
const speedIndex =
|
||||
paintBlocks === null
|
||||
? 0
|
||||
: calculateSpeedIndex(firstContentfulPaint || firstPaint, paintBlocks);
|
||||
: calculateSpeedIndex(firstContentfulPaint || firstPaint, paintBlocks)
|
||||
const timeToInteractive =
|
||||
interactiveWindowTickTime === null
|
||||
? Math.max(
|
||||
|
|
@ -234,15 +234,15 @@ export default function (app: App, opts: Partial<Options>): void {
|
|||
performance.timing.domContentLoadedEventEnd - performance.timing.navigationStart ||
|
||||
0,
|
||||
)
|
||||
: 0;
|
||||
: 0
|
||||
app.send(
|
||||
new PageRenderTiming(
|
||||
PageRenderTiming(
|
||||
speedIndex,
|
||||
firstContentfulPaint > visuallyComplete ? firstContentfulPaint : visuallyComplete,
|
||||
timeToInteractive,
|
||||
),
|
||||
);
|
||||
)
|
||||
}
|
||||
});
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,40 +1,40 @@
|
|||
import type App from '../app/index.js';
|
||||
import { SetPageLocation, SetViewportSize, SetPageVisibility } from '../../common/messages.js';
|
||||
import type App from '../app/index.js'
|
||||
import { SetPageLocation, SetViewportSize, SetPageVisibility } from '../app/messages.gen.js'
|
||||
|
||||
export default function (app: App): void {
|
||||
let url: string, width: number, height: number;
|
||||
let navigationStart = performance.timing.navigationStart;
|
||||
let url: string, width: number, height: number
|
||||
let navigationStart = performance.timing.navigationStart
|
||||
|
||||
const sendSetPageLocation = app.safe(() => {
|
||||
const { URL } = document;
|
||||
const { URL } = document
|
||||
if (URL !== url) {
|
||||
url = URL;
|
||||
app.send(new SetPageLocation(url, document.referrer, navigationStart));
|
||||
navigationStart = 0;
|
||||
url = URL
|
||||
app.send(SetPageLocation(url, document.referrer, navigationStart))
|
||||
navigationStart = 0
|
||||
}
|
||||
});
|
||||
})
|
||||
|
||||
const sendSetViewportSize = app.safe(() => {
|
||||
const { innerWidth, innerHeight } = window;
|
||||
const { innerWidth, innerHeight } = window
|
||||
if (innerWidth !== width || innerHeight !== height) {
|
||||
width = innerWidth;
|
||||
height = innerHeight;
|
||||
app.send(new SetViewportSize(width, height));
|
||||
width = innerWidth
|
||||
height = innerHeight
|
||||
app.send(SetViewportSize(width, height))
|
||||
}
|
||||
});
|
||||
})
|
||||
|
||||
const sendSetPageVisibility =
|
||||
document.hidden === undefined
|
||||
? Function.prototype
|
||||
: app.safe(() => app.send(new SetPageVisibility(document.hidden)));
|
||||
: app.safe(() => app.send(SetPageVisibility(document.hidden)))
|
||||
|
||||
app.attachStartCallback(() => {
|
||||
url = '';
|
||||
width = height = -1;
|
||||
sendSetPageLocation();
|
||||
sendSetViewportSize();
|
||||
sendSetPageVisibility();
|
||||
});
|
||||
url = ''
|
||||
width = height = -1
|
||||
sendSetPageLocation()
|
||||
sendSetViewportSize()
|
||||
sendSetPageVisibility()
|
||||
})
|
||||
|
||||
if (document.hidden !== undefined) {
|
||||
app.attachEventListener(
|
||||
|
|
@ -43,9 +43,9 @@ export default function (app: App): void {
|
|||
sendSetPageVisibility as EventListener,
|
||||
false,
|
||||
false,
|
||||
);
|
||||
)
|
||||
}
|
||||
|
||||
app.ticker.attach(sendSetPageLocation, 1, false);
|
||||
app.ticker.attach(sendSetViewportSize, 5, false);
|
||||
app.ticker.attach(sendSetPageLocation, 1, false)
|
||||
app.ticker.attach(sendSetViewportSize, 5, false)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,65 +1,65 @@
|
|||
export function timestamp(): number {
|
||||
return Math.round(performance.now()) + performance.timing.navigationStart;
|
||||
return Math.round(performance.now()) + performance.timing.navigationStart
|
||||
}
|
||||
|
||||
export const stars: (str: string) => string =
|
||||
'repeat' in String.prototype
|
||||
? (str: string): string => '*'.repeat(str.length)
|
||||
: (str: string): string => str.replace(/./g, '*');
|
||||
: (str: string): string => str.replace(/./g, '*')
|
||||
|
||||
export function normSpaces(str: string): string {
|
||||
return str.trim().replace(/\s+/g, ' ');
|
||||
return str.trim().replace(/\s+/g, ' ')
|
||||
}
|
||||
|
||||
// isAbsoluteUrl regexp: /^([a-z][a-z\d\+\-\.]*:)?\/\//i.test(url)
|
||||
export function isURL(s: string): boolean {
|
||||
return s.startsWith('https://') || s.startsWith('http://');
|
||||
return s.startsWith('https://') || s.startsWith('http://')
|
||||
}
|
||||
|
||||
export const IN_BROWSER = !(typeof window === 'undefined');
|
||||
export const IN_BROWSER = !(typeof window === 'undefined')
|
||||
|
||||
// TODO: JOIN IT WITH LOGGER somehow (use logging decorators?); Don't forget about index.js loggin when there is no logger instance.
|
||||
|
||||
export const DOCS_HOST = 'https://docs.openreplay.com';
|
||||
export const DOCS_HOST = 'https://docs.openreplay.com'
|
||||
|
||||
const warnedFeatures: { [key: string]: boolean } = {};
|
||||
const warnedFeatures: { [key: string]: boolean } = {}
|
||||
export function deprecationWarn(nameOfFeature: string, useInstead: string, docsPath = '/'): void {
|
||||
if (warnedFeatures[nameOfFeature]) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
console.warn(
|
||||
`OpenReplay: ${nameOfFeature} is deprecated. ${
|
||||
useInstead ? `Please, use ${useInstead} instead.` : ''
|
||||
} Visit ${DOCS_HOST}${docsPath} for more information.`,
|
||||
);
|
||||
warnedFeatures[nameOfFeature] = true;
|
||||
)
|
||||
warnedFeatures[nameOfFeature] = true
|
||||
}
|
||||
|
||||
export function getLabelAttribute(e: Element): string | null {
|
||||
let value = e.getAttribute('data-openreplay-label');
|
||||
let value = e.getAttribute('data-openreplay-label')
|
||||
if (value !== null) {
|
||||
return value;
|
||||
return value
|
||||
}
|
||||
value = e.getAttribute('data-asayer-label');
|
||||
value = e.getAttribute('data-asayer-label')
|
||||
if (value !== null) {
|
||||
deprecationWarn('"data-asayer-label" attribute', '"data-openreplay-label" attribute', '/');
|
||||
deprecationWarn('"data-asayer-label" attribute', '"data-openreplay-label" attribute', '/')
|
||||
}
|
||||
return value;
|
||||
return value
|
||||
}
|
||||
|
||||
export function hasOpenreplayAttribute(e: Element, name: string): boolean {
|
||||
const newName = `data-openreplay-${name}`;
|
||||
const newName = `data-openreplay-${name}`
|
||||
if (e.hasAttribute(newName)) {
|
||||
return true;
|
||||
return true
|
||||
}
|
||||
const oldName = `data-asayer-${name}`;
|
||||
const oldName = `data-asayer-${name}`
|
||||
if (e.hasAttribute(oldName)) {
|
||||
deprecationWarn(
|
||||
`"${oldName}" attribute`,
|
||||
`"${newName}" attribute`,
|
||||
'/installation/sanitize-data',
|
||||
);
|
||||
return true;
|
||||
)
|
||||
return true
|
||||
}
|
||||
return false;
|
||||
return false
|
||||
}
|
||||
|
|
|
|||
284
tracker/tracker/src/main/vendors/finder/finder.ts
vendored
284
tracker/tracker/src/main/vendors/finder/finder.ts
vendored
|
|
@ -1,10 +1,10 @@
|
|||
type Node = {
|
||||
name: string;
|
||||
penalty: number;
|
||||
level?: number;
|
||||
};
|
||||
name: string
|
||||
penalty: number
|
||||
level?: number
|
||||
}
|
||||
|
||||
type Path = Node[];
|
||||
type Path = Node[]
|
||||
|
||||
enum Limit {
|
||||
All,
|
||||
|
|
@ -13,27 +13,27 @@ enum Limit {
|
|||
}
|
||||
|
||||
export type Options = {
|
||||
root: Element;
|
||||
idName: (name: string) => boolean;
|
||||
className: (name: string) => boolean;
|
||||
tagName: (name: string) => boolean;
|
||||
attr: (name: string, value: string) => boolean;
|
||||
seedMinLength: number;
|
||||
optimizedMinLength: number;
|
||||
threshold: number;
|
||||
maxNumberOfTries: number;
|
||||
};
|
||||
root: Element
|
||||
idName: (name: string) => boolean
|
||||
className: (name: string) => boolean
|
||||
tagName: (name: string) => boolean
|
||||
attr: (name: string, value: string) => boolean
|
||||
seedMinLength: number
|
||||
optimizedMinLength: number
|
||||
threshold: number
|
||||
maxNumberOfTries: number
|
||||
}
|
||||
|
||||
let config: Options;
|
||||
let config: Options
|
||||
|
||||
let rootDocument: Document | Element;
|
||||
let rootDocument: Document | Element
|
||||
export function finder(input: Element, options?: Partial<Options>) {
|
||||
if (input.nodeType !== Node.ELEMENT_NODE) {
|
||||
throw new Error("Can't generate CSS selector for non-element node type.");
|
||||
throw new Error("Can't generate CSS selector for non-element node type.")
|
||||
}
|
||||
|
||||
if ('html' === input.tagName.toLowerCase()) {
|
||||
return 'html';
|
||||
return 'html'
|
||||
}
|
||||
|
||||
const defaults: Options = {
|
||||
|
|
@ -46,264 +46,264 @@ export function finder(input: Element, options?: Partial<Options>) {
|
|||
optimizedMinLength: 2,
|
||||
threshold: 1000,
|
||||
maxNumberOfTries: 10000,
|
||||
};
|
||||
}
|
||||
|
||||
config = { ...defaults, ...options };
|
||||
config = { ...defaults, ...options }
|
||||
|
||||
rootDocument = findRootDocument(config.root, defaults);
|
||||
rootDocument = findRootDocument(config.root, defaults)
|
||||
|
||||
let path = bottomUpSearch(input, Limit.All, () =>
|
||||
bottomUpSearch(input, Limit.Two, () => bottomUpSearch(input, Limit.One)),
|
||||
);
|
||||
)
|
||||
|
||||
if (path) {
|
||||
const optimized = sort(optimize(path, input));
|
||||
const optimized = sort(optimize(path, input))
|
||||
|
||||
if (optimized.length > 0) {
|
||||
path = optimized[0];
|
||||
path = optimized[0]
|
||||
}
|
||||
|
||||
return selector(path);
|
||||
return selector(path)
|
||||
} else {
|
||||
throw new Error('Selector was not found.');
|
||||
throw new Error('Selector was not found.')
|
||||
}
|
||||
}
|
||||
|
||||
function findRootDocument(rootNode: Element | Document, defaults: Options) {
|
||||
if (rootNode.nodeType === Node.DOCUMENT_NODE) {
|
||||
return rootNode;
|
||||
return rootNode
|
||||
}
|
||||
if (rootNode === defaults.root) {
|
||||
return rootNode.ownerDocument;
|
||||
return rootNode.ownerDocument
|
||||
}
|
||||
return rootNode;
|
||||
return rootNode
|
||||
}
|
||||
|
||||
function bottomUpSearch(input: Element, limit: Limit, fallback?: () => Path | null): Path | null {
|
||||
let path: Path | null = null;
|
||||
const stack: Node[][] = [];
|
||||
let current: Element | null = input;
|
||||
let i = 0;
|
||||
let path: Path | null = null
|
||||
const stack: Node[][] = []
|
||||
let current: Element | null = input
|
||||
let i = 0
|
||||
|
||||
while (current && current !== config.root.parentElement) {
|
||||
let level: Node[] = maybe(id(current)) ||
|
||||
maybe(...attr(current)) ||
|
||||
maybe(...classNames(current)) ||
|
||||
maybe(tagName(current)) || [any()];
|
||||
maybe(tagName(current)) || [any()]
|
||||
|
||||
const nth = index(current);
|
||||
const nth = index(current)
|
||||
|
||||
if (limit === Limit.All) {
|
||||
if (nth) {
|
||||
level = level.concat(level.filter(dispensableNth).map((node) => nthChild(node, nth)));
|
||||
level = level.concat(level.filter(dispensableNth).map((node) => nthChild(node, nth)))
|
||||
}
|
||||
} else if (limit === Limit.Two) {
|
||||
level = level.slice(0, 1);
|
||||
level = level.slice(0, 1)
|
||||
|
||||
if (nth) {
|
||||
level = level.concat(level.filter(dispensableNth).map((node) => nthChild(node, nth)));
|
||||
level = level.concat(level.filter(dispensableNth).map((node) => nthChild(node, nth)))
|
||||
}
|
||||
} else if (limit === Limit.One) {
|
||||
const [node] = (level = level.slice(0, 1));
|
||||
const [node] = (level = level.slice(0, 1))
|
||||
|
||||
if (nth && dispensableNth(node)) {
|
||||
level = [nthChild(node, nth)];
|
||||
level = [nthChild(node, nth)]
|
||||
}
|
||||
}
|
||||
|
||||
for (const node of level) {
|
||||
node.level = i;
|
||||
node.level = i
|
||||
}
|
||||
|
||||
stack.push(level);
|
||||
stack.push(level)
|
||||
|
||||
if (stack.length >= config.seedMinLength) {
|
||||
path = findUniquePath(stack, fallback);
|
||||
path = findUniquePath(stack, fallback)
|
||||
if (path) {
|
||||
break;
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
current = current.parentElement;
|
||||
i++;
|
||||
current = current.parentElement
|
||||
i++
|
||||
}
|
||||
|
||||
if (!path) {
|
||||
path = findUniquePath(stack, fallback);
|
||||
path = findUniquePath(stack, fallback)
|
||||
}
|
||||
|
||||
return path;
|
||||
return path
|
||||
}
|
||||
|
||||
function findUniquePath(stack: Node[][], fallback?: () => Path | null): Path | null {
|
||||
const paths = sort(combinations(stack));
|
||||
const paths = sort(combinations(stack))
|
||||
|
||||
if (paths.length > config.threshold) {
|
||||
return fallback ? fallback() : null;
|
||||
return fallback ? fallback() : null
|
||||
}
|
||||
|
||||
for (const candidate of paths) {
|
||||
if (unique(candidate)) {
|
||||
return candidate;
|
||||
return candidate
|
||||
}
|
||||
}
|
||||
|
||||
return null;
|
||||
return null
|
||||
}
|
||||
|
||||
function selector(path: Path): string {
|
||||
let node = path[0];
|
||||
let query = node.name;
|
||||
let node = path[0]
|
||||
let query = node.name
|
||||
for (let i = 1; i < path.length; i++) {
|
||||
const level = path[i].level || 0;
|
||||
const level = path[i].level || 0
|
||||
|
||||
if (node.level === level - 1) {
|
||||
query = `${path[i].name} > ${query}`;
|
||||
query = `${path[i].name} > ${query}`
|
||||
} else {
|
||||
query = `${path[i].name} ${query}`;
|
||||
query = `${path[i].name} ${query}`
|
||||
}
|
||||
|
||||
node = path[i];
|
||||
node = path[i]
|
||||
}
|
||||
return query;
|
||||
return query
|
||||
}
|
||||
|
||||
function penalty(path: Path): number {
|
||||
return path.map((node) => node.penalty).reduce((acc, i) => acc + i, 0);
|
||||
return path.map((node) => node.penalty).reduce((acc, i) => acc + i, 0)
|
||||
}
|
||||
|
||||
function unique(path: Path) {
|
||||
switch (rootDocument.querySelectorAll(selector(path)).length) {
|
||||
case 0:
|
||||
throw new Error(`Can't select any node with this selector: ${selector(path)}`);
|
||||
throw new Error(`Can't select any node with this selector: ${selector(path)}`)
|
||||
case 1:
|
||||
return true;
|
||||
return true
|
||||
default:
|
||||
return false;
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
function id(input: Element): Node | null {
|
||||
const elementId = input.getAttribute('id');
|
||||
const elementId = input.getAttribute('id')
|
||||
if (elementId && config.idName(elementId)) {
|
||||
return {
|
||||
name: '#' + cssesc(elementId, { isIdentifier: true }),
|
||||
penalty: 0,
|
||||
};
|
||||
}
|
||||
}
|
||||
return null;
|
||||
return null
|
||||
}
|
||||
|
||||
function attr(input: Element): Node[] {
|
||||
const attrs = Array.from(input.attributes).filter((attr) => config.attr(attr.name, attr.value));
|
||||
const attrs = Array.from(input.attributes).filter((attr) => config.attr(attr.name, attr.value))
|
||||
|
||||
return attrs.map(
|
||||
(attr): Node => ({
|
||||
name: '[' + cssesc(attr.name, { isIdentifier: true }) + '="' + cssesc(attr.value) + '"]',
|
||||
penalty: 0.5,
|
||||
}),
|
||||
);
|
||||
)
|
||||
}
|
||||
|
||||
function classNames(input: Element): Node[] {
|
||||
const names = Array.from(input.classList).filter(config.className);
|
||||
const names = Array.from(input.classList).filter(config.className)
|
||||
|
||||
return names.map(
|
||||
(name): Node => ({
|
||||
name: '.' + cssesc(name, { isIdentifier: true }),
|
||||
penalty: 1,
|
||||
}),
|
||||
);
|
||||
)
|
||||
}
|
||||
|
||||
function tagName(input: Element): Node | null {
|
||||
const name = input.tagName.toLowerCase();
|
||||
const name = input.tagName.toLowerCase()
|
||||
if (config.tagName(name)) {
|
||||
return {
|
||||
name,
|
||||
penalty: 2,
|
||||
};
|
||||
}
|
||||
}
|
||||
return null;
|
||||
return null
|
||||
}
|
||||
|
||||
function any(): Node {
|
||||
return {
|
||||
name: '*',
|
||||
penalty: 3,
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
function index(input: Element): number | null {
|
||||
const parent = input.parentNode;
|
||||
const parent = input.parentNode
|
||||
if (!parent) {
|
||||
return null;
|
||||
return null
|
||||
}
|
||||
|
||||
let child = parent.firstChild;
|
||||
let child = parent.firstChild
|
||||
if (!child) {
|
||||
return null;
|
||||
return null
|
||||
}
|
||||
|
||||
let i = 0;
|
||||
let i = 0
|
||||
while (child) {
|
||||
if (child.nodeType === Node.ELEMENT_NODE) {
|
||||
i++;
|
||||
i++
|
||||
}
|
||||
|
||||
if (child === input) {
|
||||
break;
|
||||
break
|
||||
}
|
||||
|
||||
child = child.nextSibling;
|
||||
child = child.nextSibling
|
||||
}
|
||||
|
||||
return i;
|
||||
return i
|
||||
}
|
||||
|
||||
function nthChild(node: Node, i: number): Node {
|
||||
return {
|
||||
name: node.name + `:nth-child(${i})`,
|
||||
penalty: node.penalty + 1,
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
function dispensableNth(node: Node) {
|
||||
return node.name !== 'html' && !node.name.startsWith('#');
|
||||
return node.name !== 'html' && !node.name.startsWith('#')
|
||||
}
|
||||
|
||||
function maybe(...level: (Node | null)[]): Node[] | null {
|
||||
const list = level.filter(notEmpty);
|
||||
const list = level.filter(notEmpty)
|
||||
if (list.length > 0) {
|
||||
return list;
|
||||
return list
|
||||
}
|
||||
return null;
|
||||
return null
|
||||
}
|
||||
|
||||
function notEmpty<T>(value: T | null | undefined): value is T {
|
||||
return value !== null && value !== undefined;
|
||||
return value !== null && value !== undefined
|
||||
}
|
||||
|
||||
function combinations(stack: Node[][], path: Node[] = []): Node[][] {
|
||||
const paths: Node[][] = [];
|
||||
const paths: Node[][] = []
|
||||
if (stack.length > 0) {
|
||||
for (const node of stack[0]) {
|
||||
paths.push(...combinations(stack.slice(1, stack.length), path.concat(node)));
|
||||
paths.push(...combinations(stack.slice(1, stack.length), path.concat(node)))
|
||||
}
|
||||
} else {
|
||||
paths.push(path);
|
||||
paths.push(path)
|
||||
}
|
||||
return paths;
|
||||
return paths
|
||||
}
|
||||
|
||||
function sort(paths: Iterable<Path>): Path[] {
|
||||
return Array.from(paths).sort((a, b) => penalty(a) - penalty(b));
|
||||
return Array.from(paths).sort((a, b) => penalty(a) - penalty(b))
|
||||
}
|
||||
|
||||
type Scope = {
|
||||
counter: number;
|
||||
visited: Map<string, boolean>;
|
||||
};
|
||||
counter: number
|
||||
visited: Map<string, boolean>
|
||||
}
|
||||
|
||||
function optimize(
|
||||
path: Path,
|
||||
|
|
@ -313,103 +313,103 @@ function optimize(
|
|||
visited: new Map<string, boolean>(),
|
||||
},
|
||||
): Node[][] {
|
||||
const paths: Node[][] = [];
|
||||
const paths: Node[][] = []
|
||||
if (path.length > 2 && path.length > config.optimizedMinLength) {
|
||||
for (let i = 1; i < path.length - 1; i++) {
|
||||
if (scope.counter > config.maxNumberOfTries) {
|
||||
return paths; // Okay At least I tried!
|
||||
return paths // Okay At least I tried!
|
||||
}
|
||||
scope.counter += 1;
|
||||
const newPath = [...path];
|
||||
newPath.splice(i, 1);
|
||||
const newPathKey = selector(newPath);
|
||||
scope.counter += 1
|
||||
const newPath = [...path]
|
||||
newPath.splice(i, 1)
|
||||
const newPathKey = selector(newPath)
|
||||
if (scope.visited.has(newPathKey)) {
|
||||
return paths;
|
||||
return paths
|
||||
}
|
||||
if (unique(newPath) && same(newPath, input)) {
|
||||
paths.push(newPath);
|
||||
scope.visited.set(newPathKey, true);
|
||||
paths.push(...optimize(newPath, input, scope));
|
||||
paths.push(newPath)
|
||||
scope.visited.set(newPathKey, true)
|
||||
paths.push(...optimize(newPath, input, scope))
|
||||
}
|
||||
}
|
||||
}
|
||||
return paths;
|
||||
return paths
|
||||
}
|
||||
|
||||
function same(path: Path, input: Element) {
|
||||
return rootDocument.querySelector(selector(path)) === input;
|
||||
return rootDocument.querySelector(selector(path)) === input
|
||||
}
|
||||
|
||||
const regexAnySingleEscape = /[ -,\.\/:-@\[-\^`\{-~]/;
|
||||
const regexSingleEscape = /[ -,\.\/:-@\[\]\^`\{-~]/;
|
||||
const regexExcessiveSpaces = /(^|\\+)?(\\[A-F0-9]{1,6})\x20(?![a-fA-F0-9\x20])/g;
|
||||
const regexAnySingleEscape = /[ -,\.\/:-@\[-\^`\{-~]/
|
||||
const regexSingleEscape = /[ -,\.\/:-@\[\]\^`\{-~]/
|
||||
const regexExcessiveSpaces = /(^|\\+)?(\\[A-F0-9]{1,6})\x20(?![a-fA-F0-9\x20])/g
|
||||
|
||||
const defaultOptions = {
|
||||
escapeEverything: false,
|
||||
isIdentifier: false,
|
||||
quotes: 'single',
|
||||
wrap: false,
|
||||
};
|
||||
}
|
||||
|
||||
function cssesc(string: string, opt: Partial<typeof defaultOptions> = {}) {
|
||||
const options = { ...defaultOptions, ...opt };
|
||||
const options = { ...defaultOptions, ...opt }
|
||||
if (options.quotes != 'single' && options.quotes != 'double') {
|
||||
options.quotes = 'single';
|
||||
options.quotes = 'single'
|
||||
}
|
||||
const quote = options.quotes == 'double' ? '"' : "'";
|
||||
const isIdentifier = options.isIdentifier;
|
||||
const quote = options.quotes == 'double' ? '"' : "'"
|
||||
const isIdentifier = options.isIdentifier
|
||||
|
||||
const firstChar = string.charAt(0);
|
||||
let output = '';
|
||||
let counter = 0;
|
||||
const length = string.length;
|
||||
const firstChar = string.charAt(0)
|
||||
let output = ''
|
||||
let counter = 0
|
||||
const length = string.length
|
||||
while (counter < length) {
|
||||
const character = string.charAt(counter++);
|
||||
let codePoint = character.charCodeAt(0);
|
||||
let value: string | undefined = void 0;
|
||||
const character = string.charAt(counter++)
|
||||
let codePoint = character.charCodeAt(0)
|
||||
let value: string | undefined = void 0
|
||||
// If it’s not a printable ASCII character…
|
||||
if (codePoint < 0x20 || codePoint > 0x7e) {
|
||||
if (codePoint >= 0xd800 && codePoint <= 0xdbff && counter < length) {
|
||||
// It’s a high surrogate, and there is a next character.
|
||||
const extra = string.charCodeAt(counter++);
|
||||
const extra = string.charCodeAt(counter++)
|
||||
if ((extra & 0xfc00) == 0xdc00) {
|
||||
// next character is low surrogate
|
||||
codePoint = ((codePoint & 0x3ff) << 10) + (extra & 0x3ff) + 0x10000;
|
||||
codePoint = ((codePoint & 0x3ff) << 10) + (extra & 0x3ff) + 0x10000
|
||||
} else {
|
||||
// It’s an unmatched surrogate; only append this code unit, in case
|
||||
// the next code unit is the high surrogate of a surrogate pair.
|
||||
counter--;
|
||||
counter--
|
||||
}
|
||||
}
|
||||
value = '\\' + codePoint.toString(16).toUpperCase() + ' ';
|
||||
value = '\\' + codePoint.toString(16).toUpperCase() + ' '
|
||||
} else {
|
||||
if (options.escapeEverything) {
|
||||
if (regexAnySingleEscape.test(character)) {
|
||||
value = '\\' + character;
|
||||
value = '\\' + character
|
||||
} else {
|
||||
value = '\\' + codePoint.toString(16).toUpperCase() + ' ';
|
||||
value = '\\' + codePoint.toString(16).toUpperCase() + ' '
|
||||
}
|
||||
} else if (/[\t\n\f\r\x0B]/.test(character)) {
|
||||
value = '\\' + codePoint.toString(16).toUpperCase() + ' ';
|
||||
value = '\\' + codePoint.toString(16).toUpperCase() + ' '
|
||||
} else if (
|
||||
character == '\\' ||
|
||||
(!isIdentifier &&
|
||||
((character == '"' && quote == character) || (character == "'" && quote == character))) ||
|
||||
(isIdentifier && regexSingleEscape.test(character))
|
||||
) {
|
||||
value = '\\' + character;
|
||||
value = '\\' + character
|
||||
} else {
|
||||
value = character;
|
||||
value = character
|
||||
}
|
||||
}
|
||||
output += value;
|
||||
output += value
|
||||
}
|
||||
|
||||
if (isIdentifier) {
|
||||
if (/^-[-\d]/.test(output)) {
|
||||
output = '\\-' + output.slice(1);
|
||||
output = '\\-' + output.slice(1)
|
||||
} else if (/\d/.test(firstChar)) {
|
||||
output = '\\3' + firstChar + ' ' + output.slice(1);
|
||||
output = '\\3' + firstChar + ' ' + output.slice(1)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -419,14 +419,14 @@ function cssesc(string: string, opt: Partial<typeof defaultOptions> = {}) {
|
|||
output = output.replace(regexExcessiveSpaces, function ($0, $1, $2) {
|
||||
if ($1 && $1.length % 2) {
|
||||
// It’s not safe to remove the space, so don’t.
|
||||
return $0;
|
||||
return $0
|
||||
}
|
||||
// Strip the space.
|
||||
return ($1 || '') + $2;
|
||||
});
|
||||
return ($1 || '') + $2
|
||||
})
|
||||
|
||||
if (!isIdentifier && options.wrap) {
|
||||
return quote + output + quote;
|
||||
return quote + output + quote
|
||||
}
|
||||
return output;
|
||||
return output
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,74 +1,123 @@
|
|||
import type Message from '../common/messages.js';
|
||||
import PrimitiveWriter from './PrimitiveWriter.js';
|
||||
import { BatchMeta, Timestamp } from '../common/messages.js';
|
||||
import type Message from '../common/messages.gen.js'
|
||||
import * as Messages from '../common/messages.gen.js'
|
||||
import MessageEncoder from './MessageEncoder.gen.js'
|
||||
|
||||
const SIZE_BYTES = 3
|
||||
const MAX_M_SIZE = (1 << (SIZE_BYTES * 8)) - 1
|
||||
|
||||
export default class BatchWriter {
|
||||
private nextIndex = 0;
|
||||
private beaconSize = 2 * 1e5; // Default 200kB
|
||||
private writer = new PrimitiveWriter(this.beaconSize);
|
||||
private isEmpty = true;
|
||||
private nextIndex = 0
|
||||
private beaconSize = 2 * 1e5 // Default 200kB
|
||||
private encoder = new MessageEncoder(this.beaconSize)
|
||||
private readonly sizeBuffer = new Uint8Array(SIZE_BYTES)
|
||||
private isEmpty = true
|
||||
|
||||
constructor(
|
||||
private readonly pageNo: number,
|
||||
private timestamp: number,
|
||||
private url: string,
|
||||
private readonly onBatch: (batch: Uint8Array) => void,
|
||||
) {
|
||||
this.prepare();
|
||||
this.prepare()
|
||||
}
|
||||
|
||||
private writeType(m: Message): boolean {
|
||||
return this.encoder.uint(m[0])
|
||||
}
|
||||
private writeFields(m: Message): boolean {
|
||||
return this.encoder.encode(m)
|
||||
}
|
||||
private writeSizeAt(size: number, offset: number): void {
|
||||
//boolean?
|
||||
for (let i = 0; i < SIZE_BYTES; i++) {
|
||||
this.sizeBuffer[i] = size >> (i * 8) // BigEndian
|
||||
}
|
||||
this.encoder.set(this.sizeBuffer, offset)
|
||||
}
|
||||
|
||||
private prepare(): void {
|
||||
if (!this.writer.isEmpty()) {
|
||||
return;
|
||||
if (!this.encoder.isEmpty()) {
|
||||
return
|
||||
}
|
||||
new BatchMeta(this.pageNo, this.nextIndex, this.timestamp).encode(this.writer);
|
||||
|
||||
// MBTODO: move service-messages creation to webworker
|
||||
const batchMetadata: Messages.BatchMetadata = [
|
||||
Messages.Type.BatchMetadata,
|
||||
1,
|
||||
this.pageNo,
|
||||
this.nextIndex,
|
||||
this.timestamp,
|
||||
this.url,
|
||||
]
|
||||
this.writeType(batchMetadata)
|
||||
this.writeFields(batchMetadata)
|
||||
this.isEmpty = true
|
||||
}
|
||||
|
||||
private write(message: Message): boolean {
|
||||
const wasWritten = message.encode(this.writer);
|
||||
private writeWithSize(message: Message): boolean {
|
||||
const e = this.encoder
|
||||
if (!this.writeType(message) || !e.skip(SIZE_BYTES)) {
|
||||
// app.debug.log
|
||||
return false
|
||||
}
|
||||
const startOffset = e.getCurrentOffset()
|
||||
const wasWritten = this.writeFields(message)
|
||||
if (wasWritten) {
|
||||
this.isEmpty = false;
|
||||
this.writer.checkpoint();
|
||||
this.nextIndex++;
|
||||
const endOffset = e.getCurrentOffset()
|
||||
const size = endOffset - startOffset
|
||||
if (size > MAX_M_SIZE) {
|
||||
console.warn('OpenReplay: max message size overflow.')
|
||||
return false
|
||||
}
|
||||
this.writeSizeAt(size, startOffset - SIZE_BYTES)
|
||||
|
||||
e.checkpoint()
|
||||
this.isEmpty = this.isEmpty && message[0] === Messages.Type.Timestamp
|
||||
this.nextIndex++
|
||||
}
|
||||
return wasWritten;
|
||||
// app.debug.log
|
||||
return wasWritten
|
||||
}
|
||||
|
||||
private beaconSizeLimit = 1e6;
|
||||
private beaconSizeLimit = 1e6
|
||||
setBeaconSizeLimit(limit: number) {
|
||||
this.beaconSizeLimit = limit;
|
||||
this.beaconSizeLimit = limit
|
||||
}
|
||||
|
||||
writeMessage(message: Message) {
|
||||
if (message instanceof Timestamp) {
|
||||
this.timestamp = (<any>message).timestamp;
|
||||
if (message[0] === Messages.Type.Timestamp) {
|
||||
this.timestamp = message[1] // .timestamp
|
||||
}
|
||||
while (!this.write(message)) {
|
||||
this.finaliseBatch();
|
||||
if (message[0] === Messages.Type.SetPageLocation) {
|
||||
this.url = message[1] // .url
|
||||
}
|
||||
if (this.writeWithSize(message)) {
|
||||
return
|
||||
}
|
||||
this.finaliseBatch()
|
||||
while (!this.writeWithSize(message)) {
|
||||
if (this.beaconSize === this.beaconSizeLimit) {
|
||||
console.warn('OpenReplay: beacon size overflow. Skipping large message.');
|
||||
this.writer.reset();
|
||||
this.prepare();
|
||||
this.isEmpty = true;
|
||||
return;
|
||||
console.warn('OpenReplay: beacon size overflow. Skipping large message.', message)
|
||||
this.encoder.reset()
|
||||
this.prepare()
|
||||
return
|
||||
}
|
||||
// MBTODO: tempWriter for one message?
|
||||
this.beaconSize = Math.min(this.beaconSize * 2, this.beaconSizeLimit);
|
||||
this.writer = new PrimitiveWriter(this.beaconSize);
|
||||
this.prepare();
|
||||
this.isEmpty = true;
|
||||
this.beaconSize = Math.min(this.beaconSize * 2, this.beaconSizeLimit)
|
||||
this.encoder = new MessageEncoder(this.beaconSize)
|
||||
this.prepare()
|
||||
}
|
||||
}
|
||||
|
||||
finaliseBatch() {
|
||||
if (this.isEmpty) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
this.onBatch(this.writer.flush());
|
||||
this.prepare();
|
||||
this.isEmpty = true;
|
||||
this.onBatch(this.encoder.flush())
|
||||
this.prepare()
|
||||
}
|
||||
|
||||
clean() {
|
||||
this.writer.reset();
|
||||
this.encoder.reset()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
216
tracker/tracker/src/webworker/MessageEncoder.gen.ts
Normal file
216
tracker/tracker/src/webworker/MessageEncoder.gen.ts
Normal file
|
|
@ -0,0 +1,216 @@
|
|||
// Auto-generated, do not edit
|
||||
|
||||
import * as Messages from '../common/messages.gen.js'
|
||||
import Message from '../common/messages.gen.js'
|
||||
import PrimitiveEncoder from './PrimitiveEncoder.js'
|
||||
|
||||
|
||||
export default class MessageEncoder extends PrimitiveEncoder {
|
||||
encode(msg: Message): boolean {
|
||||
switch(msg[0]) {
|
||||
|
||||
case Messages.Type.BatchMetadata:
|
||||
return this.uint(msg[1]) && this.uint(msg[2]) && this.uint(msg[3]) && this.int(msg[4]) && this.string(msg[5])
|
||||
break
|
||||
|
||||
case Messages.Type.PartitionedMessage:
|
||||
return this.uint(msg[1]) && this.uint(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.Timestamp:
|
||||
return this.uint(msg[1])
|
||||
break
|
||||
|
||||
case Messages.Type.SetPageLocation:
|
||||
return this.string(msg[1]) && this.string(msg[2]) && this.uint(msg[3])
|
||||
break
|
||||
|
||||
case Messages.Type.SetViewportSize:
|
||||
return this.uint(msg[1]) && this.uint(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.SetViewportScroll:
|
||||
return this.int(msg[1]) && this.int(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.CreateDocument:
|
||||
return true
|
||||
break
|
||||
|
||||
case Messages.Type.CreateElementNode:
|
||||
return this.uint(msg[1]) && this.uint(msg[2]) && this.uint(msg[3]) && this.string(msg[4]) && this.boolean(msg[5])
|
||||
break
|
||||
|
||||
case Messages.Type.CreateTextNode:
|
||||
return this.uint(msg[1]) && this.uint(msg[2]) && this.uint(msg[3])
|
||||
break
|
||||
|
||||
case Messages.Type.MoveNode:
|
||||
return this.uint(msg[1]) && this.uint(msg[2]) && this.uint(msg[3])
|
||||
break
|
||||
|
||||
case Messages.Type.RemoveNode:
|
||||
return this.uint(msg[1])
|
||||
break
|
||||
|
||||
case Messages.Type.SetNodeAttribute:
|
||||
return this.uint(msg[1]) && this.string(msg[2]) && this.string(msg[3])
|
||||
break
|
||||
|
||||
case Messages.Type.RemoveNodeAttribute:
|
||||
return this.uint(msg[1]) && this.string(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.SetNodeData:
|
||||
return this.uint(msg[1]) && this.string(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.SetNodeScroll:
|
||||
return this.uint(msg[1]) && this.int(msg[2]) && this.int(msg[3])
|
||||
break
|
||||
|
||||
case Messages.Type.SetInputTarget:
|
||||
return this.uint(msg[1]) && this.string(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.SetInputValue:
|
||||
return this.uint(msg[1]) && this.string(msg[2]) && this.int(msg[3])
|
||||
break
|
||||
|
||||
case Messages.Type.SetInputChecked:
|
||||
return this.uint(msg[1]) && this.boolean(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.MouseMove:
|
||||
return this.uint(msg[1]) && this.uint(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.ConsoleLog:
|
||||
return this.string(msg[1]) && this.string(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.PageLoadTiming:
|
||||
return this.uint(msg[1]) && this.uint(msg[2]) && this.uint(msg[3]) && this.uint(msg[4]) && this.uint(msg[5]) && this.uint(msg[6]) && this.uint(msg[7]) && this.uint(msg[8]) && this.uint(msg[9])
|
||||
break
|
||||
|
||||
case Messages.Type.PageRenderTiming:
|
||||
return this.uint(msg[1]) && this.uint(msg[2]) && this.uint(msg[3])
|
||||
break
|
||||
|
||||
case Messages.Type.JSException:
|
||||
return this.string(msg[1]) && this.string(msg[2]) && this.string(msg[3])
|
||||
break
|
||||
|
||||
case Messages.Type.RawCustomEvent:
|
||||
return this.string(msg[1]) && this.string(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.UserID:
|
||||
return this.string(msg[1])
|
||||
break
|
||||
|
||||
case Messages.Type.UserAnonymousID:
|
||||
return this.string(msg[1])
|
||||
break
|
||||
|
||||
case Messages.Type.Metadata:
|
||||
return this.string(msg[1]) && this.string(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.CSSInsertRule:
|
||||
return this.uint(msg[1]) && this.string(msg[2]) && this.uint(msg[3])
|
||||
break
|
||||
|
||||
case Messages.Type.CSSDeleteRule:
|
||||
return this.uint(msg[1]) && this.uint(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.Fetch:
|
||||
return this.string(msg[1]) && this.string(msg[2]) && this.string(msg[3]) && this.string(msg[4]) && this.uint(msg[5]) && this.uint(msg[6]) && this.uint(msg[7])
|
||||
break
|
||||
|
||||
case Messages.Type.Profiler:
|
||||
return this.string(msg[1]) && this.uint(msg[2]) && this.string(msg[3]) && this.string(msg[4])
|
||||
break
|
||||
|
||||
case Messages.Type.OTable:
|
||||
return this.string(msg[1]) && this.string(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.StateAction:
|
||||
return this.string(msg[1])
|
||||
break
|
||||
|
||||
case Messages.Type.Redux:
|
||||
return this.string(msg[1]) && this.string(msg[2]) && this.uint(msg[3])
|
||||
break
|
||||
|
||||
case Messages.Type.Vuex:
|
||||
return this.string(msg[1]) && this.string(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.MobX:
|
||||
return this.string(msg[1]) && this.string(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.NgRx:
|
||||
return this.string(msg[1]) && this.string(msg[2]) && this.uint(msg[3])
|
||||
break
|
||||
|
||||
case Messages.Type.GraphQL:
|
||||
return this.string(msg[1]) && this.string(msg[2]) && this.string(msg[3]) && this.string(msg[4])
|
||||
break
|
||||
|
||||
case Messages.Type.PerformanceTrack:
|
||||
return this.int(msg[1]) && this.int(msg[2]) && this.uint(msg[3]) && this.uint(msg[4])
|
||||
break
|
||||
|
||||
case Messages.Type.ResourceTiming:
|
||||
return this.uint(msg[1]) && this.uint(msg[2]) && this.uint(msg[3]) && this.uint(msg[4]) && this.uint(msg[5]) && this.uint(msg[6]) && this.string(msg[7]) && this.string(msg[8])
|
||||
break
|
||||
|
||||
case Messages.Type.ConnectionInformation:
|
||||
return this.uint(msg[1]) && this.string(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.SetPageVisibility:
|
||||
return this.boolean(msg[1])
|
||||
break
|
||||
|
||||
case Messages.Type.LongTask:
|
||||
return this.uint(msg[1]) && this.uint(msg[2]) && this.uint(msg[3]) && this.uint(msg[4]) && this.string(msg[5]) && this.string(msg[6]) && this.string(msg[7])
|
||||
break
|
||||
|
||||
case Messages.Type.SetNodeAttributeURLBased:
|
||||
return this.uint(msg[1]) && this.string(msg[2]) && this.string(msg[3]) && this.string(msg[4])
|
||||
break
|
||||
|
||||
case Messages.Type.SetCSSDataURLBased:
|
||||
return this.uint(msg[1]) && this.string(msg[2]) && this.string(msg[3])
|
||||
break
|
||||
|
||||
case Messages.Type.TechnicalInfo:
|
||||
return this.string(msg[1]) && this.string(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.CustomIssue:
|
||||
return this.string(msg[1]) && this.string(msg[2])
|
||||
break
|
||||
|
||||
case Messages.Type.CSSInsertRuleURLBased:
|
||||
return this.uint(msg[1]) && this.string(msg[2]) && this.uint(msg[3]) && this.string(msg[4])
|
||||
break
|
||||
|
||||
case Messages.Type.MouseClick:
|
||||
return this.uint(msg[1]) && this.uint(msg[2]) && this.string(msg[3]) && this.string(msg[4])
|
||||
break
|
||||
|
||||
case Messages.Type.CreateIFrameDocument:
|
||||
return this.uint(msg[1]) && this.uint(msg[2])
|
||||
break
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
declare const TextEncoder: any;
|
||||
declare const TextEncoder: any
|
||||
const textEncoder: { encode(str: string): Uint8Array } =
|
||||
typeof TextEncoder === 'function'
|
||||
? new TextEncoder()
|
||||
|
|
@ -6,102 +6,112 @@ const textEncoder: { encode(str: string): Uint8Array } =
|
|||
// Based on https://developer.mozilla.org/en-US/docs/Web/API/TextEncoder
|
||||
encode(str): Uint8Array {
|
||||
const Len = str.length,
|
||||
resArr = new Uint8Array(Len * 3);
|
||||
let resPos = -1;
|
||||
resArr = new Uint8Array(Len * 3)
|
||||
let resPos = -1
|
||||
for (let point = 0, nextcode = 0, i = 0; i !== Len; ) {
|
||||
(point = str.charCodeAt(i)), (i += 1);
|
||||
;(point = str.charCodeAt(i)), (i += 1)
|
||||
if (point >= 0xd800 && point <= 0xdbff) {
|
||||
if (i === Len) {
|
||||
resArr[(resPos += 1)] = 0xef; /*0b11101111*/
|
||||
resArr[(resPos += 1)] = 0xbf; /*0b10111111*/
|
||||
resArr[(resPos += 1)] = 0xbd; /*0b10111101*/
|
||||
break;
|
||||
resArr[(resPos += 1)] = 0xef /*0b11101111*/
|
||||
resArr[(resPos += 1)] = 0xbf /*0b10111111*/
|
||||
resArr[(resPos += 1)] = 0xbd /*0b10111101*/
|
||||
break
|
||||
}
|
||||
// https://mathiasbynens.be/notes/javascript-encoding#surrogate-formulae
|
||||
nextcode = str.charCodeAt(i);
|
||||
nextcode = str.charCodeAt(i)
|
||||
if (nextcode >= 0xdc00 && nextcode <= 0xdfff) {
|
||||
point = (point - 0xd800) * 0x400 + nextcode - 0xdc00 + 0x10000;
|
||||
i += 1;
|
||||
point = (point - 0xd800) * 0x400 + nextcode - 0xdc00 + 0x10000
|
||||
i += 1
|
||||
if (point > 0xffff) {
|
||||
resArr[(resPos += 1)] = (0x1e /*0b11110*/ << 3) | (point >>> 18);
|
||||
resArr[(resPos += 1)] = (0x1e /*0b11110*/ << 3) | (point >>> 18)
|
||||
resArr[(resPos += 1)] =
|
||||
(0x2 /*0b10*/ << 6) | ((point >>> 12) & 0x3f); /*0b00111111*/
|
||||
(0x2 /*0b10*/ << 6) | ((point >>> 12) & 0x3f) /*0b00111111*/
|
||||
resArr[(resPos += 1)] =
|
||||
(0x2 /*0b10*/ << 6) | ((point >>> 6) & 0x3f); /*0b00111111*/
|
||||
resArr[(resPos += 1)] = (0x2 /*0b10*/ << 6) | (point & 0x3f); /*0b00111111*/
|
||||
continue;
|
||||
(0x2 /*0b10*/ << 6) | ((point >>> 6) & 0x3f) /*0b00111111*/
|
||||
resArr[(resPos += 1)] = (0x2 /*0b10*/ << 6) | (point & 0x3f) /*0b00111111*/
|
||||
continue
|
||||
}
|
||||
} else {
|
||||
resArr[(resPos += 1)] = 0xef; /*0b11101111*/
|
||||
resArr[(resPos += 1)] = 0xbf; /*0b10111111*/
|
||||
resArr[(resPos += 1)] = 0xbd; /*0b10111101*/
|
||||
continue;
|
||||
resArr[(resPos += 1)] = 0xef /*0b11101111*/
|
||||
resArr[(resPos += 1)] = 0xbf /*0b10111111*/
|
||||
resArr[(resPos += 1)] = 0xbd /*0b10111101*/
|
||||
continue
|
||||
}
|
||||
}
|
||||
if (point <= 0x007f) {
|
||||
resArr[(resPos += 1)] = (0x0 /*0b0*/ << 7) | point;
|
||||
resArr[(resPos += 1)] = (0x0 /*0b0*/ << 7) | point
|
||||
} else if (point <= 0x07ff) {
|
||||
resArr[(resPos += 1)] = (0x6 /*0b110*/ << 5) | (point >>> 6);
|
||||
resArr[(resPos += 1)] = (0x2 /*0b10*/ << 6) | (point & 0x3f); /*0b00111111*/
|
||||
resArr[(resPos += 1)] = (0x6 /*0b110*/ << 5) | (point >>> 6)
|
||||
resArr[(resPos += 1)] = (0x2 /*0b10*/ << 6) | (point & 0x3f) /*0b00111111*/
|
||||
} else {
|
||||
resArr[(resPos += 1)] = (0xe /*0b1110*/ << 4) | (point >>> 12);
|
||||
resArr[(resPos += 1)] = (0x2 /*0b10*/ << 6) | ((point >>> 6) & 0x3f); /*0b00111111*/
|
||||
resArr[(resPos += 1)] = (0x2 /*0b10*/ << 6) | (point & 0x3f); /*0b00111111*/
|
||||
resArr[(resPos += 1)] = (0xe /*0b1110*/ << 4) | (point >>> 12)
|
||||
resArr[(resPos += 1)] = (0x2 /*0b10*/ << 6) | ((point >>> 6) & 0x3f) /*0b00111111*/
|
||||
resArr[(resPos += 1)] = (0x2 /*0b10*/ << 6) | (point & 0x3f) /*0b00111111*/
|
||||
}
|
||||
}
|
||||
return resArr.subarray(0, resPos + 1);
|
||||
return resArr.subarray(0, resPos + 1)
|
||||
},
|
||||
};
|
||||
}
|
||||
|
||||
export default class PrimitiveWriter {
|
||||
private offset = 0;
|
||||
private checkpointOffset = 0;
|
||||
private readonly data: Uint8Array;
|
||||
export default class PrimitiveEncoder {
|
||||
private offset = 0
|
||||
private checkpointOffset = 0
|
||||
private readonly data: Uint8Array
|
||||
constructor(private readonly size: number) {
|
||||
this.data = new Uint8Array(size);
|
||||
this.data = new Uint8Array(size)
|
||||
}
|
||||
getCurrentOffset(): number {
|
||||
return this.offset
|
||||
}
|
||||
checkpoint() {
|
||||
this.checkpointOffset = this.offset;
|
||||
this.checkpointOffset = this.offset
|
||||
}
|
||||
isEmpty(): boolean {
|
||||
return this.offset === 0;
|
||||
return this.offset === 0
|
||||
}
|
||||
skip(n: number): boolean {
|
||||
this.offset += n
|
||||
return this.offset <= this.size
|
||||
}
|
||||
set(bytes: Uint8Array, offset: number) {
|
||||
this.data.set(bytes, offset)
|
||||
}
|
||||
boolean(value: boolean): boolean {
|
||||
this.data[this.offset++] = +value;
|
||||
return this.offset <= this.size;
|
||||
this.data[this.offset++] = +value
|
||||
return this.offset <= this.size
|
||||
}
|
||||
uint(value: number): boolean {
|
||||
if (value < 0 || value > Number.MAX_SAFE_INTEGER) {
|
||||
value = 0;
|
||||
value = 0
|
||||
}
|
||||
while (value >= 0x80) {
|
||||
this.data[this.offset++] = value % 0x100 | 0x80;
|
||||
value = Math.floor(value / 128);
|
||||
this.data[this.offset++] = value % 0x100 | 0x80
|
||||
value = Math.floor(value / 128)
|
||||
}
|
||||
this.data[this.offset++] = value;
|
||||
return this.offset <= this.size;
|
||||
this.data[this.offset++] = value
|
||||
return this.offset <= this.size
|
||||
}
|
||||
int(value: number): boolean {
|
||||
value = Math.round(value);
|
||||
return this.uint(value >= 0 ? value * 2 : value * -2 - 1);
|
||||
value = Math.round(value)
|
||||
return this.uint(value >= 0 ? value * 2 : value * -2 - 1)
|
||||
}
|
||||
string(value: string): boolean {
|
||||
const encoded = textEncoder.encode(value);
|
||||
const length = encoded.byteLength;
|
||||
const encoded = textEncoder.encode(value)
|
||||
const length = encoded.byteLength
|
||||
if (!this.uint(length) || this.offset + length > this.size) {
|
||||
return false;
|
||||
return false
|
||||
}
|
||||
this.data.set(encoded, this.offset);
|
||||
this.offset += length;
|
||||
return true;
|
||||
this.data.set(encoded, this.offset)
|
||||
this.offset += length
|
||||
return true
|
||||
}
|
||||
reset(): void {
|
||||
this.offset = 0;
|
||||
this.checkpointOffset = 0;
|
||||
this.offset = 0
|
||||
this.checkpointOffset = 0
|
||||
}
|
||||
flush(): Uint8Array {
|
||||
const data = this.data.slice(0, this.checkpointOffset);
|
||||
this.reset();
|
||||
return data;
|
||||
const data = this.data.slice(0, this.checkpointOffset)
|
||||
this.reset()
|
||||
return data
|
||||
}
|
||||
}
|
||||
|
|
@ -1,6 +1,6 @@
|
|||
const INGEST_PATH = '/v1/web/i';
|
||||
const INGEST_PATH = '/v1/web/i'
|
||||
|
||||
const KEEPALIVE_SIZE_LIMIT = 64 << 10; // 64 kB
|
||||
const KEEPALIVE_SIZE_LIMIT = 64 << 10 // 64 kB
|
||||
|
||||
// function sendXHR(url: string, token: string, batch: Uint8Array): Promise<XMLHttpRequest> {
|
||||
// const req = new XMLHttpRequest()
|
||||
|
|
@ -21,11 +21,11 @@ const KEEPALIVE_SIZE_LIMIT = 64 << 10; // 64 kB
|
|||
// }
|
||||
|
||||
export default class QueueSender {
|
||||
private attemptsCount = 0;
|
||||
private busy = false;
|
||||
private readonly queue: Array<Uint8Array> = [];
|
||||
private readonly ingestURL;
|
||||
private token: string | null = null;
|
||||
private attemptsCount = 0
|
||||
private busy = false
|
||||
private readonly queue: Array<Uint8Array> = []
|
||||
private readonly ingestURL
|
||||
private token: string | null = null
|
||||
constructor(
|
||||
ingestBaseURL: string,
|
||||
private readonly onUnauthorised: () => any,
|
||||
|
|
@ -33,33 +33,33 @@ export default class QueueSender {
|
|||
private readonly MAX_ATTEMPTS_COUNT = 10,
|
||||
private readonly ATTEMPT_TIMEOUT = 1000,
|
||||
) {
|
||||
this.ingestURL = ingestBaseURL + INGEST_PATH;
|
||||
this.ingestURL = ingestBaseURL + INGEST_PATH
|
||||
}
|
||||
|
||||
authorise(token: string): void {
|
||||
this.token = token;
|
||||
this.token = token
|
||||
}
|
||||
|
||||
push(batch: Uint8Array): void {
|
||||
if (this.busy || !this.token) {
|
||||
this.queue.push(batch);
|
||||
this.queue.push(batch)
|
||||
} else {
|
||||
this.sendBatch(batch);
|
||||
this.sendBatch(batch)
|
||||
}
|
||||
}
|
||||
|
||||
private retry(batch: Uint8Array): void {
|
||||
if (this.attemptsCount >= this.MAX_ATTEMPTS_COUNT) {
|
||||
this.onFailure();
|
||||
return;
|
||||
this.onFailure()
|
||||
return
|
||||
}
|
||||
this.attemptsCount++;
|
||||
setTimeout(() => this.sendBatch(batch), this.ATTEMPT_TIMEOUT * this.attemptsCount);
|
||||
this.attemptsCount++
|
||||
setTimeout(() => this.sendBatch(batch), this.ATTEMPT_TIMEOUT * this.attemptsCount)
|
||||
}
|
||||
|
||||
// would be nice to use Beacon API, but it is not available in WebWorker
|
||||
private sendBatch(batch: Uint8Array): void {
|
||||
this.busy = true;
|
||||
this.busy = true
|
||||
|
||||
fetch(this.ingestURL, {
|
||||
body: batch,
|
||||
|
|
@ -73,30 +73,30 @@ export default class QueueSender {
|
|||
.then((r) => {
|
||||
if (r.status === 401) {
|
||||
// TODO: continuous session ?
|
||||
this.busy = false;
|
||||
this.onUnauthorised();
|
||||
return;
|
||||
this.busy = false
|
||||
this.onUnauthorised()
|
||||
return
|
||||
} else if (r.status >= 400) {
|
||||
this.retry(batch);
|
||||
return;
|
||||
this.retry(batch)
|
||||
return
|
||||
}
|
||||
|
||||
// Success
|
||||
this.attemptsCount = 0;
|
||||
const nextBatch = this.queue.shift();
|
||||
this.attemptsCount = 0
|
||||
const nextBatch = this.queue.shift()
|
||||
if (nextBatch) {
|
||||
this.sendBatch(nextBatch);
|
||||
this.sendBatch(nextBatch)
|
||||
} else {
|
||||
this.busy = false;
|
||||
this.busy = false
|
||||
}
|
||||
})
|
||||
.catch((e) => {
|
||||
console.warn('OpenReplay:', e);
|
||||
this.retry(batch);
|
||||
});
|
||||
console.warn('OpenReplay:', e)
|
||||
this.retry(batch)
|
||||
})
|
||||
}
|
||||
|
||||
clean() {
|
||||
this.queue.length = 0;
|
||||
this.queue.length = 0
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,9 +1,9 @@
|
|||
import type Message from '../common/messages.js';
|
||||
import { WorkerMessageData } from '../common/webworker.js';
|
||||
import type Message from '../common/messages.gen.js'
|
||||
import { Type as MType } from '../common/messages.gen.js'
|
||||
import { WorkerMessageData } from '../common/interaction.js'
|
||||
|
||||
import { classes, SetPageVisibility } from '../common/messages.js';
|
||||
import QueueSender from './QueueSender.js';
|
||||
import BatchWriter from './BatchWriter.js';
|
||||
import QueueSender from './QueueSender.js'
|
||||
import BatchWriter from './BatchWriter.js'
|
||||
|
||||
enum WorkerStatus {
|
||||
NotActive,
|
||||
|
|
@ -12,114 +12,112 @@ enum WorkerStatus {
|
|||
Active,
|
||||
}
|
||||
|
||||
const AUTO_SEND_INTERVAL = 10 * 1000;
|
||||
const AUTO_SEND_INTERVAL = 10 * 1000
|
||||
|
||||
let sender: QueueSender | null = null;
|
||||
let writer: BatchWriter | null = null;
|
||||
let workerStatus: WorkerStatus = WorkerStatus.NotActive;
|
||||
let sender: QueueSender | null = null
|
||||
let writer: BatchWriter | null = null
|
||||
let workerStatus: WorkerStatus = WorkerStatus.NotActive
|
||||
|
||||
function send(): void {
|
||||
if (!writer) {
|
||||
return;
|
||||
return
|
||||
}
|
||||
writer.finaliseBatch();
|
||||
writer.finaliseBatch()
|
||||
}
|
||||
|
||||
function reset(): void {
|
||||
workerStatus = WorkerStatus.Stopping;
|
||||
workerStatus = WorkerStatus.Stopping
|
||||
if (sendIntervalID !== null) {
|
||||
clearInterval(sendIntervalID);
|
||||
sendIntervalID = null;
|
||||
clearInterval(sendIntervalID)
|
||||
sendIntervalID = null
|
||||
}
|
||||
if (writer) {
|
||||
writer.clean();
|
||||
writer = null;
|
||||
writer.clean()
|
||||
writer = null
|
||||
}
|
||||
workerStatus = WorkerStatus.NotActive;
|
||||
workerStatus = WorkerStatus.NotActive
|
||||
}
|
||||
|
||||
function resetCleanQueue(): void {
|
||||
if (sender) {
|
||||
sender.clean();
|
||||
sender = null;
|
||||
sender.clean()
|
||||
sender = null
|
||||
}
|
||||
reset();
|
||||
reset()
|
||||
}
|
||||
|
||||
let sendIntervalID: ReturnType<typeof setInterval> | null = null;
|
||||
let restartTimeoutID: ReturnType<typeof setTimeout>;
|
||||
let sendIntervalID: ReturnType<typeof setInterval> | null = null
|
||||
let restartTimeoutID: ReturnType<typeof setTimeout>
|
||||
|
||||
self.onmessage = ({ data }: MessageEvent<WorkerMessageData>): any => {
|
||||
if (data == null) {
|
||||
send(); // TODO: sendAll?
|
||||
return;
|
||||
send() // TODO: sendAll?
|
||||
return
|
||||
}
|
||||
if (data === 'stop') {
|
||||
send();
|
||||
reset();
|
||||
return;
|
||||
send()
|
||||
reset()
|
||||
return
|
||||
}
|
||||
|
||||
if (Array.isArray(data)) {
|
||||
if (!writer) {
|
||||
throw new Error('WebWorker: writer not initialised. Service Should be Started.');
|
||||
}
|
||||
const w = writer;
|
||||
// Message[]
|
||||
data.forEach((data) => {
|
||||
// @ts-ignore
|
||||
const message: Message = new (classes.get(data._id))();
|
||||
Object.assign(message, data);
|
||||
if (message instanceof SetPageVisibility) {
|
||||
// @ts-ignore
|
||||
if ((<any>message).hidden) {
|
||||
restartTimeoutID = setTimeout(() => self.postMessage('restart'), 30 * 60 * 1000);
|
||||
if (!writer) {
|
||||
throw new Error('WebWorker: writer not initialised. Service Should be Started.')
|
||||
}
|
||||
const w = writer
|
||||
data.forEach((message) => {
|
||||
if (message[0] === MType.SetPageVisibility) {
|
||||
if (message[1]) {
|
||||
// .hidden
|
||||
restartTimeoutID = setTimeout(() => self.postMessage('restart'), 30 * 60 * 1000)
|
||||
} else {
|
||||
clearTimeout(restartTimeoutID);
|
||||
clearTimeout(restartTimeoutID)
|
||||
}
|
||||
}
|
||||
w.writeMessage(message);
|
||||
});
|
||||
return;
|
||||
w.writeMessage(message)
|
||||
})
|
||||
return
|
||||
}
|
||||
|
||||
if (data.type === 'start') {
|
||||
workerStatus = WorkerStatus.Starting;
|
||||
workerStatus = WorkerStatus.Starting
|
||||
sender = new QueueSender(
|
||||
data.ingestPoint,
|
||||
() => {
|
||||
// onUnauthorised
|
||||
self.postMessage('restart');
|
||||
self.postMessage('restart')
|
||||
},
|
||||
() => {
|
||||
// onFailure
|
||||
resetCleanQueue();
|
||||
self.postMessage('failed');
|
||||
resetCleanQueue()
|
||||
self.postMessage('failed')
|
||||
},
|
||||
data.connAttemptCount,
|
||||
data.connAttemptGap,
|
||||
);
|
||||
)
|
||||
writer = new BatchWriter(
|
||||
data.pageNo,
|
||||
data.timestamp,
|
||||
data.url,
|
||||
// onBatch
|
||||
(batch) => sender && sender.push(batch),
|
||||
);
|
||||
)
|
||||
if (sendIntervalID === null) {
|
||||
sendIntervalID = setInterval(send, AUTO_SEND_INTERVAL);
|
||||
sendIntervalID = setInterval(send, AUTO_SEND_INTERVAL)
|
||||
}
|
||||
return (workerStatus = WorkerStatus.Active);
|
||||
return (workerStatus = WorkerStatus.Active)
|
||||
}
|
||||
|
||||
if (data.type === 'auth') {
|
||||
if (!sender) {
|
||||
throw new Error('WebWorker: sender not initialised. Received auth.');
|
||||
throw new Error('WebWorker: sender not initialised. Received auth.')
|
||||
}
|
||||
if (!writer) {
|
||||
throw new Error('WebWorker: writer not initialised. Received auth.');
|
||||
throw new Error('WebWorker: writer not initialised. Received auth.')
|
||||
}
|
||||
sender.authorise(data.token);
|
||||
data.beaconSizeLimit && writer.setBeaconSizeLimit(data.beaconSizeLimit);
|
||||
return;
|
||||
sender.authorise(data.token)
|
||||
data.beaconSizeLimit && writer.setBeaconSizeLimit(data.beaconSizeLimit)
|
||||
return
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Reference in a new issue