You've already forked opentelemetry-go
mirror of
https://github.com/open-telemetry/opentelemetry-go.git
synced 2025-11-29 23:07:45 +02:00
Exporter: Initial Jaeger Trace Exporter commit. (#112)
* Exporter: Initial Jaeger Trace Exporter commit. * remove unused methods and add TODOs. * remove TODO. * fix build error by exporting event struct. * move Event from basetypes.go to export.go
This commit is contained in:
90
exporter/trace/jaeger/agent.go
Normal file
90
exporter/trace/jaeger/agent.go
Normal file
@@ -0,0 +1,90 @@
|
|||||||
|
// Copyright 2019, OpenTelemetry Authors
|
||||||
|
//
|
||||||
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
// you may not use this file except in compliance with the License.
|
||||||
|
// You may obtain a copy of the License at
|
||||||
|
//
|
||||||
|
// http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
//
|
||||||
|
// Unless required by applicable law or agreed to in writing, software
|
||||||
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
package jaeger
|
||||||
|
|
||||||
|
import (
|
||||||
|
"fmt"
|
||||||
|
"io"
|
||||||
|
"net"
|
||||||
|
|
||||||
|
"github.com/apache/thrift/lib/go/thrift"
|
||||||
|
|
||||||
|
gen "go.opentelemetry.io/exporter/trace/jaeger/internal/gen-go/jaeger"
|
||||||
|
)
|
||||||
|
|
||||||
|
// udpPacketMaxLength is the max size of UDP packet we want to send, synced with jaeger-agent
|
||||||
|
const udpPacketMaxLength = 65000
|
||||||
|
|
||||||
|
// agentClientUDP is a UDP client to Jaeger agent that implements gen.Agent interface.
|
||||||
|
type agentClientUDP struct {
|
||||||
|
gen.Agent
|
||||||
|
io.Closer
|
||||||
|
|
||||||
|
connUDP *net.UDPConn
|
||||||
|
client *gen.AgentClient
|
||||||
|
maxPacketSize int // max size of datagram in bytes
|
||||||
|
thriftBuffer *thrift.TMemoryBuffer // buffer used to calculate byte size of a span
|
||||||
|
}
|
||||||
|
|
||||||
|
// newAgentClientUDP creates a client that sends spans to Jaeger Agent over UDP.
|
||||||
|
func newAgentClientUDP(hostPort string, maxPacketSize int) (*agentClientUDP, error) {
|
||||||
|
if maxPacketSize == 0 {
|
||||||
|
maxPacketSize = udpPacketMaxLength
|
||||||
|
}
|
||||||
|
|
||||||
|
thriftBuffer := thrift.NewTMemoryBufferLen(maxPacketSize)
|
||||||
|
protocolFactory := thrift.NewTCompactProtocolFactory()
|
||||||
|
client := gen.NewAgentClientFactory(thriftBuffer, protocolFactory)
|
||||||
|
|
||||||
|
destAddr, err := net.ResolveUDPAddr("udp", hostPort)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
connUDP, err := net.DialUDP(destAddr.Network(), nil, destAddr)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
if err := connUDP.SetWriteBuffer(maxPacketSize); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
clientUDP := &agentClientUDP{
|
||||||
|
connUDP: connUDP,
|
||||||
|
client: client,
|
||||||
|
maxPacketSize: maxPacketSize,
|
||||||
|
thriftBuffer: thriftBuffer}
|
||||||
|
return clientUDP, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// EmitBatch implements EmitBatch() of Agent interface
|
||||||
|
func (a *agentClientUDP) EmitBatch(batch *gen.Batch) error {
|
||||||
|
a.thriftBuffer.Reset()
|
||||||
|
a.client.SeqId = 0 // we have no need for distinct SeqIds for our one-way UDP messages
|
||||||
|
if err := a.client.EmitBatch(batch); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
if a.thriftBuffer.Len() > a.maxPacketSize {
|
||||||
|
return fmt.Errorf("Data does not fit within one UDP packet; size %d, max %d, spans %d",
|
||||||
|
a.thriftBuffer.Len(), a.maxPacketSize, len(batch.Spans))
|
||||||
|
}
|
||||||
|
_, err := a.connUDP.Write(a.thriftBuffer.Bytes())
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
// Close implements Close() of io.Closer and closes the underlying UDP connection.
|
||||||
|
func (a *agentClientUDP) Close() error {
|
||||||
|
return a.connUDP.Close()
|
||||||
|
}
|
||||||
@@ -12,18 +12,5 @@
|
|||||||
// See the License for the specific language governing permissions and
|
// See the License for the specific language governing permissions and
|
||||||
// limitations under the License.
|
// limitations under the License.
|
||||||
|
|
||||||
package trace
|
// Package jaeger contains an OpenTelemetry tracing exporter for Jaeger.
|
||||||
|
package jaeger // import "go.opentelemetry.io/exporter/trace/jaeger"
|
||||||
import (
|
|
||||||
"time"
|
|
||||||
|
|
||||||
"go.opentelemetry.io/api/core"
|
|
||||||
)
|
|
||||||
|
|
||||||
// event is used to describe an event with a message string and set of
|
|
||||||
// attributes.
|
|
||||||
type event struct {
|
|
||||||
msg string
|
|
||||||
attributes []core.KeyValue
|
|
||||||
time time.Time
|
|
||||||
}
|
|
||||||
62
exporter/trace/jaeger/example/main.go
Normal file
62
exporter/trace/jaeger/example/main.go
Normal file
@@ -0,0 +1,62 @@
|
|||||||
|
// Copyright 2019, OpenTelemetry Authors
|
||||||
|
//
|
||||||
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
// you may not use this file except in compliance with the License.
|
||||||
|
// You may obtain a copy of the License at
|
||||||
|
//
|
||||||
|
// http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
//
|
||||||
|
// Unless required by applicable law or agreed to in writing, software
|
||||||
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
// Command jaeger is an example program that creates spans
|
||||||
|
// and uploads to Jaeger.
|
||||||
|
package main
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"log"
|
||||||
|
|
||||||
|
apitrace "go.opentelemetry.io/api/trace"
|
||||||
|
"go.opentelemetry.io/exporter/trace/jaeger"
|
||||||
|
"go.opentelemetry.io/sdk/trace"
|
||||||
|
)
|
||||||
|
|
||||||
|
func main() {
|
||||||
|
trace.Register()
|
||||||
|
ctx := context.Background()
|
||||||
|
|
||||||
|
// Register the Jaeger exporter to be able to retrieve
|
||||||
|
// the collected spans.
|
||||||
|
exporter, err := jaeger.NewExporter(jaeger.Options{
|
||||||
|
CollectorEndpoint: "http://localhost:14268/api/traces",
|
||||||
|
Process: jaeger.Process{
|
||||||
|
ServiceName: "trace-demo",
|
||||||
|
},
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
log.Fatal(err)
|
||||||
|
}
|
||||||
|
trace.RegisterExporter(exporter)
|
||||||
|
|
||||||
|
// For demoing purposes, always sample. In a production application, you should
|
||||||
|
// configure this to a trace.ProbabilitySampler set at the desired
|
||||||
|
// probability.
|
||||||
|
trace.ApplyConfig(trace.Config{DefaultSampler: trace.AlwaysSample()})
|
||||||
|
|
||||||
|
ctx, span := apitrace.GlobalTracer().Start(ctx, "/foo")
|
||||||
|
bar(ctx)
|
||||||
|
span.Finish()
|
||||||
|
|
||||||
|
exporter.Flush()
|
||||||
|
}
|
||||||
|
|
||||||
|
func bar(ctx context.Context) {
|
||||||
|
_, span := apitrace.GlobalTracer().Start(ctx, "/bar")
|
||||||
|
defer span.Finish()
|
||||||
|
|
||||||
|
// Do bar...
|
||||||
|
}
|
||||||
@@ -0,0 +1,6 @@
|
|||||||
|
// Autogenerated by Thrift Compiler (0.11.0)
|
||||||
|
// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
|
||||||
|
|
||||||
|
package jaeger
|
||||||
|
|
||||||
|
var GoUnusedProtection__ int
|
||||||
244
exporter/trace/jaeger/internal/gen-go/jaeger/agent.go
Normal file
244
exporter/trace/jaeger/internal/gen-go/jaeger/agent.go
Normal file
@@ -0,0 +1,244 @@
|
|||||||
|
// Autogenerated by Thrift Compiler (0.9.3)
|
||||||
|
// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
|
||||||
|
|
||||||
|
package jaeger
|
||||||
|
|
||||||
|
import (
|
||||||
|
"bytes"
|
||||||
|
"context"
|
||||||
|
"fmt"
|
||||||
|
|
||||||
|
"github.com/apache/thrift/lib/go/thrift"
|
||||||
|
)
|
||||||
|
|
||||||
|
// (needed to ensure safety because of naive import list construction.)
|
||||||
|
var _ = thrift.ZERO
|
||||||
|
var _ = fmt.Printf
|
||||||
|
var _ = bytes.Equal
|
||||||
|
|
||||||
|
type Agent interface {
|
||||||
|
// Parameters:
|
||||||
|
// - Batch
|
||||||
|
EmitBatch(batch *Batch) (err error)
|
||||||
|
}
|
||||||
|
|
||||||
|
type AgentClient struct {
|
||||||
|
Transport thrift.TTransport
|
||||||
|
ProtocolFactory thrift.TProtocolFactory
|
||||||
|
InputProtocol thrift.TProtocol
|
||||||
|
OutputProtocol thrift.TProtocol
|
||||||
|
SeqId int32
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewAgentClientFactory(t thrift.TTransport, f thrift.TProtocolFactory) *AgentClient {
|
||||||
|
return &AgentClient{Transport: t,
|
||||||
|
ProtocolFactory: f,
|
||||||
|
InputProtocol: f.GetProtocol(t),
|
||||||
|
OutputProtocol: f.GetProtocol(t),
|
||||||
|
SeqId: 0,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewAgentClientProtocol(t thrift.TTransport, iprot thrift.TProtocol, oprot thrift.TProtocol) *AgentClient {
|
||||||
|
return &AgentClient{Transport: t,
|
||||||
|
ProtocolFactory: nil,
|
||||||
|
InputProtocol: iprot,
|
||||||
|
OutputProtocol: oprot,
|
||||||
|
SeqId: 0,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Parameters:
|
||||||
|
// - Batch
|
||||||
|
func (p *AgentClient) EmitBatch(batch *Batch) (err error) {
|
||||||
|
if err = p.sendEmitBatch(batch); err != nil {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *AgentClient) sendEmitBatch(batch *Batch) (err error) {
|
||||||
|
oprot := p.OutputProtocol
|
||||||
|
if oprot == nil {
|
||||||
|
oprot = p.ProtocolFactory.GetProtocol(p.Transport)
|
||||||
|
p.OutputProtocol = oprot
|
||||||
|
}
|
||||||
|
p.SeqId++
|
||||||
|
if err = oprot.WriteMessageBegin("emitBatch", thrift.ONEWAY, p.SeqId); err != nil {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
args := AgentEmitBatchArgs{
|
||||||
|
Batch: batch,
|
||||||
|
}
|
||||||
|
if err = args.Write(oprot); err != nil {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
if err = oprot.WriteMessageEnd(); err != nil {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
return oprot.Flush(context.Background())
|
||||||
|
}
|
||||||
|
|
||||||
|
type AgentProcessor struct {
|
||||||
|
processorMap map[string]thrift.TProcessorFunction
|
||||||
|
handler Agent
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *AgentProcessor) AddToProcessorMap(key string, processor thrift.TProcessorFunction) {
|
||||||
|
p.processorMap[key] = processor
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *AgentProcessor) GetProcessorFunction(key string) (processor thrift.TProcessorFunction, ok bool) {
|
||||||
|
processor, ok = p.processorMap[key]
|
||||||
|
return processor, ok
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *AgentProcessor) ProcessorMap() map[string]thrift.TProcessorFunction {
|
||||||
|
return p.processorMap
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewAgentProcessor(handler Agent) *AgentProcessor {
|
||||||
|
|
||||||
|
self0 := &AgentProcessor{handler: handler, processorMap: make(map[string]thrift.TProcessorFunction)}
|
||||||
|
self0.processorMap["emitBatch"] = &agentProcessorEmitBatch{handler: handler}
|
||||||
|
return self0
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *AgentProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
|
||||||
|
ctx := context.Background()
|
||||||
|
name, _, seqId, err := iprot.ReadMessageBegin()
|
||||||
|
if err != nil {
|
||||||
|
return false, err
|
||||||
|
}
|
||||||
|
if processor, ok := p.GetProcessorFunction(name); ok {
|
||||||
|
return processor.Process(ctx, seqId, iprot, oprot)
|
||||||
|
}
|
||||||
|
iprot.Skip(thrift.STRUCT)
|
||||||
|
iprot.ReadMessageEnd()
|
||||||
|
x1 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name)
|
||||||
|
oprot.WriteMessageBegin(name, thrift.EXCEPTION, seqId)
|
||||||
|
x1.Write(oprot)
|
||||||
|
oprot.WriteMessageEnd()
|
||||||
|
oprot.Flush(ctx)
|
||||||
|
return false, x1
|
||||||
|
}
|
||||||
|
|
||||||
|
type agentProcessorEmitBatch struct {
|
||||||
|
handler Agent
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *agentProcessorEmitBatch) Process(ctx context.Context, seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
|
||||||
|
args := AgentEmitBatchArgs{}
|
||||||
|
if err = args.Read(iprot); err != nil {
|
||||||
|
iprot.ReadMessageEnd()
|
||||||
|
return false, err
|
||||||
|
}
|
||||||
|
|
||||||
|
iprot.ReadMessageEnd()
|
||||||
|
var err2 error
|
||||||
|
if err2 = p.handler.EmitBatch(args.Batch); err2 != nil {
|
||||||
|
return true, err2
|
||||||
|
}
|
||||||
|
return true, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// HELPER FUNCTIONS AND STRUCTURES
|
||||||
|
|
||||||
|
// Attributes:
|
||||||
|
// - Batch
|
||||||
|
type AgentEmitBatchArgs struct {
|
||||||
|
Batch *Batch `thrift:"batch,1" json:"batch"`
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewAgentEmitBatchArgs() *AgentEmitBatchArgs {
|
||||||
|
return &AgentEmitBatchArgs{}
|
||||||
|
}
|
||||||
|
|
||||||
|
var AgentEmitBatchArgs_Batch_DEFAULT *Batch
|
||||||
|
|
||||||
|
func (p *AgentEmitBatchArgs) GetBatch() *Batch {
|
||||||
|
if !p.IsSetBatch() {
|
||||||
|
return AgentEmitBatchArgs_Batch_DEFAULT
|
||||||
|
}
|
||||||
|
return p.Batch
|
||||||
|
}
|
||||||
|
func (p *AgentEmitBatchArgs) IsSetBatch() bool {
|
||||||
|
return p.Batch != nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *AgentEmitBatchArgs) Read(iprot thrift.TProtocol) error {
|
||||||
|
if _, err := iprot.ReadStructBegin(); err != nil {
|
||||||
|
return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
|
||||||
|
}
|
||||||
|
|
||||||
|
for {
|
||||||
|
_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
|
||||||
|
if err != nil {
|
||||||
|
return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
|
||||||
|
}
|
||||||
|
if fieldTypeId == thrift.STOP {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
switch fieldId {
|
||||||
|
case 1:
|
||||||
|
if err := p.readField1(iprot); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
default:
|
||||||
|
if err := iprot.Skip(fieldTypeId); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if err := iprot.ReadFieldEnd(); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if err := iprot.ReadStructEnd(); err != nil {
|
||||||
|
return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *AgentEmitBatchArgs) readField1(iprot thrift.TProtocol) error {
|
||||||
|
p.Batch = &Batch{}
|
||||||
|
if err := p.Batch.Read(iprot); err != nil {
|
||||||
|
return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Batch), err)
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *AgentEmitBatchArgs) Write(oprot thrift.TProtocol) error {
|
||||||
|
if err := oprot.WriteStructBegin("emitBatch_args"); err != nil {
|
||||||
|
return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
|
||||||
|
}
|
||||||
|
if err := p.writeField1(oprot); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
if err := oprot.WriteFieldStop(); err != nil {
|
||||||
|
return thrift.PrependError("write field stop error: ", err)
|
||||||
|
}
|
||||||
|
if err := oprot.WriteStructEnd(); err != nil {
|
||||||
|
return thrift.PrependError("write struct stop error: ", err)
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *AgentEmitBatchArgs) writeField1(oprot thrift.TProtocol) (err error) {
|
||||||
|
if err := oprot.WriteFieldBegin("batch", thrift.STRUCT, 1); err != nil {
|
||||||
|
return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:batch: ", p), err)
|
||||||
|
}
|
||||||
|
if err := p.Batch.Write(oprot); err != nil {
|
||||||
|
return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Batch), err)
|
||||||
|
}
|
||||||
|
if err := oprot.WriteFieldEnd(); err != nil {
|
||||||
|
return thrift.PrependError(fmt.Sprintf("%T write field end error 1:batch: ", p), err)
|
||||||
|
}
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *AgentEmitBatchArgs) String() string {
|
||||||
|
if p == nil {
|
||||||
|
return "<nil>"
|
||||||
|
}
|
||||||
|
return fmt.Sprintf("AgentEmitBatchArgs(%+v)", *p)
|
||||||
|
}
|
||||||
@@ -0,0 +1,155 @@
|
|||||||
|
// Autogenerated by Thrift Compiler (0.11.0)
|
||||||
|
// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
|
||||||
|
|
||||||
|
package main
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"flag"
|
||||||
|
"fmt"
|
||||||
|
"math"
|
||||||
|
"net"
|
||||||
|
"net/url"
|
||||||
|
"os"
|
||||||
|
"strconv"
|
||||||
|
"strings"
|
||||||
|
|
||||||
|
"go.opentelemetry.io/exporter/trace/jaeger/internal/gen-go/jaeger"
|
||||||
|
"github.com/apache/thrift/lib/go/thrift"
|
||||||
|
)
|
||||||
|
|
||||||
|
func Usage() {
|
||||||
|
fmt.Fprintln(os.Stderr, "Usage of ", os.Args[0], " [-h host:port] [-u url] [-f[ramed]] function [arg1 [arg2...]]:")
|
||||||
|
flag.PrintDefaults()
|
||||||
|
fmt.Fprintln(os.Stderr, "\nFunctions:")
|
||||||
|
fmt.Fprintln(os.Stderr, " submitBatches( batches)")
|
||||||
|
fmt.Fprintln(os.Stderr)
|
||||||
|
os.Exit(0)
|
||||||
|
}
|
||||||
|
|
||||||
|
func main() {
|
||||||
|
flag.Usage = Usage
|
||||||
|
var host string
|
||||||
|
var port int
|
||||||
|
var protocol string
|
||||||
|
var urlString string
|
||||||
|
var framed bool
|
||||||
|
var useHttp bool
|
||||||
|
var parsedUrl *url.URL
|
||||||
|
var trans thrift.TTransport
|
||||||
|
_ = strconv.Atoi
|
||||||
|
_ = math.Abs
|
||||||
|
flag.Usage = Usage
|
||||||
|
flag.StringVar(&host, "h", "localhost", "Specify host and port")
|
||||||
|
flag.IntVar(&port, "p", 9090, "Specify port")
|
||||||
|
flag.StringVar(&protocol, "P", "binary", "Specify the protocol (binary, compact, simplejson, json)")
|
||||||
|
flag.StringVar(&urlString, "u", "", "Specify the url")
|
||||||
|
flag.BoolVar(&framed, "framed", false, "Use framed transport")
|
||||||
|
flag.BoolVar(&useHttp, "http", false, "Use http")
|
||||||
|
flag.Parse()
|
||||||
|
|
||||||
|
if len(urlString) > 0 {
|
||||||
|
var err error
|
||||||
|
parsedUrl, err = url.Parse(urlString)
|
||||||
|
if err != nil {
|
||||||
|
fmt.Fprintln(os.Stderr, "Error parsing URL: ", err)
|
||||||
|
flag.Usage()
|
||||||
|
}
|
||||||
|
host = parsedUrl.Host
|
||||||
|
useHttp = len(parsedUrl.Scheme) <= 0 || parsedUrl.Scheme == "http"
|
||||||
|
} else if useHttp {
|
||||||
|
_, err := url.Parse(fmt.Sprint("http://", host, ":", port))
|
||||||
|
if err != nil {
|
||||||
|
fmt.Fprintln(os.Stderr, "Error parsing URL: ", err)
|
||||||
|
flag.Usage()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
cmd := flag.Arg(0)
|
||||||
|
var err error
|
||||||
|
if useHttp {
|
||||||
|
trans, err = thrift.NewTHttpClient(parsedUrl.String())
|
||||||
|
} else {
|
||||||
|
portStr := fmt.Sprint(port)
|
||||||
|
if strings.Contains(host, ":") {
|
||||||
|
host, portStr, err = net.SplitHostPort(host)
|
||||||
|
if err != nil {
|
||||||
|
fmt.Fprintln(os.Stderr, "error with host:", err)
|
||||||
|
os.Exit(1)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
trans, err = thrift.NewTSocket(net.JoinHostPort(host, portStr))
|
||||||
|
if err != nil {
|
||||||
|
fmt.Fprintln(os.Stderr, "error resolving address:", err)
|
||||||
|
os.Exit(1)
|
||||||
|
}
|
||||||
|
if framed {
|
||||||
|
trans = thrift.NewTFramedTransport(trans)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if err != nil {
|
||||||
|
fmt.Fprintln(os.Stderr, "Error creating transport", err)
|
||||||
|
os.Exit(1)
|
||||||
|
}
|
||||||
|
defer trans.Close()
|
||||||
|
var protocolFactory thrift.TProtocolFactory
|
||||||
|
switch protocol {
|
||||||
|
case "compact":
|
||||||
|
protocolFactory = thrift.NewTCompactProtocolFactory()
|
||||||
|
break
|
||||||
|
case "simplejson":
|
||||||
|
protocolFactory = thrift.NewTSimpleJSONProtocolFactory()
|
||||||
|
break
|
||||||
|
case "json":
|
||||||
|
protocolFactory = thrift.NewTJSONProtocolFactory()
|
||||||
|
break
|
||||||
|
case "binary", "":
|
||||||
|
protocolFactory = thrift.NewTBinaryProtocolFactoryDefault()
|
||||||
|
break
|
||||||
|
default:
|
||||||
|
fmt.Fprintln(os.Stderr, "Invalid protocol specified: ", protocol)
|
||||||
|
Usage()
|
||||||
|
os.Exit(1)
|
||||||
|
}
|
||||||
|
iprot := protocolFactory.GetProtocol(trans)
|
||||||
|
oprot := protocolFactory.GetProtocol(trans)
|
||||||
|
client := jaeger.NewCollectorClient(thrift.NewTStandardClient(iprot, oprot))
|
||||||
|
if err := trans.Open(); err != nil {
|
||||||
|
fmt.Fprintln(os.Stderr, "Error opening socket to ", host, ":", port, " ", err)
|
||||||
|
os.Exit(1)
|
||||||
|
}
|
||||||
|
|
||||||
|
switch cmd {
|
||||||
|
case "submitBatches":
|
||||||
|
if flag.NArg()-1 != 1 {
|
||||||
|
fmt.Fprintln(os.Stderr, "SubmitBatches requires 1 args")
|
||||||
|
flag.Usage()
|
||||||
|
}
|
||||||
|
arg12 := flag.Arg(1)
|
||||||
|
mbTrans13 := thrift.NewTMemoryBufferLen(len(arg12))
|
||||||
|
defer mbTrans13.Close()
|
||||||
|
_, err14 := mbTrans13.WriteString(arg12)
|
||||||
|
if err14 != nil {
|
||||||
|
Usage()
|
||||||
|
return
|
||||||
|
}
|
||||||
|
factory15 := thrift.NewTSimpleJSONProtocolFactory()
|
||||||
|
jsProt16 := factory15.GetProtocol(mbTrans13)
|
||||||
|
containerStruct0 := jaeger.NewCollectorSubmitBatchesArgs()
|
||||||
|
err17 := containerStruct0.ReadField1(jsProt16)
|
||||||
|
if err17 != nil {
|
||||||
|
Usage()
|
||||||
|
return
|
||||||
|
}
|
||||||
|
argvalue0 := containerStruct0.Batches
|
||||||
|
value0 := argvalue0
|
||||||
|
fmt.Print(client.SubmitBatches(context.Background(), value0))
|
||||||
|
fmt.Print("\n")
|
||||||
|
break
|
||||||
|
case "":
|
||||||
|
Usage()
|
||||||
|
break
|
||||||
|
default:
|
||||||
|
fmt.Fprintln(os.Stderr, "Invalid function ", cmd)
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,23 @@
|
|||||||
|
// Autogenerated by Thrift Compiler (0.11.0)
|
||||||
|
// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
|
||||||
|
|
||||||
|
package jaeger
|
||||||
|
|
||||||
|
import (
|
||||||
|
"bytes"
|
||||||
|
"context"
|
||||||
|
"fmt"
|
||||||
|
"reflect"
|
||||||
|
|
||||||
|
"github.com/apache/thrift/lib/go/thrift"
|
||||||
|
)
|
||||||
|
|
||||||
|
// (needed to ensure safety because of naive import list construction.)
|
||||||
|
var _ = thrift.ZERO
|
||||||
|
var _ = fmt.Printf
|
||||||
|
var _ = context.Background
|
||||||
|
var _ = reflect.DeepEqual
|
||||||
|
var _ = bytes.Equal
|
||||||
|
|
||||||
|
func init() {
|
||||||
|
}
|
||||||
2443
exporter/trace/jaeger/internal/gen-go/jaeger/jaeger.go
Normal file
2443
exporter/trace/jaeger/internal/gen-go/jaeger/jaeger.go
Normal file
File diff suppressed because it is too large
Load Diff
326
exporter/trace/jaeger/jaeger.go
Normal file
326
exporter/trace/jaeger/jaeger.go
Normal file
@@ -0,0 +1,326 @@
|
|||||||
|
// Copyright 2019, OpenTelemetry Authors
|
||||||
|
//
|
||||||
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
// you may not use this file except in compliance with the License.
|
||||||
|
// You may obtain a copy of the License at
|
||||||
|
//
|
||||||
|
// http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
//
|
||||||
|
// Unless required by applicable law or agreed to in writing, software
|
||||||
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
package jaeger
|
||||||
|
|
||||||
|
import (
|
||||||
|
"bytes"
|
||||||
|
"errors"
|
||||||
|
"fmt"
|
||||||
|
"io"
|
||||||
|
"io/ioutil"
|
||||||
|
"log"
|
||||||
|
"net/http"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"google.golang.org/grpc/codes"
|
||||||
|
|
||||||
|
"github.com/apache/thrift/lib/go/thrift"
|
||||||
|
"google.golang.org/api/support/bundler"
|
||||||
|
|
||||||
|
gen "go.opentelemetry.io/exporter/trace/jaeger/internal/gen-go/jaeger"
|
||||||
|
"go.opentelemetry.io/sdk/trace"
|
||||||
|
)
|
||||||
|
|
||||||
|
const defaultServiceName = "OpenTelemetry"
|
||||||
|
|
||||||
|
// Options are the options to be used when initializing a Jaeger exporter.
|
||||||
|
type Options struct {
|
||||||
|
// CollectorEndpoint is the full url to the Jaeger HTTP Thrift collector.
|
||||||
|
// For example, http://localhost:14268/api/traces
|
||||||
|
CollectorEndpoint string
|
||||||
|
|
||||||
|
// AgentEndpoint instructs exporter to send spans to jaeger-agent at this address.
|
||||||
|
// For example, localhost:6831.
|
||||||
|
AgentEndpoint string
|
||||||
|
|
||||||
|
// OnError is the hook to be called when there is
|
||||||
|
// an error occurred when uploading the stats data.
|
||||||
|
// If no custom hook is set, errors are logged.
|
||||||
|
// Optional.
|
||||||
|
OnError func(err error)
|
||||||
|
|
||||||
|
// Username to be used if basic auth is required.
|
||||||
|
// Optional.
|
||||||
|
Username string
|
||||||
|
|
||||||
|
// Password to be used if basic auth is required.
|
||||||
|
// Optional.
|
||||||
|
Password string
|
||||||
|
|
||||||
|
// Process contains the information about the exporting process.
|
||||||
|
Process Process
|
||||||
|
|
||||||
|
//BufferMaxCount defines the total number of traces that can be buffered in memory
|
||||||
|
BufferMaxCount int
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewExporter returns a trace.Exporter implementation that exports
|
||||||
|
// the collected spans to Jaeger.
|
||||||
|
func NewExporter(o Options) (*Exporter, error) {
|
||||||
|
if o.CollectorEndpoint == "" && o.AgentEndpoint == "" {
|
||||||
|
return nil, errors.New("missing endpoint for Jaeger exporter")
|
||||||
|
}
|
||||||
|
|
||||||
|
var endpoint string
|
||||||
|
var client *agentClientUDP
|
||||||
|
var err error
|
||||||
|
if o.CollectorEndpoint != "" {
|
||||||
|
endpoint = o.CollectorEndpoint
|
||||||
|
} else {
|
||||||
|
client, err = newAgentClientUDP(o.AgentEndpoint, udpPacketMaxLength)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
onError := func(err error) {
|
||||||
|
if o.OnError != nil {
|
||||||
|
o.OnError(err)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
log.Printf("Error when uploading spans to Jaeger: %v", err)
|
||||||
|
}
|
||||||
|
service := o.Process.ServiceName
|
||||||
|
if service == "" {
|
||||||
|
service = defaultServiceName
|
||||||
|
}
|
||||||
|
tags := make([]*gen.Tag, len(o.Process.Tags))
|
||||||
|
for i, tag := range o.Process.Tags {
|
||||||
|
tags[i] = attributeToTag(tag.key, tag.value)
|
||||||
|
}
|
||||||
|
e := &Exporter{
|
||||||
|
endpoint: endpoint,
|
||||||
|
agentEndpoint: o.AgentEndpoint,
|
||||||
|
client: client,
|
||||||
|
username: o.Username,
|
||||||
|
password: o.Password,
|
||||||
|
process: &gen.Process{
|
||||||
|
ServiceName: service,
|
||||||
|
Tags: tags,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
bundler := bundler.NewBundler((*gen.Span)(nil), func(bundle interface{}) {
|
||||||
|
if err := e.upload(bundle.([]*gen.Span)); err != nil {
|
||||||
|
onError(err)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
|
||||||
|
// Set BufferedByteLimit with the total number of spans that are permissible to be held in memory.
|
||||||
|
// This needs to be done since the size of messages is always set to 1. Failing to set this would allow
|
||||||
|
// 1G messages to be held in memory since that is the default value of BufferedByteLimit.
|
||||||
|
if o.BufferMaxCount != 0 {
|
||||||
|
bundler.BufferedByteLimit = o.BufferMaxCount
|
||||||
|
}
|
||||||
|
|
||||||
|
e.bundler = bundler
|
||||||
|
return e, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// Process contains the information exported to jaeger about the source
|
||||||
|
// of the trace data.
|
||||||
|
type Process struct {
|
||||||
|
// ServiceName is the Jaeger service name.
|
||||||
|
ServiceName string
|
||||||
|
|
||||||
|
// Tags are added to Jaeger Process exports
|
||||||
|
Tags []Tag
|
||||||
|
}
|
||||||
|
|
||||||
|
// Tag defines a key-value pair
|
||||||
|
// It is limited to the possible conversions to *jaeger.Tag by attributeToTag
|
||||||
|
type Tag struct {
|
||||||
|
key string
|
||||||
|
value interface{}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Exporter is an implementation of trace.Exporter that uploads spans to Jaeger.
|
||||||
|
type Exporter struct {
|
||||||
|
endpoint string
|
||||||
|
agentEndpoint string
|
||||||
|
process *gen.Process
|
||||||
|
bundler *bundler.Bundler
|
||||||
|
client *agentClientUDP
|
||||||
|
|
||||||
|
username, password string
|
||||||
|
}
|
||||||
|
|
||||||
|
var _ trace.Exporter = (*Exporter)(nil)
|
||||||
|
|
||||||
|
// ExportSpan exports a SpanData to Jaeger.
|
||||||
|
func (e *Exporter) ExportSpan(data *trace.SpanData) {
|
||||||
|
_ = e.bundler.Add(spanDataToThrift(data), 1)
|
||||||
|
// TODO(jbd): Handle oversized bundlers.
|
||||||
|
}
|
||||||
|
|
||||||
|
func spanDataToThrift(data *trace.SpanData) *gen.Span {
|
||||||
|
tags := make([]*gen.Tag, 0, len(data.Attributes))
|
||||||
|
for k, v := range data.Attributes {
|
||||||
|
tag := attributeToTag(k, v)
|
||||||
|
if tag != nil {
|
||||||
|
tags = append(tags, tag)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
tags = append(tags,
|
||||||
|
attributeToTag("status.code", int32(data.Status)),
|
||||||
|
attributeToTag("status.message", data.Status.String()),
|
||||||
|
)
|
||||||
|
|
||||||
|
// Ensure that if Status.Code is not OK, that we set the "error" tag on the Jaeger span.
|
||||||
|
// See Issue https://github.com/census-instrumentation/opencensus-go/issues/1041
|
||||||
|
if data.Status != codes.OK {
|
||||||
|
tags = append(tags, attributeToTag("error", true))
|
||||||
|
}
|
||||||
|
|
||||||
|
var logs []*gen.Log
|
||||||
|
for _, a := range data.MessageEvents {
|
||||||
|
fields := make([]*gen.Tag, 0, len(a.Attributes))
|
||||||
|
for _, kv := range a.Attributes {
|
||||||
|
tag := attributeToTag(kv.Key.Variable.Name, kv.Value.Emit())
|
||||||
|
if tag != nil {
|
||||||
|
fields = append(fields, tag)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
fields = append(fields, attributeToTag("message", a.Message))
|
||||||
|
logs = append(logs, &gen.Log{
|
||||||
|
//Timestamp: a.Time.UnixNano() / 1000,
|
||||||
|
//TODO: [rghetia] update when time is supported in the event.
|
||||||
|
Timestamp: time.Now().UnixNano() / 1000,
|
||||||
|
Fields: fields,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
//TODO: [rghetia] add links.
|
||||||
|
//
|
||||||
|
//var refs []*gen.SpanRef
|
||||||
|
//for _, link := range data.Links {
|
||||||
|
// refs = append(refs, &gen.SpanRef{
|
||||||
|
// TraceIdHigh: bytesToInt64(link.TraceID[0:8]),
|
||||||
|
// TraceIdLow: bytesToInt64(link.TraceID[8:16]),
|
||||||
|
// SpanId: bytesToInt64(link.SpanID[:]),
|
||||||
|
// })
|
||||||
|
//}
|
||||||
|
|
||||||
|
return &gen.Span{
|
||||||
|
TraceIdHigh: int64(data.SpanContext.TraceID.High),
|
||||||
|
TraceIdLow: int64(data.SpanContext.TraceID.Low),
|
||||||
|
SpanId: int64(data.SpanContext.SpanID),
|
||||||
|
ParentSpanId: int64(data.ParentSpanID),
|
||||||
|
OperationName: data.Name, // TODO: if span kind is added then add prefix "Sent"/"Recv"
|
||||||
|
Flags: int32(data.SpanContext.TraceOptions),
|
||||||
|
StartTime: data.StartTime.UnixNano() / 1000,
|
||||||
|
Duration: data.EndTime.Sub(data.StartTime).Nanoseconds() / 1000,
|
||||||
|
Tags: tags,
|
||||||
|
Logs: logs,
|
||||||
|
// TODO: goes with Links.
|
||||||
|
// References: refs,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// TODO(rghetia): remove interface{}. see https://github.com/open-telemetry/opentelemetry-go/pull/112/files#r321444786
|
||||||
|
func attributeToTag(key string, a interface{}) *gen.Tag {
|
||||||
|
var tag *gen.Tag
|
||||||
|
switch value := a.(type) {
|
||||||
|
case bool:
|
||||||
|
tag = &gen.Tag{
|
||||||
|
Key: key,
|
||||||
|
VBool: &value,
|
||||||
|
VType: gen.TagType_BOOL,
|
||||||
|
}
|
||||||
|
case string:
|
||||||
|
tag = &gen.Tag{
|
||||||
|
Key: key,
|
||||||
|
VStr: &value,
|
||||||
|
VType: gen.TagType_STRING,
|
||||||
|
}
|
||||||
|
case int64:
|
||||||
|
tag = &gen.Tag{
|
||||||
|
Key: key,
|
||||||
|
VLong: &value,
|
||||||
|
VType: gen.TagType_LONG,
|
||||||
|
}
|
||||||
|
case int32:
|
||||||
|
v := int64(value)
|
||||||
|
tag = &gen.Tag{
|
||||||
|
Key: key,
|
||||||
|
VLong: &v,
|
||||||
|
VType: gen.TagType_LONG,
|
||||||
|
}
|
||||||
|
case float64:
|
||||||
|
v := float64(value)
|
||||||
|
tag = &gen.Tag{
|
||||||
|
Key: key,
|
||||||
|
VDouble: &v,
|
||||||
|
VType: gen.TagType_DOUBLE,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return tag
|
||||||
|
}
|
||||||
|
|
||||||
|
// Flush waits for exported trace spans to be uploaded.
|
||||||
|
//
|
||||||
|
// This is useful if your program is ending and you do not want to lose recent spans.
|
||||||
|
func (e *Exporter) Flush() {
|
||||||
|
e.bundler.Flush()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (e *Exporter) upload(spans []*gen.Span) error {
|
||||||
|
batch := &gen.Batch{
|
||||||
|
Spans: spans,
|
||||||
|
Process: e.process,
|
||||||
|
}
|
||||||
|
if e.endpoint != "" {
|
||||||
|
return e.uploadCollector(batch)
|
||||||
|
}
|
||||||
|
return e.uploadAgent(batch)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (e *Exporter) uploadAgent(batch *gen.Batch) error {
|
||||||
|
return e.client.EmitBatch(batch)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (e *Exporter) uploadCollector(batch *gen.Batch) error {
|
||||||
|
body, err := serialize(batch)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
req, err := http.NewRequest("POST", e.endpoint, body)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
if e.username != "" && e.password != "" {
|
||||||
|
req.SetBasicAuth(e.username, e.password)
|
||||||
|
}
|
||||||
|
req.Header.Set("Content-Type", "application/x-thrift")
|
||||||
|
|
||||||
|
resp, err := http.DefaultClient.Do(req)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
_, _ = io.Copy(ioutil.Discard, resp.Body)
|
||||||
|
resp.Body.Close()
|
||||||
|
|
||||||
|
if resp.StatusCode < 200 || resp.StatusCode >= 300 {
|
||||||
|
return fmt.Errorf("failed to upload traces; HTTP status code: %d", resp.StatusCode)
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func serialize(obj thrift.TStruct) (*bytes.Buffer, error) {
|
||||||
|
buf := thrift.NewTMemoryBuffer()
|
||||||
|
if err := obj.Write(thrift.NewTBinaryProtocolTransport(buf)); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
return buf.Buffer, nil
|
||||||
|
}
|
||||||
97
exporter/trace/jaeger/jaeger_test.go
Normal file
97
exporter/trace/jaeger/jaeger_test.go
Normal file
@@ -0,0 +1,97 @@
|
|||||||
|
// Copyright 2019, OpenTelemetry Authors
|
||||||
|
//
|
||||||
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
// you may not use this file except in compliance with the License.
|
||||||
|
// You may obtain a copy of the License at
|
||||||
|
//
|
||||||
|
// http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
//
|
||||||
|
// Unless required by applicable law or agreed to in writing, software
|
||||||
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
package jaeger
|
||||||
|
|
||||||
|
import (
|
||||||
|
"sort"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/google/go-cmp/cmp"
|
||||||
|
"google.golang.org/grpc/codes"
|
||||||
|
|
||||||
|
"go.opentelemetry.io/api/core"
|
||||||
|
gen "go.opentelemetry.io/exporter/trace/jaeger/internal/gen-go/jaeger"
|
||||||
|
"go.opentelemetry.io/sdk/trace"
|
||||||
|
)
|
||||||
|
|
||||||
|
// TODO(rghetia): Test export.
|
||||||
|
|
||||||
|
func Test_spanDataToThrift(t *testing.T) {
|
||||||
|
now := time.Now()
|
||||||
|
traceID := core.TraceID{High: 0x0102030405060708, Low: 0x090a0b0c0d0e0f10}
|
||||||
|
spanID := uint64(0x0102030405060708)
|
||||||
|
|
||||||
|
keyValue := "value"
|
||||||
|
statusCodeValue := int64(2)
|
||||||
|
doubleValue := float64(123.456)
|
||||||
|
boolTrue := true
|
||||||
|
statusMessage := "Unknown"
|
||||||
|
|
||||||
|
tests := []struct {
|
||||||
|
name string
|
||||||
|
data *trace.SpanData
|
||||||
|
want *gen.Span
|
||||||
|
}{
|
||||||
|
{
|
||||||
|
name: "no parent",
|
||||||
|
data: &trace.SpanData{
|
||||||
|
SpanContext: core.SpanContext{
|
||||||
|
TraceID: traceID,
|
||||||
|
SpanID: spanID,
|
||||||
|
},
|
||||||
|
Name: "/foo",
|
||||||
|
StartTime: now,
|
||||||
|
EndTime: now,
|
||||||
|
Attributes: map[string]interface{}{
|
||||||
|
"double": doubleValue,
|
||||||
|
"key": keyValue,
|
||||||
|
},
|
||||||
|
// TODO: [rghetia] add events test after event is concrete type.
|
||||||
|
Status: codes.Unknown,
|
||||||
|
},
|
||||||
|
want: &gen.Span{
|
||||||
|
TraceIdLow: 651345242494996240,
|
||||||
|
TraceIdHigh: 72623859790382856,
|
||||||
|
SpanId: 72623859790382856,
|
||||||
|
OperationName: "/foo",
|
||||||
|
StartTime: now.UnixNano() / 1000,
|
||||||
|
Duration: 0,
|
||||||
|
Tags: []*gen.Tag{
|
||||||
|
{Key: "double", VType: gen.TagType_DOUBLE, VDouble: &doubleValue},
|
||||||
|
{Key: "key", VType: gen.TagType_STRING, VStr: &keyValue},
|
||||||
|
{Key: "error", VType: gen.TagType_BOOL, VBool: &boolTrue},
|
||||||
|
{Key: "status.code", VType: gen.TagType_LONG, VLong: &statusCodeValue},
|
||||||
|
{Key: "status.message", VType: gen.TagType_STRING, VStr: &statusMessage},
|
||||||
|
},
|
||||||
|
// TODO [rghetia]: check Logs when event is added.
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
for _, tt := range tests {
|
||||||
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
|
got := spanDataToThrift(tt.data)
|
||||||
|
sort.Slice(got.Tags, func(i, j int) bool {
|
||||||
|
return got.Tags[i].Key < got.Tags[j].Key
|
||||||
|
})
|
||||||
|
sort.Slice(tt.want.Tags, func(i, j int) bool {
|
||||||
|
return tt.want.Tags[i].Key < tt.want.Tags[j].Key
|
||||||
|
})
|
||||||
|
if diff := cmp.Diff(got, tt.want); diff != "" {
|
||||||
|
t.Errorf("Diff%v", diff)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
2
go.mod
2
go.mod
@@ -3,10 +3,12 @@ module go.opentelemetry.io
|
|||||||
go 1.12
|
go 1.12
|
||||||
|
|
||||||
require (
|
require (
|
||||||
|
github.com/apache/thrift v0.12.0
|
||||||
github.com/client9/misspell v0.3.4
|
github.com/client9/misspell v0.3.4
|
||||||
github.com/golangci/golangci-lint v1.17.1
|
github.com/golangci/golangci-lint v1.17.1
|
||||||
github.com/google/go-cmp v0.3.0
|
github.com/google/go-cmp v0.3.0
|
||||||
github.com/hashicorp/golang-lru v0.5.3
|
github.com/hashicorp/golang-lru v0.5.3
|
||||||
github.com/lightstep/tracecontext.go v0.0.0-20181129014701-1757c391b1ac
|
github.com/lightstep/tracecontext.go v0.0.0-20181129014701-1757c391b1ac
|
||||||
|
google.golang.org/api v0.9.0
|
||||||
google.golang.org/grpc v1.22.1
|
google.golang.org/grpc v1.22.1
|
||||||
)
|
)
|
||||||
|
|||||||
56
go.sum
56
go.sum
@@ -1,9 +1,13 @@
|
|||||||
cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
|
cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
|
||||||
|
cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
|
||||||
|
cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU=
|
||||||
github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ=
|
github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ=
|
||||||
github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
|
github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
|
||||||
github.com/OpenPeeDeeP/depguard v0.0.0-20180806142446-a69c782687b2 h1:HTOmFEEYrWi4MW5ZKUx6xfeyM10Sx3kQF65xiQJMPYA=
|
github.com/OpenPeeDeeP/depguard v0.0.0-20180806142446-a69c782687b2 h1:HTOmFEEYrWi4MW5ZKUx6xfeyM10Sx3kQF65xiQJMPYA=
|
||||||
github.com/OpenPeeDeeP/depguard v0.0.0-20180806142446-a69c782687b2/go.mod h1:7/4sitnI9YlQgTLLk734QlzXT8DuHVnAyztLplQjk+o=
|
github.com/OpenPeeDeeP/depguard v0.0.0-20180806142446-a69c782687b2/go.mod h1:7/4sitnI9YlQgTLLk734QlzXT8DuHVnAyztLplQjk+o=
|
||||||
github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg=
|
github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg=
|
||||||
|
github.com/apache/thrift v0.12.0 h1:pODnxUFNcjP9UTLZGTdeh+j16A8lJbRvD3rOtrk/7bs=
|
||||||
|
github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ=
|
||||||
github.com/client9/misspell v0.3.4 h1:ta993UF76GwbvJcIo3Y68y/M3WxlpEHPWIGDkJYwzJI=
|
github.com/client9/misspell v0.3.4 h1:ta993UF76GwbvJcIo3Y68y/M3WxlpEHPWIGDkJYwzJI=
|
||||||
github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
|
github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
|
||||||
github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
|
github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
|
||||||
@@ -46,8 +50,11 @@ github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfU
|
|||||||
github.com/golang/mock v1.0.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
|
github.com/golang/mock v1.0.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
|
||||||
github.com/golang/mock v1.1.1 h1:G5FRp8JnTd7RQH5kemVNlMeyXQAztQ3mOWV95KxsXH8=
|
github.com/golang/mock v1.1.1 h1:G5FRp8JnTd7RQH5kemVNlMeyXQAztQ3mOWV95KxsXH8=
|
||||||
github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
|
github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
|
||||||
|
github.com/golang/mock v1.2.0 h1:28o5sBqPkBsMGnC6b4MvE2TzSr5/AT4c/1fLqVGIwlk=
|
||||||
|
github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
|
||||||
github.com/golang/protobuf v1.2.0 h1:P3YflyNX/ehuJFLhxviNdFxQPkGK5cDcApsge1SqnvM=
|
github.com/golang/protobuf v1.2.0 h1:P3YflyNX/ehuJFLhxviNdFxQPkGK5cDcApsge1SqnvM=
|
||||||
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
||||||
|
github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
||||||
github.com/golangci/check v0.0.0-20180506172741-cfe4005ccda2 h1:23T5iq8rbUYlhpt5DB4XJkc6BU31uODLD1o1gKvZmD0=
|
github.com/golangci/check v0.0.0-20180506172741-cfe4005ccda2 h1:23T5iq8rbUYlhpt5DB4XJkc6BU31uODLD1o1gKvZmD0=
|
||||||
github.com/golangci/check v0.0.0-20180506172741-cfe4005ccda2/go.mod h1:k9Qvh+8juN+UKMCS/3jFtGICgW8O96FVaZsaxdzDkR4=
|
github.com/golangci/check v0.0.0-20180506172741-cfe4005ccda2/go.mod h1:k9Qvh+8juN+UKMCS/3jFtGICgW8O96FVaZsaxdzDkR4=
|
||||||
github.com/golangci/dupl v0.0.0-20180902072040-3e9179ac440a h1:w8hkcTqaFpzKqonE9uMCefW1WDie15eSP/4MssdenaM=
|
github.com/golangci/dupl v0.0.0-20180902072040-3e9179ac440a h1:w8hkcTqaFpzKqonE9uMCefW1WDie15eSP/4MssdenaM=
|
||||||
@@ -82,11 +89,18 @@ github.com/golangci/revgrep v0.0.0-20180526074752-d9c87f5ffaf0 h1:HVfrLniijszjS1
|
|||||||
github.com/golangci/revgrep v0.0.0-20180526074752-d9c87f5ffaf0/go.mod h1:qOQCunEYvmd/TLamH+7LlVccLvUH5kZNhbCgTHoBbp4=
|
github.com/golangci/revgrep v0.0.0-20180526074752-d9c87f5ffaf0/go.mod h1:qOQCunEYvmd/TLamH+7LlVccLvUH5kZNhbCgTHoBbp4=
|
||||||
github.com/golangci/unconvert v0.0.0-20180507085042-28b1c447d1f4 h1:zwtduBRr5SSWhqsYNgcuWO2kFlpdOZbP0+yRjmvPGys=
|
github.com/golangci/unconvert v0.0.0-20180507085042-28b1c447d1f4 h1:zwtduBRr5SSWhqsYNgcuWO2kFlpdOZbP0+yRjmvPGys=
|
||||||
github.com/golangci/unconvert v0.0.0-20180507085042-28b1c447d1f4/go.mod h1:Izgrg8RkN3rCIMLGE9CyYmU9pY2Jer6DgANEnZ/L/cQ=
|
github.com/golangci/unconvert v0.0.0-20180507085042-28b1c447d1f4/go.mod h1:Izgrg8RkN3rCIMLGE9CyYmU9pY2Jer6DgANEnZ/L/cQ=
|
||||||
|
github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
|
||||||
github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M=
|
github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M=
|
||||||
github.com/google/go-cmp v0.3.0 h1:crn/baboCvb5fXaQ0IJ1SGTsTVrWpDsCWC8EGETZijY=
|
github.com/google/go-cmp v0.3.0 h1:crn/baboCvb5fXaQ0IJ1SGTsTVrWpDsCWC8EGETZijY=
|
||||||
github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
|
github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
|
||||||
|
github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
|
||||||
|
github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
|
||||||
|
github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg=
|
||||||
|
github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk=
|
||||||
github.com/gostaticanalysis/analysisutil v0.0.0-20190318220348-4088753ea4d3 h1:JVnpOZS+qxli+rgVl98ILOXVNbW+kb5wcxeGx8ShUIw=
|
github.com/gostaticanalysis/analysisutil v0.0.0-20190318220348-4088753ea4d3 h1:JVnpOZS+qxli+rgVl98ILOXVNbW+kb5wcxeGx8ShUIw=
|
||||||
github.com/gostaticanalysis/analysisutil v0.0.0-20190318220348-4088753ea4d3/go.mod h1:eEOZF4jCKGi+aprrirO9e7WKB3beBRtWgqGunKl6pKE=
|
github.com/gostaticanalysis/analysisutil v0.0.0-20190318220348-4088753ea4d3/go.mod h1:eEOZF4jCKGi+aprrirO9e7WKB3beBRtWgqGunKl6pKE=
|
||||||
|
github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
|
||||||
|
github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
|
||||||
github.com/hashicorp/golang-lru v0.5.3 h1:YPkqC67at8FYaadspW/6uE0COsBxS2656RLEr8Bppgk=
|
github.com/hashicorp/golang-lru v0.5.3 h1:YPkqC67at8FYaadspW/6uE0COsBxS2656RLEr8Bppgk=
|
||||||
github.com/hashicorp/golang-lru v0.5.3/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4=
|
github.com/hashicorp/golang-lru v0.5.3/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4=
|
||||||
github.com/hashicorp/hcl v0.0.0-20180404174102-ef8a98b0bbce h1:xdsDDbiBDQTKASoGEZ+pEmF1OnWuu8AQ9I8iNbHNeno=
|
github.com/hashicorp/hcl v0.0.0-20180404174102-ef8a98b0bbce h1:xdsDDbiBDQTKASoGEZ+pEmF1OnWuu8AQ9I8iNbHNeno=
|
||||||
@@ -95,6 +109,7 @@ github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI=
|
|||||||
github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
|
github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
|
||||||
github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM=
|
github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM=
|
||||||
github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8=
|
github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8=
|
||||||
|
github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU=
|
||||||
github.com/kisielk/gotool v0.0.0-20161130080628-0de1eaf82fa3/go.mod h1:jxZFDH7ILpTPQTk+E2s+z4CUas9lVNjIuKR4c5/zKgM=
|
github.com/kisielk/gotool v0.0.0-20161130080628-0de1eaf82fa3/go.mod h1:jxZFDH7ILpTPQTk+E2s+z4CUas9lVNjIuKR4c5/zKgM=
|
||||||
github.com/kisielk/gotool v1.0.0 h1:AV2c/EiW3KqPNT9ZKl07ehoAGi4C5/01Cfbblndcapg=
|
github.com/kisielk/gotool v1.0.0 h1:AV2c/EiW3KqPNT9ZKl07ehoAGi4C5/01Cfbblndcapg=
|
||||||
github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
|
github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
|
||||||
@@ -171,42 +186,80 @@ github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyC
|
|||||||
github.com/valyala/fasthttp v1.2.0/go.mod h1:4vX61m6KN+xDduDNwXrhIAVZaZaZiQ1luJk8LWSxF3s=
|
github.com/valyala/fasthttp v1.2.0/go.mod h1:4vX61m6KN+xDduDNwXrhIAVZaZaZiQ1luJk8LWSxF3s=
|
||||||
github.com/valyala/quicktemplate v1.1.1/go.mod h1:EH+4AkTd43SvgIbQHYu59/cJyxDoOVRUAfrukLPuGJ4=
|
github.com/valyala/quicktemplate v1.1.1/go.mod h1:EH+4AkTd43SvgIbQHYu59/cJyxDoOVRUAfrukLPuGJ4=
|
||||||
github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio=
|
github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio=
|
||||||
|
go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU=
|
||||||
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
|
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
|
||||||
golang.org/x/crypto v0.0.0-20190313024323-a1f597ede03a h1:YX8ljsm6wXlHZO+aRz9Exqr0evNhKRNe5K/gi+zKh4U=
|
golang.org/x/crypto v0.0.0-20190313024323-a1f597ede03a h1:YX8ljsm6wXlHZO+aRz9Exqr0evNhKRNe5K/gi+zKh4U=
|
||||||
golang.org/x/crypto v0.0.0-20190313024323-a1f597ede03a/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
|
golang.org/x/crypto v0.0.0-20190313024323-a1f597ede03a/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
|
||||||
|
golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
|
||||||
|
golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
|
||||||
|
golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU=
|
||||||
|
golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
|
||||||
golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
|
golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
|
||||||
|
golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
|
||||||
golang.org/x/net v0.0.0-20170915142106-8351a756f30f/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
golang.org/x/net v0.0.0-20170915142106-8351a756f30f/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||||
|
golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||||
|
golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||||
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||||
golang.org/x/net v0.0.0-20180911220305-26e67e76b6c3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
golang.org/x/net v0.0.0-20180911220305-26e67e76b6c3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||||
|
golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||||
|
golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||||
golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
||||||
golang.org/x/net v0.0.0-20190313220215-9f648a60d977 h1:actzWV6iWn3GLqN8dZjzsB+CLt+gaV2+wsxroxiQI8I=
|
golang.org/x/net v0.0.0-20190313220215-9f648a60d977 h1:actzWV6iWn3GLqN8dZjzsB+CLt+gaV2+wsxroxiQI8I=
|
||||||
golang.org/x/net v0.0.0-20190313220215-9f648a60d977/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
golang.org/x/net v0.0.0-20190313220215-9f648a60d977/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
||||||
|
golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
||||||
golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
|
golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
|
||||||
|
golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
|
||||||
|
golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
|
||||||
golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
|
golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
|
golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
|
golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
golang.org/x/sync v0.0.0-20190423024810-112230192c58 h1:8gQV6CLnAEikrhgkHFbMAEhagSSnXWGV915qUMm9mrU=
|
golang.org/x/sync v0.0.0-20190423024810-112230192c58 h1:8gQV6CLnAEikrhgkHFbMAEhagSSnXWGV915qUMm9mrU=
|
||||||
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
golang.org/x/sys v0.0.0-20171026204733-164713f0dfce/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
golang.org/x/sys v0.0.0-20171026204733-164713f0dfce/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||||
|
golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||||
golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||||
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||||
golang.org/x/sys v0.0.0-20190312061237-fead79001313 h1:pczuHS43Cp2ktBEEmLwScxgjWsBSzdaQiKzUyf3DTTc=
|
golang.org/x/sys v0.0.0-20190312061237-fead79001313 h1:pczuHS43Cp2ktBEEmLwScxgjWsBSzdaQiKzUyf3DTTc=
|
||||||
golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||||
|
golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b h1:ag/x1USPSsqHud38I9BAC88qdNLDHHtQ4mlgQIZPPNA=
|
||||||
|
golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||||
golang.org/x/text v0.0.0-20170915090833-1cbadb444a80/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
golang.org/x/text v0.0.0-20170915090833-1cbadb444a80/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
||||||
golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg=
|
golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg=
|
||||||
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
||||||
|
golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
||||||
|
golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs=
|
||||||
|
golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk=
|
||||||
|
golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
|
||||||
golang.org/x/tools v0.0.0-20170915040203-e531a2a1c15f/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
golang.org/x/tools v0.0.0-20170915040203-e531a2a1c15f/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
||||||
golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
||||||
|
golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
||||||
golang.org/x/tools v0.0.0-20181117154741-2ddaf7f79a09/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
golang.org/x/tools v0.0.0-20181117154741-2ddaf7f79a09/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
||||||
golang.org/x/tools v0.0.0-20190110163146-51295c7ec13a/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
golang.org/x/tools v0.0.0-20190110163146-51295c7ec13a/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
||||||
|
golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
||||||
golang.org/x/tools v0.0.0-20190121143147-24cd39ecf745/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
golang.org/x/tools v0.0.0-20190121143147-24cd39ecf745/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
||||||
|
golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY=
|
||||||
golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||||
golang.org/x/tools v0.0.0-20190311215038-5c2858a9cfe5/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
golang.org/x/tools v0.0.0-20190311215038-5c2858a9cfe5/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||||
|
golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||||
golang.org/x/tools v0.0.0-20190322203728-c1a832b0ad89/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
golang.org/x/tools v0.0.0-20190322203728-c1a832b0ad89/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||||
|
golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
|
||||||
golang.org/x/tools v0.0.0-20190521203540-521d6ed310dd/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
|
golang.org/x/tools v0.0.0-20190521203540-521d6ed310dd/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
|
||||||
golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135 h1:5Beo0mZN8dRzgrMMkDp0jc8YXQKx9DiJ2k1dkvGsn5A=
|
golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135 h1:5Beo0mZN8dRzgrMMkDp0jc8YXQKx9DiJ2k1dkvGsn5A=
|
||||||
golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
|
golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
|
||||||
|
google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE=
|
||||||
|
google.golang.org/api v0.9.0 h1:jbyannxz0XFD3zdjgrSUsaJbgpH4eTrkdhRChkHPfO8=
|
||||||
|
google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg=
|
||||||
google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
|
google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
|
||||||
|
google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
|
||||||
|
google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
|
||||||
google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8 h1:Nw54tB0rB7hY/N0NQvRW8DG4Yk3Q6T9cu9RcFQDu1tc=
|
google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8 h1:Nw54tB0rB7hY/N0NQvRW8DG4Yk3Q6T9cu9RcFQDu1tc=
|
||||||
google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
|
google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
|
||||||
|
google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
|
||||||
|
google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
|
||||||
|
google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
|
||||||
|
google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
|
||||||
|
google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38=
|
||||||
google.golang.org/grpc v1.22.1 h1:/7cs52RnTJmD43s3uxzlq2U7nqVTd/37viQwMrMNlOM=
|
google.golang.org/grpc v1.22.1 h1:/7cs52RnTJmD43s3uxzlq2U7nqVTd/37viQwMrMNlOM=
|
||||||
google.golang.org/grpc v1.22.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
|
google.golang.org/grpc v1.22.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
|
||||||
gopkg.in/airbrake/gobrake.v2 v2.0.9 h1:7z2uVWwn7oVeeugY1DtlPAy5H+KYgB1KeKTnqjNatLo=
|
gopkg.in/airbrake/gobrake.v2 v2.0.9 h1:7z2uVWwn7oVeeugY1DtlPAy5H+KYgB1KeKTnqjNatLo=
|
||||||
@@ -224,6 +277,9 @@ gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWD
|
|||||||
gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74=
|
gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74=
|
||||||
gopkg.in/yaml.v2 v2.2.1 h1:mUhvW9EsL+naU5Q3cakzfE91YhliOondGd6ZrsDBHQE=
|
gopkg.in/yaml.v2 v2.2.1 h1:mUhvW9EsL+naU5Q3cakzfE91YhliOondGd6ZrsDBHQE=
|
||||||
gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
||||||
|
honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
||||||
|
honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
||||||
|
honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
||||||
honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
|
||||||
mvdan.cc/interfacer v0.0.0-20180901003855-c20040233aed h1:WX1yoOaKQfddO/mLzdV4wptyWgoH/6hwLs7QHTixo0I=
|
mvdan.cc/interfacer v0.0.0-20180901003855-c20040233aed h1:WX1yoOaKQfddO/mLzdV4wptyWgoH/6hwLs7QHTixo0I=
|
||||||
mvdan.cc/interfacer v0.0.0-20180901003855-c20040233aed/go.mod h1:Xkxe497xwlCKkIaQYRfC7CSLworTXY9RMqwhhCm+8Nc=
|
mvdan.cc/interfacer v0.0.0-20180901003855-c20040233aed/go.mod h1:Xkxe497xwlCKkIaQYRfC7CSLworTXY9RMqwhhCm+8Nc=
|
||||||
|
|||||||
@@ -86,7 +86,7 @@ type SpanData struct {
|
|||||||
EndTime time.Time
|
EndTime time.Time
|
||||||
// The values of Attributes each have type string, bool, or int64.
|
// The values of Attributes each have type string, bool, or int64.
|
||||||
Attributes map[string]interface{}
|
Attributes map[string]interface{}
|
||||||
MessageEvents []event
|
MessageEvents []Event
|
||||||
Status codes.Code
|
Status codes.Code
|
||||||
HasRemoteParent bool
|
HasRemoteParent bool
|
||||||
DroppedAttributeCount int
|
DroppedAttributeCount int
|
||||||
@@ -96,3 +96,16 @@ type SpanData struct {
|
|||||||
// ChildSpanCount holds the number of child span created for this span.
|
// ChildSpanCount holds the number of child span created for this span.
|
||||||
ChildSpanCount int
|
ChildSpanCount int
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Event is used to describe an Event with a message string and set of
|
||||||
|
// Attributes.
|
||||||
|
type Event struct {
|
||||||
|
// Message describes the Event.
|
||||||
|
Message string
|
||||||
|
|
||||||
|
// Attributes contains a list of keyvalue pairs.
|
||||||
|
Attributes []core.KeyValue
|
||||||
|
|
||||||
|
// Time is the time at which this event was recorded.
|
||||||
|
Time time.Time
|
||||||
|
}
|
||||||
|
|||||||
@@ -158,10 +158,10 @@ func (s *span) AddEvent(ctx context.Context, msg string, attrs ...core.KeyValue)
|
|||||||
func (s *span) addEventWithTimestamp(timestamp time.Time, msg string, attrs ...core.KeyValue) {
|
func (s *span) addEventWithTimestamp(timestamp time.Time, msg string, attrs ...core.KeyValue) {
|
||||||
s.mu.Lock()
|
s.mu.Lock()
|
||||||
defer s.mu.Unlock()
|
defer s.mu.Unlock()
|
||||||
s.messageEvents.add(event{
|
s.messageEvents.add(Event{
|
||||||
msg: msg,
|
Message: msg,
|
||||||
attributes: attrs,
|
Attributes: attrs,
|
||||||
time: timestamp,
|
Time: timestamp,
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -210,10 +210,10 @@ func (s *span) makeSpanData() *SpanData {
|
|||||||
return &sd
|
return &sd
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *span) interfaceArrayToMessageEventArray() []event {
|
func (s *span) interfaceArrayToMessageEventArray() []Event {
|
||||||
messageEventArr := make([]event, 0)
|
messageEventArr := make([]Event, 0)
|
||||||
for _, value := range s.messageEvents.queue {
|
for _, value := range s.messageEvents.queue {
|
||||||
messageEventArr = append(messageEventArr, value.(event))
|
messageEventArr = append(messageEventArr, value.(Event))
|
||||||
}
|
}
|
||||||
return messageEventArr
|
return messageEventArr
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -245,8 +245,8 @@ func TestEvents(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
for i := range got.MessageEvents {
|
for i := range got.MessageEvents {
|
||||||
if !checkTime(&got.MessageEvents[i].time) {
|
if !checkTime(&got.MessageEvents[i].Time) {
|
||||||
t.Error("exporting span: expected nonzero event Time")
|
t.Error("exporting span: expected nonzero Event Time")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -258,12 +258,12 @@ func TestEvents(t *testing.T) {
|
|||||||
ParentSpanID: sid,
|
ParentSpanID: sid,
|
||||||
Name: "span0",
|
Name: "span0",
|
||||||
HasRemoteParent: true,
|
HasRemoteParent: true,
|
||||||
MessageEvents: []event{
|
MessageEvents: []Event{
|
||||||
{msg: "foo", attributes: []core.KeyValue{k1v1}},
|
{Message: "foo", Attributes: []core.KeyValue{k1v1}},
|
||||||
{msg: "bar", attributes: []core.KeyValue{k2v2, k3v3}},
|
{Message: "bar", Attributes: []core.KeyValue{k2v2, k3v3}},
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(event{})); diff != "" {
|
if diff := cmp.Diff(got, want, cmp.AllowUnexported(Event{})); diff != "" {
|
||||||
t.Errorf("Message Events: -got +want %s", diff)
|
t.Errorf("Message Events: -got +want %s", diff)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@@ -292,8 +292,8 @@ func TestEventsOverLimit(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
for i := range got.MessageEvents {
|
for i := range got.MessageEvents {
|
||||||
if !checkTime(&got.MessageEvents[i].time) {
|
if !checkTime(&got.MessageEvents[i].Time) {
|
||||||
t.Error("exporting span: expected nonzero event Time")
|
t.Error("exporting span: expected nonzero Event Time")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -304,14 +304,14 @@ func TestEventsOverLimit(t *testing.T) {
|
|||||||
},
|
},
|
||||||
ParentSpanID: sid,
|
ParentSpanID: sid,
|
||||||
Name: "span0",
|
Name: "span0",
|
||||||
MessageEvents: []event{
|
MessageEvents: []Event{
|
||||||
{msg: "foo", attributes: []core.KeyValue{k1v1}},
|
{Message: "foo", Attributes: []core.KeyValue{k1v1}},
|
||||||
{msg: "bar", attributes: []core.KeyValue{k2v2, k3v3}},
|
{Message: "bar", Attributes: []core.KeyValue{k2v2, k3v3}},
|
||||||
},
|
},
|
||||||
DroppedMessageEventCount: 2,
|
DroppedMessageEventCount: 2,
|
||||||
HasRemoteParent: true,
|
HasRemoteParent: true,
|
||||||
}
|
}
|
||||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(event{})); diff != "" {
|
if diff := cmp.Diff(got, want, cmp.AllowUnexported(Event{})); diff != "" {
|
||||||
t.Errorf("Message Event over limit: -got +want %s", diff)
|
t.Errorf("Message Event over limit: -got +want %s", diff)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
Reference in New Issue
Block a user