[VOL-4293] OpenONU Adapter update for gRPC migration
Change-Id: I05300d3b95b878f44576a99a05f53f52fdc0cda1
diff --git a/.gitignore b/.gitignore
index c4b2953..4bb60e7 100644
--- a/.gitignore
+++ b/.gitignore
@@ -2,6 +2,9 @@
.idea
exportToHTML
+# vscode
+.vscode
+
# Python
*.pyc
diff --git a/.golangci.yml b/.golangci.yml
index 7478787..b2483cb 100644
--- a/.golangci.yml
+++ b/.golangci.yml
@@ -55,48 +55,8 @@
linters:
- gosec
exclude:
- - "don't use underscores in Go names; method Adapter_descriptor"
- - "don't use underscores in Go names; method Device_types"
- - "don't use underscores in Go names; method Adopt_device"
- - "don't use underscores in Go names; method Reconcile_device"
- - "don't use underscores in Go names; method Abandon_device"
- - "don't use underscores in Go names; method Disable_device"
- - "don't use underscores in Go names; method Reenable_device"
- - "don't use underscores in Go names; method Reboot_device"
- - "don't use underscores in Go names; method Self_test_device"
- - "don't use underscores in Go names; method Delete_device"
- - "don't use underscores in Go names; method Get_device_details"
- - "don't use underscores in Go names; method Update_flows_bulk"
- - "don't use underscores in Go names; method Update_flows_incrementally"
- - "don't use underscores in Go names; method Update_pm_config"
- - "don't use underscores in Go names; method Receive_packet_out"
- - "don't use underscores in Go names; method Suppress_event"
- - "don't use underscores in Go names; method Unsuppress_event"
- - "don't use underscores in Go names; method Get_ofp_device_info"
- - "don't use underscores in Go names; method Get_ofp_port_info"
- - "don't use underscores in Go names; method Process_inter_adapter_message"
- - "don't use underscores in Go names; method Download_image"
- - "don't use underscores in Go names; method Get_image_download_status"
- - "don't use underscores in Go names; method Cancel_image_download"
- - "don't use underscores in Go names; method Activate_image_update"
- - "don't use underscores in Go names; method Revert_image_update"
- - "don't use underscores in Go names; method Disable_port"
- - "don't use underscores in Go names; method Enable_port"
- - "don't use underscores in Go names; method Child_device_lost"
- - "don't use underscores in Go names; method Start_omci_test"
- - "don't use underscores in Go names; method Get_ext_value"
- - "don't use underscores in Go names; method Single_get_value_request"
- - "don't use underscores in Go names; method Download_onu_image"
- - "don't use underscores in Go names; method Get_onu_image_status"
- - "don't use underscores in Go names; method Abort_onu_image_upgrade"
- - "don't use underscores in Go names; method Get_onu_images"
- - "don't use underscores in Go names; method Activate_onu_image"
- - "don't use underscores in Go names; method Commit_onu_image"
- - "don't use underscores in Go names; method Process_tech_profile_instance_request"
- - "Error return value of `dh.coreProxy.PortStateUpdate` is not checked"
- "Error return value of `rxCallbackEntry.cbFunction` is not checked"
- "Error return value of `oo.sendNextRequest` is not checked"
- "Error return value of `oo.pDevOmciCC.send` is not checked"
- "Error return value of `onuDeviceEntry.mibDbClass` is not checked"
- - "Error return value of `handler.rebootDevice` is not checked"
exclude-use-default: false
diff --git a/Makefile b/Makefile
index 40525f1..cd082e0 100755
--- a/Makefile
+++ b/Makefile
@@ -64,18 +64,18 @@
## Local Development Helpers
local-protos: ## Copies a local version of the voltha-protos dependency into the vendor directory
ifdef LOCAL_PROTOS
- rm -rf vendor/github.com/opencord/voltha-protos/v4/go
- mkdir -p vendor/github.com/opencord/voltha-protos/v4/go
- cp -r ${LOCAL_PROTOS}/go/* vendor/github.com/opencord/voltha-protos/v4/go
- rm -rf vendor/github.com/opencord/voltha-protos/v4/go/vendor
+ rm -rf vendor/github.com/opencord/voltha-protos/v5/go
+ mkdir -p vendor/github.com/opencord/voltha-protos/v5/go
+ cp -r ${LOCAL_PROTOS}/go/* vendor/github.com/opencord/voltha-protos/v5/go
+ rm -rf vendor/github.com/opencord/voltha-protos/v5/go/vendor
endif
## Local Development Helpers
local-lib-go: ## Copies a local version of the voltha-lib-go dependency into the vendor directory
ifdef LOCAL_LIB_GO
- rm -rf vendor/github.com/opencord/voltha-lib-go/v5/pkg
- mkdir -p vendor/github.com/opencord/voltha-lib-go/v5/pkg
- cp -r ${LOCAL_LIB_GO}/pkg/* vendor/github.com/opencord/voltha-lib-go/v5/pkg/
+ rm -rf vendor/github.com/opencord/voltha-lib-go/v7/pkg
+ mkdir -p vendor/github.com/opencord/voltha-lib-go/v7/pkg
+ cp -r ${LOCAL_LIB_GO}/pkg/* vendor/github.com/opencord/voltha-lib-go/v7/pkg/
endif
build: docker-build ## Alias for 'docker build'
diff --git a/VERSION b/VERSION
index 0ed156c..227cea2 100755
--- a/VERSION
+++ b/VERSION
@@ -1 +1 @@
-1.4.1-dev231
+2.0.0
diff --git a/cmd/openonu-adapter/common.go b/cmd/openonu-adapter/common.go
index 1c0be5e..bf1bc68 100644
--- a/cmd/openonu-adapter/common.go
+++ b/cmd/openonu-adapter/common.go
@@ -18,7 +18,7 @@
package main
import (
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
)
var logger log.CLogger
diff --git a/cmd/openonu-adapter/main.go b/cmd/openonu-adapter/main.go
index c42cef4..ce5dc6f 100644
--- a/cmd/openonu-adapter/main.go
+++ b/cmd/openonu-adapter/main.go
@@ -28,38 +28,46 @@
"syscall"
"time"
- "github.com/opencord/voltha-lib-go/v5/pkg/adapters"
- "github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
- com "github.com/opencord/voltha-lib-go/v5/pkg/adapters/common"
- conf "github.com/opencord/voltha-lib-go/v5/pkg/config"
- "github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
- "github.com/opencord/voltha-lib-go/v5/pkg/events"
- "github.com/opencord/voltha-lib-go/v5/pkg/events/eventif"
- "github.com/opencord/voltha-lib-go/v5/pkg/kafka"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- "github.com/opencord/voltha-lib-go/v5/pkg/probe"
- "github.com/opencord/voltha-lib-go/v5/pkg/version"
- ic "github.com/opencord/voltha-protos/v4/go/inter_container"
- "github.com/opencord/voltha-protos/v4/go/voltha"
+ "github.com/golang/protobuf/ptypes/empty"
+ conf "github.com/opencord/voltha-lib-go/v7/pkg/config"
+ "github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+ "github.com/opencord/voltha-lib-go/v7/pkg/events"
+ "github.com/opencord/voltha-lib-go/v7/pkg/events/eventif"
+ vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+ "github.com/opencord/voltha-lib-go/v7/pkg/kafka"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ "github.com/opencord/voltha-lib-go/v7/pkg/probe"
+ "github.com/opencord/voltha-lib-go/v7/pkg/version"
+ "github.com/opencord/voltha-protos/v5/go/adapter_services"
+ "github.com/opencord/voltha-protos/v5/go/core"
+ "github.com/opencord/voltha-protos/v5/go/voltha"
+ "google.golang.org/grpc"
+
+ ic "github.com/opencord/voltha-protos/v5/go/inter_container"
"github.com/opencord/voltha-openonu-adapter-go/internal/pkg/config"
ac "github.com/opencord/voltha-openonu-adapter-go/internal/pkg/onuadaptercore"
)
+const (
+ clusterMessagingService = "cluster-message-service"
+ onuAdapterService = "onu-adapter-service"
+ kvService = "kv-service"
+ coreService = "core-service"
+)
+
type adapter struct {
//defaultAppName string
- instanceID string
- config *config.AdapterFlags
- iAdapter adapters.IAdapter // from Voltha interface adapters
- kafkaClient kafka.Client
- kvClient kvstore.Client
- kip kafka.InterContainerProxy
- coreProxy adapterif.CoreProxy
- adapterProxy adapterif.AdapterProxy
- eventProxy eventif.EventProxy
- halted bool
- exitChannel chan int
- receiverChannels []<-chan *ic.InterContainerMessage //from inter-container
+ instanceID string
+ config *config.AdapterFlags
+ kafkaClient kafka.Client
+ kvClient kvstore.Client
+ eventProxy eventif.EventProxy
+ grpcServer *vgrpc.GrpcServer
+ onuAdapter *ac.OpenONUAC
+ coreClient *vgrpc.Client
+ halted bool
+ exitChannel chan int
}
func newAdapter(cf *config.AdapterFlags) *adapter {
@@ -68,7 +76,6 @@
a.config = cf
a.halted = false
a.exitChannel = make(chan int, 1)
- a.receiverChannels = make([]<-chan *ic.InterContainerMessage, 0)
return &a
}
@@ -82,11 +89,10 @@
p = value.(*probe.Probe)
p.RegisterService(
ctx,
- "message-bus",
- "kv-store",
- "container-proxy",
- "core-request-handler",
- "register-with-core",
+ clusterMessagingService,
+ kvService,
+ onuAdapterService,
+ coreService,
)
}
}
@@ -97,60 +103,84 @@
logger.Fatalw(ctx, "error-setting-kv-client", log.Fields{"error": err})
}
- if p != nil {
- p.UpdateStatus(ctx, "kv-store", probe.ServiceStatusRunning)
- }
-
// Setup Log Config
cm := conf.NewConfigManager(ctx, a.kvClient, a.config.KVStoreType, a.config.KVStoreAddress, a.config.KVStoreTimeout)
go conf.StartLogLevelConfigProcessing(cm, ctx)
// Setup Kafka Client
- if a.kafkaClient, err = newKafkaClient(ctx, "sarama", a.config.KafkaAdapterAddress); err != nil {
+ if a.kafkaClient, err = newKafkaClient(ctx, "sarama", a.config.KafkaClusterAddress); err != nil {
logger.Fatalw(ctx, "Unsupported-common-client", log.Fields{"error": err})
}
- if p != nil {
- p.UpdateStatus(ctx, "message-bus", probe.ServiceStatusRunning)
+ // Start kafka communication with the broker
+ if err := kafka.StartAndWaitUntilKafkaConnectionIsUp(ctx, a.kafkaClient, a.config.HeartbeatCheckInterval, clusterMessagingService); err != nil {
+ logger.Fatal(ctx, "unable-to-connect-to-kafka")
}
- // Start the common InterContainer Proxy - retries as per program arguments or indefinitely per default
- if a.kip, err = a.startInterContainerProxy(ctx, a.config.KafkaReconnectRetries); err != nil {
- logger.Fatalw(ctx, "error-starting-inter-container-proxy", log.Fields{"error": err})
- //aborting the complete processing here (does notmake sense after set Retry number [else use -1 for infinite])
- return err
+ // Wait until connection to KV store is established
+ if err := WaitUntilKvStoreConnectionIsUp(ctx, a.kvClient, a.config.KVStoreTimeout, kvService); err != nil {
+ logger.Fatal(ctx, "unable-to-connect-to-kv-store")
}
- // Create the core proxy to handle requests to the Core
- a.coreProxy = com.NewCoreProxy(ctx, a.kip, a.config.Topic, a.config.CoreTopic)
-
- logger.Debugw(ctx, "create adapter proxy", log.Fields{"CoreTopic": a.config.CoreTopic})
- a.adapterProxy = com.NewAdapterProxy(ctx, a.kip, a.config.CoreTopic, cm.Backend)
-
// Create the event proxy to post events to KAFKA
a.eventProxy = events.NewEventProxy(events.MsgClient(a.kafkaClient), events.MsgTopic(kafka.Topic{Name: a.config.EventTopic}))
+ go func() {
+ if err := a.eventProxy.Start(); err != nil {
+ logger.Fatalw(ctx, "event-proxy-cannot-start", log.Fields{"error": err})
+ }
+ }()
+
+ // Create the Core client to handle requests to the Core. Note that the coreClient is an interface and needs to be
+ // cast to the appropriate grpc client by invoking GetCoreGrpcClient on the a.coreClient
+ if a.coreClient, err = vgrpc.NewClient(a.config.CoreEndpoint,
+ a.coreRestarted,
+ vgrpc.ActivityCheck(true)); err != nil {
+ logger.Fatal(ctx, "grpc-client-not-created")
+ }
+ // Start the core grpc client
+ go a.coreClient.Start(ctx, setAndTestCoreServiceHandler)
// Create the open ONU interface adapter
- if a.iAdapter, err = a.startVolthaInterfaceAdapter(ctx, a.kip, a.coreProxy, a.adapterProxy, a.eventProxy,
- a.config, cm); err != nil {
+ if a.onuAdapter, err = a.startVolthaInterfaceAdapter(ctx, a.coreClient, a.eventProxy, a.config, cm); err != nil {
logger.Fatalw(ctx, "error-starting-volthaInterfaceAdapter for OpenOnt", log.Fields{"error": err})
}
- // Register the core request handler
- if err = a.setupRequestHandler(ctx, a.instanceID, a.iAdapter); err != nil {
- logger.Fatalw(ctx, "error-setting-core-request-handler", log.Fields{"error": err})
- }
+ // Create and start the grpc server
+ a.grpcServer = vgrpc.NewGrpcServer(a.config.GrpcAddress, nil, false, p)
+
+ //Register the adapter service
+ a.addAdapterService(ctx, a.grpcServer, a.onuAdapter)
+
+ //Register the onu inter adapter service
+ a.addOnuInterAdapterService(ctx, a.grpcServer, a.onuAdapter)
+
+ go a.startGRPCService(ctx, a.grpcServer, onuAdapterService)
// Register this adapter to the Core - retries indefinitely
- if err = a.registerWithCore(ctx, -1); err != nil {
+ if err = a.registerWithCore(ctx, coreService, -1); err != nil {
logger.Fatalw(ctx, "error-registering-with-core", log.Fields{"error": err})
}
- // check the readiness and liveliness and update the probe status
+ // Start the readiness and liveliness check and update the probe status
a.checkServicesReadiness(ctx)
return err
}
+// TODO: Any action the adapter needs to do following a Core restart?
+func (a *adapter) coreRestarted(ctx context.Context, endPoint string) error {
+ logger.Errorw(ctx, "core-restarted", log.Fields{"endpoint": endPoint})
+ return nil
+}
+
+// setAndTestCoreServiceHandler is used to test whether the remote gRPC service is up
+func setAndTestCoreServiceHandler(ctx context.Context, conn *grpc.ClientConn) interface{} {
+ svc := core.NewCoreServiceClient(conn)
+ if h, err := svc.GetHealthStatus(ctx, &empty.Empty{}); err != nil || h.State != voltha.HealthStatus_HEALTHY {
+ return nil
+ }
+ return svc
+}
+
func (a *adapter) stop(ctx context.Context) {
// Stop leadership tracking
a.halted = true
@@ -168,9 +198,20 @@
a.kvClient.Close(ctx)
}
- if a.kip != nil {
- a.kip.Stop(ctx)
+ if a.eventProxy != nil {
+ a.eventProxy.Stop()
}
+
+ if a.kafkaClient != nil {
+ a.kafkaClient.Stop(ctx)
+ }
+
+ // Stop core client
+ if a.coreClient != nil {
+ a.coreClient.Stop(ctx)
+ }
+
+ // TODO: More cleanup
}
// #############################################
@@ -214,38 +255,10 @@
return nil
}
-func (a *adapter) startInterContainerProxy(ctx context.Context, retries int) (kafka.InterContainerProxy, error) {
- logger.Infow(ctx, "starting-intercontainer-messaging-proxy", log.Fields{"addr": a.config.KafkaAdapterAddress, "topic": a.config.Topic})
- var err error
- /* address config update acc. to [VOL-2736] */
- kip := kafka.NewInterContainerProxy(
- kafka.InterContainerAddress(a.config.KafkaAdapterAddress),
- kafka.MsgClient(a.kafkaClient),
- kafka.DefaultTopic(&kafka.Topic{Name: a.config.Topic}))
- count := 0
- for {
- if err = kip.Start(ctx); err != nil {
- logger.Warnw(ctx, "error-starting-messaging-proxy", log.Fields{"error": err, "retry": retries, "count": count})
- if retries == count {
- return nil, err
- }
- count++
- // Take a nap before retrying
- time.Sleep(2 * time.Second)
- } else {
- break
- }
- }
- probe.UpdateStatusFromContext(ctx, "container-proxy", probe.ServiceStatusRunning)
- logger.Info(ctx, "common-messaging-proxy-created")
- return kip, nil
-}
-
-func (a *adapter) startVolthaInterfaceAdapter(ctx context.Context, kip kafka.InterContainerProxy,
- cp adapterif.CoreProxy, ap adapterif.AdapterProxy, ep eventif.EventProxy,
+func (a *adapter) startVolthaInterfaceAdapter(ctx context.Context, cc *vgrpc.Client, ep eventif.EventProxy,
cfg *config.AdapterFlags, cm *conf.ConfigManager) (*ac.OpenONUAC, error) {
var err error
- sAcONU := ac.NewOpenONUAC(ctx, a.kip, cp, ap, ep, a.kvClient, cfg, cm)
+ sAcONU := ac.NewOpenONUAC(ctx, cc, ep, a.kvClient, cfg, cm)
if err = sAcONU.Start(ctx); err != nil {
logger.Fatalw(ctx, "error-starting-OpenOnuAdapterCore", log.Fields{"error": err})
@@ -256,20 +269,7 @@
return sAcONU, nil
}
-func (a *adapter) setupRequestHandler(ctx context.Context, coreInstanceID string, iadapter adapters.IAdapter) error {
- logger.Info(ctx, "setting-request-handler")
- requestProxy := com.NewRequestHandlerProxy(coreInstanceID, iadapter, a.coreProxy)
- if err := a.kip.SubscribeWithRequestHandlerInterface(ctx, kafka.Topic{Name: a.config.Topic}, requestProxy); err != nil {
- logger.Errorw(ctx, "request-handler-setup-failed", log.Fields{"error": err})
- return err
-
- }
- probe.UpdateStatusFromContext(ctx, "core-request-handler", probe.ServiceStatusRunning)
- logger.Info(ctx, "request-handler-setup-done")
- return nil
-}
-
-func (a *adapter) registerWithCore(ctx context.Context, retries int) error {
+func (a *adapter) registerWithCore(ctx context.Context, serviceName string, retries int) error {
adapterID := fmt.Sprintf("brcm_openomci_onu_%d", a.config.CurrentReplica)
vendorIdsList := strings.Split(a.config.OnuVendorIds, ",")
logger.Infow(ctx, "registering-with-core", log.Fields{
@@ -282,43 +282,78 @@
Id: adapterID, // Unique name for the device type ->exact type required for OLT comm????
Vendor: "VOLTHA OpenONUGo",
Version: version.VersionInfo.Version,
- Endpoint: a.config.Topic,
+ Endpoint: a.config.AdapterEndpoint,
Type: "brcm_openomci_onu",
CurrentReplica: int32(a.config.CurrentReplica),
TotalReplicas: int32(a.config.TotalReplicas),
}
types := []*voltha.DeviceType{{Id: "brcm_openomci_onu",
VendorIds: vendorIdsList,
- Adapter: "brcm_openomci_onu", // Name of the adapter that handles device type
+ AdapterType: "brcm_openomci_onu", // Type of adapter that handles this device type
+ Adapter: "brcm_openomci_onu", // Deprecated attribute
AcceptsBulkFlowUpdate: false, // Currently openolt adapter does not support bulk flow handling
AcceptsAddRemoveFlowUpdates: true}}
deviceTypes := &voltha.DeviceTypes{Items: types}
count := 0
for {
- if err := a.coreProxy.RegisterAdapter(context.TODO(), adapterDescription, deviceTypes); err != nil {
- logger.Warnw(ctx, "registering-with-core-failed", log.Fields{"error": err})
+ gClient, err := a.coreClient.GetCoreServiceClient()
+ if gClient != nil {
+ if gClient != nil {
+ if _, err = gClient.RegisterAdapter(log.WithSpanFromContext(context.TODO(), ctx), &ic.AdapterRegistration{
+ Adapter: adapterDescription,
+ DTypes: deviceTypes}); err == nil {
+ break
+ }
+ }
+ logger.Warnw(ctx, "registering-with-core-failed", log.Fields{"endpoint": a.config.CoreEndpoint, "error": err, "count": count, "gclient": gClient})
if retries == count {
return err
}
count++
- // Take a nap before retrying
+ // Take a power nap before retrying
time.Sleep(2 * time.Second)
- } else {
- break
+
}
}
- probe.UpdateStatusFromContext(ctx, "register-with-core", probe.ServiceStatusRunning)
+ probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
logger.Info(ctx, "registered-with-core")
return nil
}
+// startGRPCService creates the grpc service handlers, registers it to the grpc server and starts the server
+func (a *adapter) startGRPCService(ctx context.Context, server *vgrpc.GrpcServer, serviceName string) {
+ logger.Infow(ctx, "service-created", log.Fields{"service": serviceName})
+
+ probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
+ logger.Infow(ctx, "service-started", log.Fields{"service": serviceName})
+
+ server.Start(ctx)
+ probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusStopped)
+}
+
+func (a *adapter) addAdapterService(ctx context.Context, server *vgrpc.GrpcServer, handler adapter_services.AdapterServiceServer) {
+ logger.Info(ctx, "adding-adapter-service")
+
+ server.AddService(func(gs *grpc.Server) {
+ adapter_services.RegisterAdapterServiceServer(gs, handler)
+ })
+}
+
+func (a *adapter) addOnuInterAdapterService(ctx context.Context, server *vgrpc.GrpcServer, handler adapter_services.OnuInterAdapterServiceServer) {
+ logger.Info(ctx, "adding-onu-inter-adapter-service")
+
+ server.AddService(func(gs *grpc.Server) {
+ adapter_services.RegisterOnuInterAdapterServiceServer(gs, handler)
+ })
+}
+
/**
This function checks the liveliness and readiness of the kakfa and kv-client services
and update the status in the probe.
*/
func (a *adapter) checkServicesReadiness(ctx context.Context) {
// checks the kafka readiness
- go a.checkKafkaReadiness(ctx)
+ go kafka.MonitorKafkaReadiness(ctx, a.kafkaClient, a.config.LiveProbeInterval, a.config.NotLiveProbeInterval, clusterMessagingService)
// checks the kv-store readiness
go a.checkKvStoreReadiness(ctx)
@@ -333,19 +368,19 @@
timeout := a.config.LiveProbeInterval / 2
kvStoreChannel := make(chan bool, 1)
- // Default false to check the liveliness.
- kvStoreChannel <- false
+ // Default true - we are here only after we already had a KV store connection
+ kvStoreChannel <- true
for {
timeoutTimer := time.NewTimer(timeout)
select {
case liveliness := <-kvStoreChannel:
if !liveliness {
// kv-store not reachable or down, updating the status to not ready state
- probe.UpdateStatusFromContext(ctx, "kv-store", probe.ServiceStatusNotReady)
+ probe.UpdateStatusFromContext(ctx, kvService, probe.ServiceStatusNotReady)
timeout = a.config.NotLiveProbeInterval
} else {
// kv-store is reachable , updating the status to running state
- probe.UpdateStatusFromContext(ctx, "kv-store", probe.ServiceStatusRunning)
+ probe.UpdateStatusFromContext(ctx, kvService, probe.ServiceStatusRunning)
timeout = a.config.LiveProbeInterval / 2
}
// Check if the timer has expired or not
@@ -364,48 +399,28 @@
}
}
-/**
-This function checks the liveliness and readiness of the kafka service
-and update the status in the probe.
-*/
-func (a *adapter) checkKafkaReadiness(ctx context.Context) {
- livelinessChannel := a.kafkaClient.EnableLivenessChannel(ctx, true)
- healthinessChannel := a.kafkaClient.EnableHealthinessChannel(ctx, true)
- timeout := a.config.LiveProbeInterval
+// WaitUntilKvStoreConnectionIsUp waits until the KV client can establish a connection to the KV server or until the
+// context times out.
+func WaitUntilKvStoreConnectionIsUp(ctx context.Context, kvClient kvstore.Client, connectionRetryInterval time.Duration, serviceName string) error {
+ if kvClient == nil {
+ return errors.New("kvclient-is-nil")
+ }
for {
- timeoutTimer := time.NewTimer(timeout)
-
- select {
- case healthiness := <-healthinessChannel:
- if !healthiness {
- // logger.Fatal will call os.Exit(1) to terminate
- logger.Fatal(ctx, "Kafka service has become unhealthy")
- }
- case liveliness := <-livelinessChannel:
- if !liveliness {
- // kafka not reachable or down, updating the status to not ready state
- probe.UpdateStatusFromContext(ctx, "message-bus", probe.ServiceStatusNotReady)
- timeout = a.config.NotLiveProbeInterval
- } else {
- // kafka is reachable , updating the status to running state
- probe.UpdateStatusFromContext(ctx, "message-bus", probe.ServiceStatusRunning)
- timeout = a.config.LiveProbeInterval
- }
- // Check if the timer has expired or not
- if !timeoutTimer.Stop() {
- <-timeoutTimer.C
- }
- case <-timeoutTimer.C:
- logger.Info(ctx, "kafka-proxy-liveness-recheck")
- // send the liveness probe in a goroutine; we don't want to deadlock ourselves as
- // the liveness probe may wait (and block) writing to our channel.
- err := a.kafkaClient.SendLiveness(ctx)
- if err != nil {
- // Catch possible error case if sending liveness after Sarama has been stopped.
- logger.Warnw(ctx, "error-kafka-send-liveness", log.Fields{"error": err})
+ if !kvClient.IsConnectionUp(ctx) {
+ probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusNotReady)
+ logger.Warnw(ctx, "kvconnection-down", log.Fields{"service-name": serviceName, "connect-retry-interval": connectionRetryInterval})
+ select {
+ case <-time.After(connectionRetryInterval):
+ continue
+ case <-ctx.Done():
+ return ctx.Err()
}
}
+ probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
+ logger.Info(ctx, "kv-connection-up")
+ break
}
+ return nil
}
// Adapter Utility methods ##### end #########
@@ -478,9 +493,10 @@
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- cf := config.NewAdapterFlags()
+ cf := &config.AdapterFlags{}
+ cf.ParseCommandArguments(os.Args[1:])
+
defaultAppName := cf.InstanceID + "_" + getVerifiedCodeVersion(ctx)
- cf.ParseCommandArguments()
// Setup logging
diff --git a/docker/Dockerfile.openonu b/docker/Dockerfile.openonu
index 34664cc..43c959c 100755
--- a/docker/Dockerfile.openonu
+++ b/docker/Dockerfile.openonu
@@ -40,13 +40,13 @@
RUN \
CGO_ENABLED=$CGO_PARAMETER go build $EXTRA_GO_BUILD_TAGS -mod=vendor -o /app/openonu \
-ldflags \
-"-X github.com/opencord/voltha-lib-go/v5/pkg/version.version=$org_label_schema_version \
--X github.com/opencord/voltha-lib-go/v5/pkg/version.vcsRef=$org_label_schema_vcs_ref \
--X github.com/opencord/voltha-lib-go/v5/pkg/version.vcsDirty=$org_opencord_vcs_dirty \
--X github.com/opencord/voltha-lib-go/v5/pkg/version.goVersion=$(go version 2>&1 | sed -E 's/.*go([0-9]+\.[0-9]+\.[0-9]+).*/\1/g') \
--X github.com/opencord/voltha-lib-go/v5/pkg/version.os=$(go env GOHOSTOS) \
--X github.com/opencord/voltha-lib-go/v5/pkg/version.arch=$(go env GOHOSTARCH) \
--X github.com/opencord/voltha-lib-go/v5/pkg/version.buildTime=$org_label_schema_build_date" \
+"-X github.com/opencord/voltha-lib-go/v7/pkg/version.version=$org_label_schema_version \
+-X github.com/opencord/voltha-lib-go/v7/pkg/version.vcsRef=$org_label_schema_vcs_ref \
+-X github.com/opencord/voltha-lib-go/v7/pkg/version.vcsDirty=$org_opencord_vcs_dirty \
+-X github.com/opencord/voltha-lib-go/v7/pkg/version.goVersion=$(go version 2>&1 | sed -E 's/.*go([0-9]+\.[0-9]+\.[0-9]+).*/\1/g') \
+-X github.com/opencord/voltha-lib-go/v7/pkg/version.os=$(go env GOHOSTOS) \
+-X github.com/opencord/voltha-lib-go/v7/pkg/version.arch=$(go env GOHOSTARCH) \
+-X github.com/opencord/voltha-lib-go/v7/pkg/version.buildTime=$org_label_schema_build_date" \
./cmd/openonu-adapter/
WORKDIR /app
diff --git a/go.mod b/go.mod
index 5a001ff..4e92e23 100644
--- a/go.mod
+++ b/go.mod
@@ -2,16 +2,22 @@
go 1.16
+replace (
+ github.com/coreos/bbolt v1.3.4 => go.etcd.io/bbolt v1.3.4
+ go.etcd.io/bbolt v1.3.4 => github.com/coreos/bbolt v1.3.4
+ google.golang.org/grpc => google.golang.org/grpc v1.25.1
+)
+
require (
github.com/boguslaw-wojcik/crc32a v1.0.0
github.com/cevaris/ordered_map v0.0.0-20190319150403-3adeae072e73
github.com/gogo/protobuf v1.3.2
- github.com/golang/protobuf v1.3.2
+ github.com/golang/protobuf v1.5.2
github.com/google/gopacket v1.1.17
github.com/looplab/fsm v0.2.0
github.com/opencord/omci-lib-go v1.3.3
- github.com/opencord/voltha-lib-go/v5 v5.0.5
- github.com/opencord/voltha-protos/v4 v4.2.0
- github.com/stretchr/testify v1.6.1
- google.golang.org/grpc v1.25.1 // indirect
+ github.com/opencord/voltha-lib-go/v7 v7.0.0
+ github.com/opencord/voltha-protos/v5 v5.0.0
+ github.com/stretchr/testify v1.7.0
+ google.golang.org/grpc v1.41.0
)
diff --git a/go.sum b/go.sum
index 4b6ca32..dadc804 100644
--- a/go.sum
+++ b/go.sum
@@ -1,48 +1,47 @@
-cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
-github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
-github.com/DataDog/zstd v1.3.6-0.20190409195224-796139022798/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo=
-github.com/DataDog/zstd v1.4.1 h1:3oxKN3wbHibqx897utPC2LTQU4J+IHWWJO+glkAkpFM=
-github.com/DataDog/zstd v1.4.1/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo=
-github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE=
-github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU=
-github.com/Shopify/sarama v1.23.1 h1:XxJBCZEoWJtoWjf/xRbmGUpAmTZGnuuF0ON0EvxxBrs=
-github.com/Shopify/sarama v1.23.1/go.mod h1:XLH1GYJnLVE0XCr6KdJGVJRTwY30moWNJ4sERjXX6fs=
+cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
+dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
+github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo=
+github.com/HdrHistogram/hdrhistogram-go v1.1.0 h1:6dpdDPTRoo78HxAJ6T1HfMiKSnqhgRRqzCuPshRkQ7I=
+github.com/HdrHistogram/hdrhistogram-go v1.1.0/go.mod h1:yDgFjdqOqDEKOvasDdhWNXYg9BVp4O+o5f6V/ehm6Oo=
+github.com/Shopify/sarama v1.29.1 h1:wBAacXbYVLmWieEA/0X/JagDdCZ8NVFOfS6l6+2u5S0=
+github.com/Shopify/sarama v1.29.1/go.mod h1:mdtqvCSg8JOxk8PmpTNGyo6wzd4BMm4QXSfDnTXmgkE=
github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc=
github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI=
github.com/aead/cmac v0.0.0-20160719120800-7af84192f0b1 h1:+JkXLHME8vLJafGhOH4aoV2Iu8bR55nU6iKMVfYVLjY=
github.com/aead/cmac v0.0.0-20160719120800-7af84192f0b1/go.mod h1:nuudZmJhzWtx2212z+pkuy7B6nkBqa+xwNXZHL1j8cg=
+github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw=
github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
+github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
+github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
+github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho=
+github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY=
+github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8=
+github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA=
github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
-github.com/beorn7/perks v1.0.0 h1:HWo1m869IqiPhD389kmkxeTalrjNbbJTC8LXupb+sl0=
github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
-github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs=
+github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
+github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
github.com/boguslaw-wojcik/crc32a v1.0.0 h1:rZUcnG4WkADJBW8tmb5ZWFN30w/mHV8+/hzM66X2ptQ=
github.com/boguslaw-wojcik/crc32a v1.0.0/go.mod h1:BnG1x2VM7pNqIjOAHQKMu4NZ0Rn7cCxr+BmuXIbNOhM=
github.com/boljen/go-bitmap v0.0.0-20151001105940-23cd2fb0ce7d/go.mod h1:f1iKL6ZhUWvbk7PdWVmOaak10o86cqMUYEmn1CZNGEI=
github.com/bsm/sarama-cluster v2.1.15+incompatible h1:RkV6WiNRnqEEbp81druK8zYhmnIgdOjqSVi0+9Cnl2A=
github.com/bsm/sarama-cluster v2.1.15+incompatible/go.mod h1:r7ao+4tTNXvWm+VRpRJchr2kQhqxgmAp2iEX5W96gMM=
-github.com/buraksezer/consistent v0.0.0-20191006190839-693edf70fd72 h1:fUmDBbSvv1uOzo/t8WaxZMVb7BxJ8JECo5lGoR9c5bA=
-github.com/buraksezer/consistent v0.0.0-20191006190839-693edf70fd72/go.mod h1:OEE5igu/CDjGegM1Jn6ZMo7R6LlV/JChAkjfQQIRLpg=
github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
-github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko=
-github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc=
+github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY=
+github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
github.com/cevaris/ordered_map v0.0.0-20190319150403-3adeae072e73 h1:q1g9lSyo/nOIC3W5E3FK3Unrz8b9LdLXCyuC+ZcpPC0=
github.com/cevaris/ordered_map v0.0.0-20190319150403-3adeae072e73/go.mod h1:507vXsotcZop7NZfBWdhPmVeOse4ko2R7AagJYrpoEg=
-github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
-github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr7kxeODyLWsRMC+OD03aFUH+mW6r2d+MWa5Y=
-github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8=
-github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd h1:qMd81Ts1T2OTKmB4acZcyKaMtRnY5Y44NuXGX2GFJ1w=
-github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI=
-github.com/coreos/go-semver v0.2.0 h1:3Jm3tLmsgAYcjC+4Up7hJrFBPr+n7rAqYeSw/SZazuY=
-github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk=
-github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
-github.com/coreos/go-systemd v0.0.0-20190620071333-e64a0ec8b42a h1:W8b4lQ4tFF21aspRGoBuCNV6V2fFJBF+pm1J6OY8Lys=
-github.com/coreos/go-systemd v0.0.0-20190620071333-e64a0ec8b42a/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
-github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
-github.com/coreos/pkg v0.0.0-20180108230652-97fdf19511ea h1:n2Ltr3SrfQlf/9nOna1DoGKxLx3qTSI8Ttl6Xrqp6mw=
-github.com/coreos/pkg v0.0.0-20180108230652-97fdf19511ea/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
-github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY=
+github.com/coreos/bbolt v1.3.4/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ=
+github.com/coreos/etcd v3.3.25+incompatible h1:0GQEw6h3YnuOVdtwygkIfJ+Omx0tZ8/QkVyXI4LkbeY=
+github.com/coreos/etcd v3.3.25+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE=
+github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM=
+github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk=
+github.com/coreos/go-systemd v0.0.0-20191104093116-d3cd4ed1dbcf h1:iW4rZ826su+pqaw19uhpSCzhj44qo35pNgKFGqzDKkU=
+github.com/coreos/go-systemd v0.0.0-20191104093116-d3cd4ed1dbcf/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
+github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg=
+github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
+github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E=
github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
@@ -50,9 +49,8 @@
github.com/deckarep/golang-set v1.7.1/go.mod h1:93vsz/8Wt4joVM7c2AVqh+YRMiUSc14yDtF28KmMOgQ=
github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM=
github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
-github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4 h1:qk/FSDDxo05wdJH28W+p5yivv7LuLYLRXPPD8KQCtZs=
-github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk=
-github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs=
+github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo=
+github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk=
github.com/eapache/go-resiliency v1.2.0 h1:v7g92e/KSN71Rq7vSThKaWIq68fL4YHvWyiUKorFR1Q=
github.com/eapache/go-resiliency v1.2.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs=
github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw=
@@ -61,81 +59,116 @@
github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I=
github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
-github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4=
-github.com/frankban/quicktest v1.5.0 h1:Tb4jWdSpdjKzTUicPnY61PZxKbDoGa7ABbrReT3gQVY=
-github.com/frankban/quicktest v1.5.0/go.mod h1:jaStnuzAqU1AJdCO0l53JDCJrVDKcS03DbaAcR7Ks/o=
+github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k=
+github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw=
+github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g=
+github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY=
+github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k=
github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
+github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4=
+github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ=
github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
+github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU=
github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
+github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
+github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY=
github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE=
+github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
+github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A=
github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
+github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE=
github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
-github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4=
-github.com/gogo/protobuf v1.3.0/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o=
github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q=
github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q=
+github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k=
github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58=
github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q=
-github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903 h1:LbsanbbD6LieFkXbj9YNNBupiGHJgFeLpO0j0Fza1h8=
-github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
+github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE=
+github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
+github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs=
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/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs=
github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
-github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4=
-github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
-github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo=
-github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
+github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw=
+github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8=
+github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA=
+github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs=
+github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
+github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
+github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
+github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
+github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk=
+github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw=
+github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY=
+github.com/golang/snappy v0.0.3 h1:fHPg5GQYlCeLIPB9BZqMVR5nR9A+IM5zcgeTdjMYmLA=
+github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
+github.com/google/btree v1.0.1 h1:gK4Kx5IaGY9CD5sPJ36FHiBJ6ZXl0kilRiiCj+jdYp4=
+github.com/google/btree v1.0.1/go.mod h1:xXMiIv4Fb/0kKde4SpL7qlzvu5cMJDRkFDxJfI9uaxA=
github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M=
-github.com/google/go-cmp v0.3.1 h1:Xye71clBPdm5HgqGwUkwhbynsUJZhDbS20FvLhQ2izg=
+github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
+github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
+github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
+github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU=
+github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
github.com/google/gopacket v1.1.17 h1:rMrlX2ZY2UbvT+sdz3+6J+pp2z+msCq9MxTU6ymxbBY=
github.com/google/gopacket v1.1.17/go.mod h1:UdDNZ1OO62aGYVnPhxT1U6aI7ukYtA/kB8vaU0diBUM=
-github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY=
-github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c h1:Lh2aW+HnU2Nbe1gqD9SOJLJxW1jBMmQOktN2acDyJk8=
-github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ=
-github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 h1:z53tR0945TRRQO/fLEVPI6SMv7ZflF0TEaTAoU7tOzg=
-github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs=
+github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I=
+github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
+github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4=
+github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM=
+github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc=
+github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
+github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw=
+github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y=
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho=
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk=
-github.com/grpc-ecosystem/grpc-gateway v1.9.5 h1:UImYN5qQ8tuGpGE16ZmjvcTtTw24zw1QAp/SlnNrZhI=
-github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
-github.com/hashicorp/go-uuid v1.0.1 h1:fv1ep09latC32wFoVwnqcnKJGnMSdBanPczbHAYm1BE=
-github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
-github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI=
+github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo=
+github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw=
+github.com/hashicorp/go-uuid v1.0.2 h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2IGE=
+github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
-github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8=
-github.com/jcmturner/gofork v0.0.0-20190328161633-dc7c13fece03/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o=
+github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8=
+github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs=
+github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo=
+github.com/jcmturner/dnsutils/v2 v2.0.0/go.mod h1:b0TnjGOvI/n42bZa+hmXL+kFJZsFT7G4t3HTlQ184QM=
github.com/jcmturner/gofork v1.0.0 h1:J7uCkflzTEhUZ64xqKnkDxq3kzc96ajM1Gli5ktUem8=
github.com/jcmturner/gofork v1.0.0/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o=
-github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo=
-github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo=
+github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o=
+github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg=
+github.com/jcmturner/gokrb5/v8 v8.4.2 h1:6ZIM6b/JJN0X8UM43ZOM6Z4SJzla+a/u7scXFJzodkA=
+github.com/jcmturner/gokrb5/v8 v8.4.2/go.mod h1:sb+Xq/fTY5yktf/VxLsE3wlfPqQjp0aWNYyvBVK62bc=
+github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY=
+github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc=
+github.com/jonboulle/clockwork v0.2.2 h1:UOGuzwb1PwsrDAObMuhUnj0p5ULPj8V/xJ7Kx9qUBdQ=
+github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8=
+github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4=
github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
-github.com/json-iterator/go v1.1.7 h1:KfgG9LzI+pYjr4xvmz/5H4FXjokeP+rlHLhv3iH62Fo=
-github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
+github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
+github.com/json-iterator/go v1.1.11 h1:uVUAXhF2To8cbw/3xN3pxj6kk7TYKs98NIrTqPlMWAQ=
+github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w=
-github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q=
-github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00=
+github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM=
+github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes=
github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8=
github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
-github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk=
+github.com/klauspost/compress v1.12.2 h1:2KCfW3I9M7nSc5wOqXAlW2v2U6v+w6cbjvbfp+OykW8=
+github.com/klauspost/compress v1.12.2/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg=
github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
+github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8=
+github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc=
github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI=
github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
-github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE=
github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
+github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=
+github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=
github.com/looplab/fsm v0.2.0 h1:M8hf5EF4AYLcT1FNKVUX8nu7D0xfp291iGeuigSxfrw=
github.com/looplab/fsm v0.2.0/go.mod h1:p+IElwgCnAByqr2DWMuNbPjgMwqcHvTRZZn3dvKEke0=
-github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU=
-github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4=
-github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU=
github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU=
github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
@@ -145,202 +178,262 @@
github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI=
github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
-github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo=
-github.com/onsi/ginkgo v1.6.0 h1:Ix8l273rp3QzYgXSR+c8d1fTG7UPgYkOSELPhiY/YGw=
+github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
+github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno=
+github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A=
+github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE=
+github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU=
github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
-github.com/onsi/gomega v1.4.2 h1:3mYCb7aPxS/RU7TI1y4rkEn1oKmPRjNJLNEXgw7MH2I=
-github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
+github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk=
+github.com/onsi/ginkgo v1.16.4 h1:29JGrr5oVBm5ulCWet69zQkzWipVXIol6ygQUe/EzNc=
+github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0=
+github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY=
+github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo=
+github.com/onsi/gomega v1.14.0 h1:ep6kpPVwmr/nTbklSx2nrLNSIO62DoYAhnPNIMhK8gI=
+github.com/onsi/gomega v1.14.0/go.mod h1:cIuvLEne0aoVhAgh/O6ac0Op8WWw9H6eYCriF+tEHG0=
github.com/opencord/omci-lib-go v1.3.3 h1:BzywkXVHSphhpl9hHHOJxc9IQ7MrzIB3aY3LG1wbwuk=
github.com/opencord/omci-lib-go v1.3.3/go.mod h1:moNk4j00XaM3olsu4a8lRAqGmcZJoyIbxtSr+VERLq4=
-github.com/opencord/voltha-lib-go/v5 v5.0.5 h1:S3m984EaHNGJY68Hrv9LOf0rwd92UiVLFBBbTaL+fNg=
-github.com/opencord/voltha-lib-go/v5 v5.0.5/go.mod h1:i1fwPMicFccG38L200+IQAlfHSbszWg//jF1pDQxTPQ=
-github.com/opencord/voltha-protos/v4 v4.2.0 h1:QJZqHPRKa1E1xh40F3UA4xSjBI+6EmW7OfIcJqPNc4A=
-github.com/opencord/voltha-protos/v4 v4.2.0/go.mod h1:wNzWqmTwe7+DbYbpmOX6eMlglREtMkNxIDv3lyI2bco=
-github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU=
+github.com/opencord/voltha-lib-go/v7 v7.0.0 h1:xDMFbXdku7GOsJeMlsxw8WHAZINBZhy+8m9Hyqt+zdk=
+github.com/opencord/voltha-lib-go/v7 v7.0.0/go.mod h1:iZueJRS4XJ3rpm3iy0Zdnhz1lG5bWx2pZoPormwgUKk=
+github.com/opencord/voltha-protos/v5 v5.0.0 h1:US2k7qYPMnOueOCrprq9LjuMT3wK9uyxPwAVwjMmKhc=
+github.com/opencord/voltha-protos/v5 v5.0.0/go.mod h1:uVKXQB499Ir6G+rc47dSThNja1S4Vy3h9JLSDuJGmzI=
github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
+github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs=
+github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc=
github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 h1:JhzVVoYvbOACxoUmOs6V/G4D5nPVUW73rKvXxP4XUJc=
github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE=
-github.com/pierrec/lz4 v0.0.0-20190327172049-315a67e90e41/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc=
-github.com/pierrec/lz4 v2.3.0+incompatible h1:CZzRn4Ut9GbUkHlQ7jqBXeZQV41ZSKWFc302ZU6lUTk=
-github.com/pierrec/lz4 v2.3.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY=
+github.com/pierrec/lz4 v2.6.0+incompatible h1:Ix9yFKn1nSPBLFl/yZknTp8TU5G4Ps0JDmguYK6iH1A=
+github.com/pierrec/lz4 v2.6.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY=
github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
-github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I=
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
-github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA=
+github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4=
+github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw=
-github.com/prometheus/client_golang v1.0.0 h1:vrDKnkGzuGvhNAL56c7DBz29ZL+KxnoR0x7enabFceM=
github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo=
+github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M=
+github.com/prometheus/client_golang v1.11.0 h1:HNkLOAEQMIDv/K+04rukrLx6ch7msSRwf3/SASFAGtQ=
+github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0=
github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
-github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4 h1:gQz4mCbXsO+nc9n1hCxHcGA3Zx3Eo+UHZoInFGUIXNM=
github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
-github.com/prometheus/common v0.4.1 h1:K0MGApIoQvMw27RTdJkPbr3JZ7DNbtxQNyi5STVM6Kw=
+github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M=
+github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
+github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo=
+github.com/prometheus/common v0.26.0 h1:iMAkS2TDoNWnKM+Kopnx/8tnEStIfpYA0ur0xQzzhMQ=
+github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc=
github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
-github.com/prometheus/procfs v0.0.2 h1:6LJUbpNm42llc4HRCuvApCSWB/WfhuNo9K98Q9sNGfs=
github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
-github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
-github.com/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563 h1:dY6ETXrvDG7Sa4vE8ZQG4yqWg6UnOcbqTAahkV813vQ=
-github.com/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
-github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg=
-github.com/sirupsen/logrus v1.2.0 h1:juTguoYk5qI21pwyTXY3B3Y5cOTH3ZUyZCg1v/mihuo=
+github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU=
+github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4=
+github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA=
+github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM=
+github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
+github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ=
github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo=
-github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E=
-github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM=
-github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ=
-github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA=
-github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ=
-github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
-github.com/spf13/pflag v1.0.3 h1:zPAT6CGy6wXeQ7NtTnaTerfKOsV6V6F8agHXFiazDkg=
-github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
+github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE=
+github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I=
+github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88=
+github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js=
+github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE1GqG0=
github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
+github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A=
github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA=
-github.com/stretchr/testify v1.6.1 h1:hDPOHmpOpP40lSULcqw7IrRb/u7w6RpDC9399XyoNd0=
github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
-github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8 h1:ndzgwNDnKIqyCvHTXaCqh9KlOWKvBry6nuXMJmonVsE=
-github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
-github.com/uber/jaeger-client-go v2.23.1+incompatible h1:uArBYHQR0HqLFFAypI7RsWTzPSj/bDpmZZuQjMLSg1A=
-github.com/uber/jaeger-client-go v2.23.1+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk=
-github.com/uber/jaeger-lib v2.2.0+incompatible h1:MxZXOiR2JuoANZ3J6DE/U0kSFv/eJ/GfSYVCjK7dyaw=
-github.com/uber/jaeger-lib v2.2.0+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U=
-github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA=
-github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I=
-github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y=
+github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY=
+github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
+github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 h1:uruHq4dN7GR16kFc5fp3d1RIYzJW5onx8Ybykw2YQFA=
+github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
+github.com/uber/jaeger-client-go v2.29.1+incompatible h1:R9ec3zO3sGpzs0abd43Y+fBZRJ9uiH6lXyR/+u6brW4=
+github.com/uber/jaeger-client-go v2.29.1+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk=
+github.com/uber/jaeger-lib v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVKhn2Um6rjCsSsg=
+github.com/uber/jaeger-lib v2.4.1+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U=
+github.com/xdg/scram v1.0.3/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I=
+github.com/xdg/stringprep v1.0.3/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y=
github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8=
github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU=
github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
-go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk=
-go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
-go.etcd.io/etcd v0.0.0-20190930204107-236ac2a90522 h1:GQU7sDaYW5CN6WpkPCWZQrZ/dEO6NDc2cHfd9bbsqso=
-go.etcd.io/etcd v0.0.0-20190930204107-236ac2a90522/go.mod h1:uQccEQvXbbNc3vI3weFUy1S42v0dtl0CtCePpj8fRSk=
-go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
-go.uber.org/atomic v1.4.0 h1:cxzIVoETapQEqDhQu3QfnvXAV4AlzcvUCxkVUFw3+EU=
+github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k=
+go.etcd.io/bbolt v1.3.4 h1:hi1bXHMVrlQh6WwxAy+qZCV/SYIlqo+Ushwdpa4tAKg=
+go.etcd.io/bbolt v1.3.4/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ=
+go.etcd.io/etcd v3.3.25+incompatible h1:V1RzkZJj9LqsJRy+TUBgpWSbZXITLB819lstuTFoZOY=
+go.etcd.io/etcd v3.3.25+incompatible/go.mod h1:yaeTdrJi5lOmYerz05bd8+V7KubZs8YSFZfzsF9A6aI=
go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
+go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw=
+go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc=
+go.uber.org/goleak v1.1.10 h1:z+mqJhf6ss6BSfSM671tgKyZBFPTTJM+HLxnhPC3wu0=
+go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A=
go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
-go.uber.org/multierr v1.2.0 h1:6I+W7f5VwC5SV9dNrZ3qXrDB9mD0dyGOi/ZJmYw03T4=
-go.uber.org/multierr v1.2.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
-go.uber.org/zap v1.10.0 h1:ORx85nbTijNz8ljznvCMR1ZBIPKFn3jQrag10X2AsuM=
+go.uber.org/multierr v1.6.0 h1:y6IPFStTAIT5Ytl7/XYmHvzXQ7S3g/IeZW9hyZ5thw4=
+go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU=
go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
+go.uber.org/zap v1.18.1 h1:CSUJ2mjFszzEWt4CdKISEuChVIXGBn3lAPwkRGyVrc4=
+go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI=
golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
-golang.org/x/crypto v0.0.0-20190404164418-38d8ce5564a5/go.mod h1:WFFai1msRO1wXaEeE5yQxYXgSfI8pQAWXbQop6sCtWE=
-golang.org/x/crypto v0.0.0-20191001170739-f9e2070545dc/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
+golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
-golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 h1:psW17arqaxU48Z5kZ0CQnkZWQJsqcURM6tKiBApRjXI=
golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
+golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
+golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e h1:gsTQYXdTw2Gq7RBsWvlQ91b+aEQ6bXFUngBGuR8sPpI=
+golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
+golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
+golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
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/exp v0.0.0-20190125153040-c74c464bbbf2/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
+golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
+golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY=
+golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs=
+golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js=
+golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=
golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU=
golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
+golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs=
+golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
+golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o=
+golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY=
golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
+golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
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-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20181220203305-927f97764cc3/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-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
+golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20190930134127-c5a3c61f89f3/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20201021035429-f5854403a974 h1:IX6qOQeG5uLjB/hjjwjedwfjND0hgjPMMyO1RoIXQNI=
+golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A=
+golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
+golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
+golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
+golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
+golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM=
+golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk=
+golang.org/x/net v0.0.0-20210614182718-04defd469f4e h1:XpT3nA5TvE525Ne3hInMh6+GETgn27Zfm9dxsThnX2Q=
+golang.org/x/net v0.0.0-20210614182718-04defd469f4e/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y=
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-20200107190931-bf48bf16ab8d/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-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-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/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-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/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-20190403152447-81d4e9dc473e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190405154228-4b34438f7a67/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f h1:+Nyd8tzPX9R7BWHguqsrbFdRx3WQ/1ib8I44HXV5yTA=
+golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1 h1:SrN+KX8Art/Sf4HNj6Zcz06G7VEz+7w9tdXTPOZ7+l4=
+golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk=
-golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k=
golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
-golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2 h1:+DCIGbF/swA92ohVg0//6X2IVY3KZs6p9mix0ziNYJM=
-golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
-golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
-golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
+golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M=
+golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
+golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac h1:7zkz7BUtwNFFqcowJ+RIgu2MaV/MapERkDIy+mwPyjs=
+golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
+golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/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-20181030221726-6c7e314b6563/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-20190206041539-40960b6deb8e/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-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
+golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE=
+golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
+golang.org/x/tools v0.1.1 h1:wGiQel/hW0NnEkJUk8lbzkX2gFJU6PFxf1v5OlCfuOs=
+golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE=
golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
-google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
+gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo=
+gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0=
+gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw=
+gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc=
google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
-google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
-google.golang.org/genproto v0.0.0-20190927181202-20e1ac93f88c h1:hrpEMCZ2O7DR5gC1n2AJGVhrwiEjOi35+jxtIuZpTMo=
-google.golang.org/genproto v0.0.0-20190927181202-20e1ac93f88c/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8=
-google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs=
-google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
-google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
-google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
-google.golang.org/grpc v1.24.0/go.mod h1:XDChyiUovWa60DnaeDeZmSW86xtLtjtZbwvSiRnRtcA=
+google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
+google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
+google.golang.org/genproto v0.0.0-20210924002016-3dee208752a0 h1:5Tbluzus3QxoAJx4IefGt1W0HQZW4nuMrVk684jI74Q=
+google.golang.org/genproto v0.0.0-20210924002016-3dee208752a0/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc=
google.golang.org/grpc v1.25.1 h1:wdKvqQk7IttEw92GoRyKG2IDrUIpgpj6H6m81yfeMW0=
google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY=
+google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8=
+google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0=
+google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM=
+google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE=
+google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo=
+google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
+google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw=
+google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc=
+google.golang.org/protobuf v1.27.1 h1:SnqbnDw1V7RiZcXPx5MEeqPv2s79L9i7BJUlG/+RurQ=
+google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc=
gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
-gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo=
gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
-gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw=
-gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4=
+gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=
+gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q=
gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
-gopkg.in/jcmturner/aescts.v1 v1.0.1 h1:cVVZBK2b1zY26haWB4vbBiZrfFQnfbTVrE3xZq6hrEw=
-gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo=
-gopkg.in/jcmturner/dnsutils.v1 v1.0.1 h1:cIuC1OLRGZrld+16ZJvvZxVJeKPsvd5eUIvxfoN5hSM=
-gopkg.in/jcmturner/dnsutils.v1 v1.0.1/go.mod h1:m3v+5svpVOhtFAP/wSz+yzh4Mc0Fg7eRhxkJMWSIz9Q=
-gopkg.in/jcmturner/goidentity.v3 v3.0.0 h1:1duIyWiTaYvVx3YX2CYtpJbUFd7/UuPYCfgXtQ3VTbI=
-gopkg.in/jcmturner/goidentity.v3 v3.0.0/go.mod h1:oG2kH0IvSYNIu80dVAyu/yoefjq1mNfM5bm88whjWx4=
-gopkg.in/jcmturner/gokrb5.v7 v7.2.3/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuvyavf11/WM=
-gopkg.in/jcmturner/gokrb5.v7 v7.3.0 h1:0709Jtq/6QXEuWRfAm260XqlpcwL1vxtO1tUE2qK8Z4=
-gopkg.in/jcmturner/gokrb5.v7 v7.3.0/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuvyavf11/WM=
-gopkg.in/jcmturner/rpc.v1 v1.1.0 h1:QHIUxTX1ISuAv9dD2wJ9HWQVuWDX/Zc0PfeC2tjc4rU=
-gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8=
-gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo=
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ=
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw=
-gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74=
gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
-gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10=
+gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
-gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo=
+gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY=
+gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ=
gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
-honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
+gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo=
+gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
-sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs=
-sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o=
+rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4=
+sigs.k8s.io/yaml v1.2.0 h1:kr/MCeFWJWTwyaHoR9c8EjH9OumOmoF9YGiZd7lFm/Q=
+sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc=
diff --git a/internal/pkg/config/config.go b/internal/pkg/config/config.go
index b61a909..aff1de3 100644
--- a/internal/pkg/config/config.go
+++ b/internal/pkg/config/config.go
@@ -19,73 +19,24 @@
import (
"flag"
- "fmt"
"os"
"time"
)
// Open ONU default constants
const (
- etcdStoreName = "etcd"
- defaultInstanceid = "openonu"
- defaultKafkaadapteraddress = "127.0.0.1:9092"
- defaultKafkaclusteraddress = "127.0.0.1:9092"
- defaultKvstoretype = etcdStoreName
- defaultKvstoretimeout = 5 * time.Second
- defaultKvstoreaddress = "127.0.0.1:2379"
- defaultLoglevel = "WARN"
- defaultBanner = false
- defaultDisplayVersionOnly = false
- defaultAccIncrEvto = false
- defaultTopic = "openonu"
- defaultCoreTopic = "rwcore"
- defaultEventTopic = "voltha.events"
- defaultOnunumber = 1
- defaultProbeHost = ""
- defaultProbePort = 8080
- defaultLiveProbeInterval = 60 * time.Second
- defaultNotLiveProbeInterval = 5 * time.Second // Probe more frequently when not alive
- //defaultHearbeatFailReportInterval is the time in seconds the adapter will keep checking the hardware for heartbeat.
- defaultHearbeatCheckInterval = 30 * time.Second
- // defaultHearbeatFailReportInterval is the time adapter will wait before updating the state to the core.
- defaultHearbeatFailReportInterval = 180 * time.Second
- //defaultKafkaReconnectRetries -1: reconnect endlessly.
- defaultKafkaReconnectRetries = -1
- defaultCurrentReplica = 1
- defaultTotalReplicas = 1
- defaultMaxTimeoutInterAdapterComm = 30 * time.Second
- defaultMaxTimeoutReconciling = 10 * time.Second
- defaultOnuVendorIds = "OPEN,ALCL,BRCM,TWSH,ALPH,ISKT,SFAA,BBSM,SCOM,ARPX,DACM,ERSN,HWTC,CIGG,ADTN,ARCA,AVMG"
-
- // For Tracing
- defaultTraceEnabled = false
- defaultTraceAgentAddress = "127.0.0.1:6831"
- defaultLogCorrelationEnabled = true
-
- defaultMetricsEnabled = false
- defaultMibAuditInterval = 0
- defaultAlarmAuditInterval = 300 * time.Second
-
- defaultOmciTimeout = 3 * time.Second
- defaultDlToAdapterTimeout = 10 * time.Second
- defaultDlToOnuTimeoutPer4MB = 60 * time.Minute //assumed for 4 MB of the image
- //Mask to indicate which possibly active ONU UNI state is really reported to the core
- // compare python code - at the moment restrict active state to the first ONU UNI port
- // check is limited to max 16 uni ports - cmp above UNI limit!!!
- defaultUniPortMask = 0x0001
+ EtcdStoreName = "etcd"
+ OnuVendorIds = "OPEN,ALCL,BRCM,TWSH,ALPH,ISKT,SFAA,BBSM,SCOM,ARPX,DACM,ERSN,HWTC,CIGG,ADTN,ARCA,AVMG"
)
// AdapterFlags represents the set of configurations used by the read-write adaptercore service
type AdapterFlags struct {
// Command line parameters
InstanceID string
- KafkaAdapterAddress string
KafkaClusterAddress string // NOTE this is unused across the adapter
KVStoreType string
KVStoreTimeout time.Duration
KVStoreAddress string
- Topic string
- CoreTopic string
EventTopic string
LogLevel string
OnuNumber int
@@ -114,169 +65,220 @@
DownloadToAdapterTimeout time.Duration
DownloadToOnuTimeout4MB time.Duration
UniPortMask int
-}
-
-// NewAdapterFlags returns a new RWCore config
-func NewAdapterFlags() *AdapterFlags {
- var adapterFlags = AdapterFlags{ // Default values
- InstanceID: defaultInstanceid,
- KafkaAdapterAddress: defaultKafkaadapteraddress,
- KafkaClusterAddress: defaultKafkaclusteraddress,
- KVStoreType: defaultKvstoretype,
- KVStoreTimeout: defaultKvstoretimeout,
- KVStoreAddress: defaultKvstoreaddress,
- Topic: defaultTopic,
- CoreTopic: defaultCoreTopic,
- EventTopic: defaultEventTopic,
- LogLevel: defaultLoglevel,
- OnuNumber: defaultOnunumber,
- Banner: defaultBanner,
- DisplayVersionOnly: defaultDisplayVersionOnly,
- AccIncrEvto: defaultAccIncrEvto,
- ProbeHost: defaultProbeHost,
- ProbePort: defaultProbePort,
- LiveProbeInterval: defaultLiveProbeInterval,
- NotLiveProbeInterval: defaultNotLiveProbeInterval,
- HeartbeatCheckInterval: defaultHearbeatCheckInterval,
- HeartbeatFailReportInterval: defaultHearbeatFailReportInterval,
- KafkaReconnectRetries: defaultKafkaReconnectRetries,
- CurrentReplica: defaultCurrentReplica,
- TotalReplicas: defaultTotalReplicas,
- MaxTimeoutInterAdapterComm: defaultMaxTimeoutInterAdapterComm,
- MaxTimeoutReconciling: defaultMaxTimeoutReconciling,
- TraceEnabled: defaultTraceEnabled,
- TraceAgentAddress: defaultTraceAgentAddress,
- LogCorrelationEnabled: defaultLogCorrelationEnabled,
- OnuVendorIds: defaultOnuVendorIds,
- MetricsEnabled: defaultMetricsEnabled,
- MibAuditInterval: defaultMibAuditInterval,
- AlarmAuditInterval: defaultAlarmAuditInterval,
- OmciTimeout: defaultOmciTimeout,
- DownloadToAdapterTimeout: defaultDlToAdapterTimeout,
- DownloadToOnuTimeout4MB: defaultDlToOnuTimeoutPer4MB,
- UniPortMask: defaultUniPortMask,
- }
- return &adapterFlags
+ MinBackoffRetryDelay time.Duration
+ MaxBackoffRetryDelay time.Duration
+ AdapterEndpoint string
+ GrpcAddress string
+ CoreEndpoint string
+ RPCTimeout time.Duration
}
// ParseCommandArguments parses the arguments when running read-write adaptercore service
-func (so *AdapterFlags) ParseCommandArguments() {
+func (so *AdapterFlags) ParseCommandArguments(args []string) {
- help := "Kafka - Adapter messaging address"
- flag.StringVar(&(so.KafkaAdapterAddress), "kafka_adapter_address", defaultKafkaadapteraddress, help)
+ fs := flag.NewFlagSet(os.Args[0], flag.ExitOnError)
- help = "Kafka - Cluster messaging address"
- flag.StringVar(&(so.KafkaClusterAddress), "kafka_cluster_address", defaultKafkaclusteraddress, help)
+ fs.StringVar(&(so.KafkaClusterAddress),
+ "kafka_cluster_address",
+ "127.0.0.1:9092",
+ "Kafka - Cluster messaging address")
- help = "Open ONU topic"
- baseAdapterTopic := flag.String("adapter_topic", defaultTopic, help)
+ fs.StringVar(&(so.EventTopic),
+ "event_topic",
+ "voltha.events",
+ "Event topic")
- help = "Core topic"
- flag.StringVar(&(so.CoreTopic), "core_topic", defaultCoreTopic, help)
+ fs.StringVar(&(so.KVStoreType),
+ "kv_store_type",
+ EtcdStoreName,
+ "KV store type")
- help = "Event topic"
- flag.StringVar(&(so.EventTopic), "event_topic", defaultEventTopic, help)
+ fs.DurationVar(&(so.KVStoreTimeout),
+ "kv_store_request_timeout",
+ 5*time.Second,
+ "The default timeout when making a kv store request")
- help = "KV store type"
- flag.StringVar(&(so.KVStoreType), "kv_store_type", defaultKvstoretype, help)
+ fs.StringVar(&(so.KVStoreAddress),
+ "kv_store_address",
+ "127.0.0.1:2379",
+ "KV store address")
- help = "The default timeout when making a kv store request"
- flag.DurationVar(&(so.KVStoreTimeout), "kv_store_request_timeout", defaultKvstoretimeout, help)
+ fs.StringVar(&(so.LogLevel),
+ "log_level",
+ "WARN",
+ "Log level")
- help = "KV store address"
- flag.StringVar(&(so.KVStoreAddress), "kv_store_address", defaultKvstoreaddress, help)
+ fs.IntVar(&(so.OnuNumber),
+ "onu_number",
+ 1,
+ "Number of ONUs")
- help = "Log level"
- flag.StringVar(&(so.LogLevel), "log_level", defaultLoglevel, help)
+ fs.BoolVar(&(so.Banner),
+ "banner",
+ false,
+ "Show startup banner log lines")
- help = "Number of ONUs"
- flag.IntVar(&(so.OnuNumber), "onu_number", defaultOnunumber, help)
+ fs.BoolVar(&(so.DisplayVersionOnly),
+ "version",
+ false,
+ "Show version information and exit")
- help = "Show startup banner log lines"
- flag.BoolVar(&(so.Banner), "banner", defaultBanner, help)
+ fs.BoolVar(&(so.AccIncrEvto),
+ "accept_incr_evto",
+ false,
+ "Acceptance of incremental EVTOCD configuration")
- help = "Show version information and exit"
- flag.BoolVar(&(so.DisplayVersionOnly), "version", defaultDisplayVersionOnly, help)
+ fs.StringVar(&(so.ProbeHost),
+ "probe_host",
+ "",
+ "The address on which to listen to answer liveness and readiness probe queries over HTTP")
- help = "Acceptance of incremental EVTOCD configuration"
- flag.BoolVar(&(so.AccIncrEvto), "accept_incr_evto", defaultAccIncrEvto, help)
+ fs.IntVar(&(so.ProbePort),
+ "probe_port",
+ 8080,
+ "The port on which to listen to answer liveness and readiness probe queries over HTTP")
- help = "The address on which to listen to answer liveness and readiness probe queries over HTTP"
- flag.StringVar(&(so.ProbeHost), "probe_host", defaultProbeHost, help)
+ fs.DurationVar(&(so.LiveProbeInterval),
+ "live_probe_interval",
+ 60*time.Second,
+ "Number of seconds for the default liveliness check")
- help = "The port on which to listen to answer liveness and readiness probe queries over HTTP"
- flag.IntVar(&(so.ProbePort), "probe_port", defaultProbePort, help)
+ fs.DurationVar(&(so.NotLiveProbeInterval),
+ "not_live_probe_interval",
+ 60*time.Second,
+ "Number of seconds for liveliness check if probe is not running")
- help = "Number of seconds for the default liveliness check"
- flag.DurationVar(&(so.LiveProbeInterval), "live_probe_interval", defaultLiveProbeInterval, help)
+ fs.DurationVar(&(so.HeartbeatCheckInterval),
+ "hearbeat_check_interval",
+ 30*time.Second,
+ "Number of seconds for heartbeat check interval")
- help = "Number of seconds for liveliness check if probe is not running"
- flag.DurationVar(&(so.NotLiveProbeInterval), "not_live_probe_interval", defaultNotLiveProbeInterval, help)
+ fs.DurationVar(&(so.HeartbeatFailReportInterval),
+ "hearbeat_fail_interval",
+ 30*time.Second,
+ "Number of seconds adapter has to wait before reporting core on the hearbeat check failure")
- help = "Number of seconds for heartbeat check interval"
- flag.DurationVar(&(so.HeartbeatCheckInterval), "hearbeat_check_interval", defaultHearbeatCheckInterval, help)
+ fs.IntVar(&(so.KafkaReconnectRetries),
+ "kafka_reconnect_retries",
+ -1,
+ "Number of retries to connect to Kafka")
- help = "Number of seconds adapter has to wait before reporting core on the hearbeat check failure"
- flag.DurationVar(&(so.HeartbeatFailReportInterval), "hearbeat_fail_interval", defaultHearbeatFailReportInterval, help)
+ fs.IntVar(&(so.CurrentReplica),
+ "current_replica",
+ 1,
+ "Replica number of this particular instance")
- help = "Number of retries to connect to Kafka"
- flag.IntVar(&(so.KafkaReconnectRetries), "kafka_reconnect_retries", defaultKafkaReconnectRetries, help)
+ fs.IntVar(&(so.TotalReplicas),
+ "total_replica",
+ 1,
+ "Total number of instances for this adapter")
- help = "Replica number of this particular instance"
- flag.IntVar(&(so.CurrentReplica), "current_replica", defaultCurrentReplica, help)
+ fs.DurationVar(&(so.MaxTimeoutInterAdapterComm),
+ "max_timeout_interadapter_comm",
+ 30*time.Second,
+ "Maximum Number of seconds for the default interadapter communication timeout")
- help = "Total number of instances for this adapter"
- flag.IntVar(&(so.TotalReplicas), "total_replica", defaultTotalReplicas, help)
+ fs.DurationVar(&(so.MaxTimeoutReconciling),
+ "max_timeout_reconciling",
+ 10*time.Second,
+ "Maximum Number of seconds for the default ONU reconciling timeout")
- help = "Maximum Number of seconds for the default interadapter communication timeout"
- flag.DurationVar(&(so.MaxTimeoutInterAdapterComm), "max_timeout_interadapter_comm",
- defaultMaxTimeoutInterAdapterComm, help)
+ fs.BoolVar(&(so.TraceEnabled),
+ "trace_enabled",
+ false,
+ "Whether to send logs to tracing agent")
- help = "Maximum Number of seconds for the default ONU reconciling timeout"
- flag.DurationVar(&(so.MaxTimeoutReconciling), "max_timeout_reconciling",
- defaultMaxTimeoutReconciling, help)
+ fs.StringVar(&(so.TraceAgentAddress),
+ "trace_agent_address",
+ "127.0.0.1:6831",
+ "The address of tracing agent to which span info should be sent")
- help = "Whether to send logs to tracing agent"
- flag.BoolVar(&(so.TraceEnabled), "trace_enabled", defaultTraceEnabled, help)
+ fs.BoolVar(&(so.LogCorrelationEnabled),
+ "log_correlation_enabled",
+ true,
+ "Whether to enrich log statements with fields denoting operation being executed for achieving correlation")
- help = "The address of tracing agent to which span info should be sent"
- flag.StringVar(&(so.TraceAgentAddress), "trace_agent_address", defaultTraceAgentAddress, help)
+ fs.StringVar(&(so.OnuVendorIds),
+ "allowed_onu_vendors",
+ OnuVendorIds,
+ "List of Allowed ONU Vendor Ids")
- help = "Whether to enrich log statements with fields denoting operation being executed for achieving correlation"
- flag.BoolVar(&(so.LogCorrelationEnabled), "log_correlation_enabled", defaultLogCorrelationEnabled, help)
+ fs.BoolVar(&(so.MetricsEnabled),
+ "metrics_enabled",
+ false,
+ "Whether to enable metrics collection")
- help = "List of Allowed ONU Vendor Ids"
- flag.StringVar(&(so.OnuVendorIds), "allowed_onu_vendors", defaultOnuVendorIds, help)
+ fs.DurationVar(&(so.MibAuditInterval),
+ "mib_audit_interval",
+ 300*time.Second,
+ "Mib Audit Interval in seconds - the value zero will disable Mib Audit")
- help = "Whether to enable metrics collection"
- flag.BoolVar(&(so.MetricsEnabled), "metrics_enabled", defaultMetricsEnabled, help)
+ fs.DurationVar(&(so.OmciTimeout),
+ "omci_timeout",
+ 3*time.Second,
+ "OMCI timeout duration - this timeout value is used on the OMCI channel for waiting on response from ONU")
- help = "Mib Audit Interval in seconds - the value zero will disable Mib Audit"
- flag.DurationVar(&(so.MibAuditInterval), "mib_audit_interval", defaultMibAuditInterval, help)
+ fs.DurationVar(&(so.AlarmAuditInterval),
+ "alarm_audit_interval",
+ 300*time.Second,
+ "Alarm Audit Interval in seconds - the value zero will disable alarm audit")
- help = "OMCI timeout duration - this timeout value is used on the OMCI channel for waiting on response from ONU"
- flag.DurationVar(&(so.OmciTimeout), "omci_timeout", defaultOmciTimeout, help)
+ fs.DurationVar(&(so.DownloadToAdapterTimeout),
+ "download_to_adapter_timeout",
+ 10*time.Second,
+ "File download to adapter timeout in seconds")
- help = "Alarm Audit Interval in seconds - the value zero will disable alarm audit"
- flag.DurationVar(&(so.AlarmAuditInterval), "alarm_audit_interval", defaultAlarmAuditInterval, help)
+ fs.DurationVar(&(so.DownloadToOnuTimeout4MB),
+ "download_to_onu_timeout_4MB",
+ 60*time.Minute,
+ "File download to ONU timeout in minutes for a block of 4MB")
- help = "File download to adapter timeout in seconds"
- flag.DurationVar(&(so.DownloadToAdapterTimeout), "download_to_adapter_timeout", defaultDlToAdapterTimeout, help)
+ //Mask to indicate which possibly active ONU UNI state is really reported to the core
+ // compare python code - at the moment restrict active state to the first ONU UNI port
+ // check is limited to max 16 uni ports - cmp above UNI limit!!!
+ fs.IntVar(&(so.UniPortMask),
+ "uni_port_mask",
+ 0x0001,
+ "The bitmask to identify UNI ports that need to be enabled")
- help = "File download to ONU timeout in minutes for a block of 4MB"
- flag.DurationVar(&(so.DownloadToOnuTimeout4MB), "download_to_onu_timeout_4MB", defaultDlToOnuTimeoutPer4MB, help)
+ fs.StringVar(&(so.GrpcAddress),
+ "grpc_address",
+ ":50060",
+ "Adapter GRPC Server address")
- help = "The bitmask to identify UNI ports that need to be enabled"
- flag.IntVar(&(so.UniPortMask), "uni_port_mask", defaultUniPortMask, help)
+ fs.StringVar(&(so.CoreEndpoint),
+ "core_endpoint",
+ ":55555",
+ "Core endpoint")
- flag.Parse()
+ fs.StringVar(&(so.AdapterEndpoint),
+ "adapter_endpoint",
+ "",
+ "Adapter Endpoint")
+
+ fs.DurationVar(&(so.RPCTimeout),
+ "rpc_timeout",
+ 10*time.Second,
+ "The default timeout when making an RPC request")
+
+ fs.DurationVar(&(so.MinBackoffRetryDelay),
+ "min_retry_delay",
+ 500*time.Millisecond,
+ "The minimum number of milliseconds to delay before a connection retry attempt")
+
+ fs.DurationVar(&(so.MaxBackoffRetryDelay),
+ "max_retry_delay",
+ 10*time.Second,
+ "The maximum number of milliseconds to delay before a connection retry attempt")
+
+ _ = fs.Parse(args)
containerName := getContainerInfo()
if len(containerName) > 0 {
so.InstanceID = containerName
+ } else {
+ so.InstanceID = "openonu"
}
- so.Topic = fmt.Sprintf("%s_%d", *baseAdapterTopic, int32(so.CurrentReplica))
-
}
func getContainerInfo() string {
diff --git a/internal/pkg/onuadaptercore/adapter_download_manager.go b/internal/pkg/onuadaptercore/adapter_download_manager.go
index 6803aed..6328f72 100644
--- a/internal/pkg/onuadaptercore/adapter_download_manager.go
+++ b/internal/pkg/onuadaptercore/adapter_download_manager.go
@@ -30,9 +30,9 @@
"sync"
"time"
- "github.com/opencord/voltha-protos/v4/go/voltha"
+ "github.com/opencord/voltha-protos/v5/go/voltha"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
)
// ### downloadToAdapter related definitions ####
diff --git a/internal/pkg/onuadaptercore/alarm_manager.go b/internal/pkg/onuadaptercore/alarm_manager.go
index 0dbc5b8..b9d4d68 100644
--- a/internal/pkg/onuadaptercore/alarm_manager.go
+++ b/internal/pkg/onuadaptercore/alarm_manager.go
@@ -28,9 +28,9 @@
"github.com/looplab/fsm"
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/events/eventif"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- "github.com/opencord/voltha-protos/v4/go/voltha"
+ "github.com/opencord/voltha-lib-go/v7/pkg/events/eventif"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ "github.com/opencord/voltha-protos/v5/go/voltha"
)
const (
diff --git a/internal/pkg/onuadaptercore/common.go b/internal/pkg/onuadaptercore/common.go
index 0a39a7e..b0b10e3 100644
--- a/internal/pkg/onuadaptercore/common.go
+++ b/internal/pkg/onuadaptercore/common.go
@@ -18,7 +18,7 @@
package adaptercoreonu
import (
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
)
var logger log.CLogger
diff --git a/internal/pkg/onuadaptercore/device_handler.go b/internal/pkg/onuadaptercore/device_handler.go
index cfe99c9..2c5a4c8 100644
--- a/internal/pkg/onuadaptercore/device_handler.go
+++ b/internal/pkg/onuadaptercore/device_handler.go
@@ -25,25 +25,23 @@
"sync"
"time"
- "github.com/opencord/voltha-protos/v4/go/tech_profile"
+ "github.com/opencord/voltha-openonu-adapter-go/internal/pkg/config"
+ "github.com/opencord/voltha-protos/v5/go/tech_profile"
"github.com/gogo/protobuf/proto"
- "github.com/golang/protobuf/ptypes"
"github.com/looplab/fsm"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
- "github.com/opencord/voltha-lib-go/v5/pkg/db"
- "github.com/opencord/voltha-lib-go/v5/pkg/events/eventif"
- flow "github.com/opencord/voltha-lib-go/v5/pkg/flows"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- vc "github.com/opencord/voltha-protos/v4/go/common"
- "github.com/opencord/voltha-protos/v4/go/extension"
- ic "github.com/opencord/voltha-protos/v4/go/inter_container"
- "github.com/opencord/voltha-protos/v4/go/openflow_13"
- of "github.com/opencord/voltha-protos/v4/go/openflow_13"
- ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
- oop "github.com/opencord/voltha-protos/v4/go/openolt"
- "github.com/opencord/voltha-protos/v4/go/voltha"
+ "github.com/opencord/voltha-lib-go/v7/pkg/db"
+ "github.com/opencord/voltha-lib-go/v7/pkg/events/eventif"
+ flow "github.com/opencord/voltha-lib-go/v7/pkg/flows"
+ vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ vc "github.com/opencord/voltha-protos/v5/go/common"
+ "github.com/opencord/voltha-protos/v5/go/extension"
+ ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+ of "github.com/opencord/voltha-protos/v5/go/openflow_13"
+ oop "github.com/opencord/voltha-protos/v5/go/openolt"
+ "github.com/opencord/voltha-protos/v5/go/voltha"
)
// Constants for timeouts
@@ -178,11 +176,11 @@
parentID string
ponPortNumber uint32
- coreProxy adapterif.CoreProxy
- AdapterProxy adapterif.AdapterProxy
- EventProxy eventif.EventProxy
+ coreClient *vgrpc.Client
+ EventProxy eventif.EventProxy
pmConfigs *voltha.PmConfigs
+ config *config.AdapterFlags
pOpenOnuAc *OpenONUAC
pDeviceStateFsm *fsm.FSM
@@ -237,11 +235,11 @@
}
//newDeviceHandler creates a new device handler
-func newDeviceHandler(ctx context.Context, cp adapterif.CoreProxy, ap adapterif.AdapterProxy, ep eventif.EventProxy, device *voltha.Device, adapter *OpenONUAC) *deviceHandler {
+func newDeviceHandler(ctx context.Context, cc *vgrpc.Client, ep eventif.EventProxy, device *voltha.Device, adapter *OpenONUAC) *deviceHandler {
var dh deviceHandler
- dh.coreProxy = cp
- dh.AdapterProxy = ap
+ dh.coreClient = cc
dh.EventProxy = ep
+ dh.config = adapter.config
cloned := (proto.Clone(device)).(*voltha.Device)
dh.deviceID = cloned.Id
dh.DeviceType = cloned.Type
@@ -327,7 +325,7 @@
//adoptOrReconcileDevice adopts the ONU device
func (dh *deviceHandler) adoptOrReconcileDevice(ctx context.Context, device *voltha.Device) {
- logger.Debugw(ctx, "Adopt_or_reconcile_device", log.Fields{"device-id": device.Id, "Address": device.GetHostAndPort()})
+ logger.Debugw(ctx, "adopt_or_reconcile_device", log.Fields{"device-id": device.Id, "Address": device.GetHostAndPort()})
logger.Debugw(ctx, "Device FSM: ", log.Fields{"state": string(dh.pDeviceStateFsm.Current())})
if dh.pDeviceStateFsm.Is(devStNull) {
@@ -338,7 +336,7 @@
// device.PmConfigs is not nil in cases when adapter restarts. We should not re-set the core again.
if device.PmConfigs == nil {
// Now, set the initial PM configuration for that device
- if err := dh.coreProxy.DevicePMConfigUpdate(ctx, dh.pmConfigs); err != nil {
+ if err := dh.updatePMConfigInCore(ctx, dh.pmConfigs); err != nil {
logger.Errorw(ctx, "error updating pm config to core", log.Fields{"device-id": dh.deviceID, "err": err})
}
}
@@ -348,36 +346,26 @@
}
-func (dh *deviceHandler) processInterAdapterOMCIReceiveMessage(ctx context.Context, msg *ic.InterAdapterMessage) error {
- msgBody := msg.GetBody()
- omciMsg := &ic.InterAdapterOmciMessage{}
- if err := ptypes.UnmarshalAny(msgBody, omciMsg); err != nil {
- logger.Warnw(ctx, "cannot-unmarshal-omci-msg-body", log.Fields{
- "device-id": dh.deviceID, "error": err})
- return err
- }
-
+func (dh *deviceHandler) handleOMCIIndication(ctx context.Context, msg *ic.OmciMessage) error {
/* msg print moved symmetrically to omci_cc, if wanted here as additional debug, than perhaps only based on additional debug setting!
//assuming omci message content is hex coded!
// with restricted output of 16(?) bytes would be ...omciMsg.Message[:16]
logger.Debugw(ctx, "inter-adapter-recv-omci", log.Fields{
"device-id": dh.deviceID, "RxOmciMessage": hex.EncodeToString(omciMsg.Message)})
*/
+
pDevEntry := dh.getOnuDeviceEntry(ctx, true)
if pDevEntry != nil {
if pDevEntry.PDevOmciCC != nil {
- return pDevEntry.PDevOmciCC.receiveMessage(log.WithSpanFromContext(context.TODO(), ctx), omciMsg.Message)
+ return pDevEntry.PDevOmciCC.receiveMessage(log.WithSpanFromContext(context.TODO(), ctx), msg.Message)
}
- logger.Debugw(ctx, "omciCC not ready to receive omci messages - incoming omci message ignored", log.Fields{"rxMsg": omciMsg.Message})
+ logger.Debugw(ctx, "omciCC not ready to receive omci messages - incoming omci message ignored", log.Fields{"rxMsg": msg.Message})
}
logger.Errorw(ctx, "No valid OnuDevice -aborting", log.Fields{"device-id": dh.deviceID})
return fmt.Errorf("no valid OnuDevice: %s", dh.deviceID)
}
-func (dh *deviceHandler) processInterAdapterTechProfileDownloadReqMessage(
- ctx context.Context,
- msg *ic.InterAdapterMessage) error {
-
+func (dh *deviceHandler) handleTechProfileDownloadRequest(ctx context.Context, techProfMsg *ic.TechProfileDownloadMessage) error {
logger.Infow(ctx, "tech-profile-download-request", log.Fields{"device-id": dh.deviceID})
pDevEntry := dh.getOnuDeviceEntry(ctx, true)
@@ -400,14 +388,6 @@
// at least 'mib-downloaded' should be reached for processing of this specific ONU configuration
// if (dh.deviceReason == "stopping-openomci") || (dh.deviceReason == "omci-admin-lock")
- msgBody := msg.GetBody()
- techProfMsg := &ic.InterAdapterTechProfileDownloadMessage{}
- if err := ptypes.UnmarshalAny(msgBody, techProfMsg); err != nil {
- logger.Warnw(ctx, "cannot-unmarshal-techprof-msg-body", log.Fields{
- "device-id": dh.deviceID, "error": err})
- return err
- }
-
// we have to lock access to TechProfile processing based on different messageType calls or
// even to fast subsequent calls of the same messageType as well as OnuKVStore processing due
// to possible concurrent access by flow processing
@@ -429,7 +409,7 @@
if bTpModify := pDevEntry.updateOnuUniTpPath(ctx, uniID, uint8(tpID), techProfMsg.TpInstancePath); bTpModify {
switch tpInst := techProfMsg.TechTpInstance.(type) {
- case *ic.InterAdapterTechProfileDownloadMessage_TpInstance:
+ case *ic.TechProfileDownloadMessage_TpInstance:
logger.Debugw(ctx, "onu-uni-tp-path-modified", log.Fields{"uniID": uniID, "tp-path": techProfMsg.TpInstancePath, "tpID": tpID})
// if there has been some change for some uni TechProfilePath
//in order to allow concurrent calls to other dh instances we do not wait for execution here
@@ -475,9 +455,8 @@
return nil
}
-func (dh *deviceHandler) processInterAdapterDeleteGemPortReqMessage(
- ctx context.Context,
- msg *ic.InterAdapterMessage) error {
+func (dh *deviceHandler) handleDeleteGemPortRequest(ctx context.Context, delGemPortMsg *ic.DeleteGemPortMessage) error {
+ logger.Infow(ctx, "delete-gem-port-request", log.Fields{"device-id": dh.deviceID})
if dh.pOnuTP == nil {
//should normally not happen ...
@@ -485,15 +464,6 @@
log.Fields{"device-id": dh.deviceID})
return fmt.Errorf("techProfile DelGem request while onuTechProf instance not setup: %s", dh.deviceID)
}
-
- msgBody := msg.GetBody()
- delGemPortMsg := &ic.InterAdapterDeleteGemPortMessage{}
- if err := ptypes.UnmarshalAny(msgBody, delGemPortMsg); err != nil {
- logger.Warnw(ctx, "cannot-unmarshal-delete-gem-msg-body", log.Fields{
- "device-id": dh.deviceID, "error": err})
- return err
- }
-
//compare TECH_PROFILE_DOWNLOAD_REQUEST
dh.pOnuTP.lockTpProcMutex()
defer dh.pOnuTP.unlockTpProcMutex()
@@ -516,9 +486,8 @@
}
-func (dh *deviceHandler) processInterAdapterDeleteTcontReqMessage(
- ctx context.Context,
- msg *ic.InterAdapterMessage) error {
+func (dh *deviceHandler) handleDeleteTcontRequest(ctx context.Context, delTcontMsg *ic.DeleteTcontMessage) error {
+ logger.Infow(ctx, "delete-tcont-request", log.Fields{"device-id": dh.deviceID})
pDevEntry := dh.getOnuDeviceEntry(ctx, true)
if pDevEntry == nil {
@@ -532,14 +501,6 @@
return fmt.Errorf("techProfile DelTcont request while onuTechProf instance not setup: %s", dh.deviceID)
}
- msgBody := msg.GetBody()
- delTcontMsg := &ic.InterAdapterDeleteTcontMessage{}
- if err := ptypes.UnmarshalAny(msgBody, delTcontMsg); err != nil {
- logger.Warnw(ctx, "cannot-unmarshal-delete-tcont-msg-body", log.Fields{
- "device-id": dh.deviceID, "error": err})
- return err
- }
-
//compare TECH_PROFILE_DOWNLOAD_REQUEST
dh.pOnuTP.lockTpProcMutex()
defer dh.pOnuTP.unlockTpProcMutex()
@@ -616,52 +577,10 @@
return nil
}
-//processInterAdapterMessage sends the proxied messages to the target device
-// If the proxy address is not found in the unmarshalled message, it first fetches the onu device for which the message
-// is meant, and then send the unmarshalled omci message to this onu
-func (dh *deviceHandler) processInterAdapterMessage(ctx context.Context, msg *ic.InterAdapterMessage) error {
- msgID := msg.Header.Id
- msgType := msg.Header.Type
- fromTopic := msg.Header.FromTopic
- toTopic := msg.Header.ToTopic
- toDeviceID := msg.Header.ToDeviceId
- proxyDeviceID := msg.Header.ProxyDeviceId
- logger.Debugw(ctx, "InterAdapter message header", log.Fields{"msgID": msgID, "msgType": msgType,
- "fromTopic": fromTopic, "toTopic": toTopic, "toDeviceID": toDeviceID, "proxyDeviceID": proxyDeviceID})
-
- switch msgType {
- // case ic.InterAdapterMessageType_ONU_IND_REQUEST: was handled by OpenONUAC already - see comments there
- //OMCI_RESPONSE also accepted acc. to VOL-3756 (OMCI_REQUEST request was legacy code)
- case ic.InterAdapterMessageType_OMCI_RESPONSE, ic.InterAdapterMessageType_OMCI_REQUEST:
- {
- return dh.processInterAdapterOMCIReceiveMessage(ctx, msg)
- }
- case ic.InterAdapterMessageType_TECH_PROFILE_DOWNLOAD_REQUEST:
- {
- return dh.processInterAdapterTechProfileDownloadReqMessage(ctx, msg)
- }
- case ic.InterAdapterMessageType_DELETE_GEM_PORT_REQUEST:
- {
- return dh.processInterAdapterDeleteGemPortReqMessage(ctx, msg)
-
- }
- case ic.InterAdapterMessageType_DELETE_TCONT_REQUEST:
- {
- return dh.processInterAdapterDeleteTcontReqMessage(ctx, msg)
- }
- default:
- {
- logger.Errorw(ctx, "inter-adapter-unhandled-type", log.Fields{
- "msgType": msg.Header.Type, "device-id": dh.deviceID})
- return fmt.Errorf("inter-adapter-unhandled-type: %d, %s", msg.Header.Type, dh.deviceID)
- }
- }
-}
-
//FlowUpdateIncremental removes and/or adds the flow changes on a given device
func (dh *deviceHandler) FlowUpdateIncremental(ctx context.Context,
- apOfFlowChanges *openflow_13.FlowChanges,
- apOfGroupChanges *openflow_13.FlowGroupChanges, apFlowMetaData *voltha.FlowMetadata) error {
+ apOfFlowChanges *of.FlowChanges,
+ apOfGroupChanges *of.FlowGroupChanges, apFlowMetaData *voltha.FlowMetadata) error {
logger.Debugw(ctx, "FlowUpdateIncremental started", log.Fields{"device-id": dh.deviceID, "metadata": apFlowMetaData})
var retError error = nil
//Remove flows (always remove flows first - remove old and add new with same cookie may be part of the same request)
@@ -806,8 +725,11 @@
} else {
logger.Debugw(ctx, "DeviceStateUpdate upon disable", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
"OperStatus": voltha.OperStatus_UNKNOWN, "device-id": dh.deviceID})
- if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx),
- dh.deviceID, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN); err != nil {
+ if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+ DeviceId: dh.deviceID,
+ ConnStatus: voltha.ConnectStatus_REACHABLE,
+ OperStatus: voltha.OperStatus_UNKNOWN,
+ }); err != nil {
//TODO with VOL-3045/VOL-3046: return the error and stop further processing
logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
}
@@ -914,10 +836,16 @@
techProfsFound = true // set to true if we found TP once for any UNI port
for tpID := range uniData.PersTpPathMap {
// Request the TpInstance again from the openolt adapter in case of reconcile
- iaTechTpInst, err := dh.AdapterProxy.TechProfileInstanceRequest(ctx, uniData.PersTpPathMap[tpID],
- dh.device.ParentPortNo, dh.device.ProxyAddress.OnuId, uint32(uniData.PersUniID),
- dh.pOpenOnuAc.config.Topic, dh.ProxyAddressType,
- dh.parentID, dh.ProxyAddressID)
+ iaTechTpInst, err := dh.getTechProfileInstanceFromParentAdapter(ctx,
+ dh.device.ProxyAddress.AdapterEndpoint,
+ &ic.TechProfileInstanceRequestMessage{
+ DeviceId: dh.device.Id,
+ TpInstancePath: uniData.PersTpPathMap[tpID],
+ ParentDeviceId: dh.parentID,
+ ParentPonPort: dh.device.ParentPortNo,
+ OnuId: dh.device.ProxyAddress.OnuId,
+ UniId: uint32(uniData.PersUniID),
+ })
if err != nil || iaTechTpInst == nil {
logger.Errorw(ctx, "error fetching tp instance",
log.Fields{"tp-id": tpID, "tpPath": uniData.PersTpPathMap[tpID], "uni-id": uniData.PersUniID, "device-id": dh.deviceID, "err": err})
@@ -926,7 +854,7 @@
}
var tpInst tech_profile.TechProfileInstance
switch techTpInst := iaTechTpInst.TechTpInstance.(type) {
- case *ic.InterAdapterTechProfileDownloadMessage_TpInstance: // supports only GPON, XGPON, XGS-PON
+ case *ic.TechProfileDownloadMessage_TpInstance: // supports only GPON, XGPON, XGS-PON
tpInst = *techTpInst.TpInstance
logger.Debugw(ctx, "received-tp-instance-successfully-after-reconcile", log.Fields{
"tp-id": tpID, "tpPath": uniData.PersTpPathMap[tpID], "uni-id": uniData.PersUniID, "device-id": dh.deviceID})
@@ -1152,8 +1080,11 @@
logger.Debugw(ctx, "call DeviceStateUpdate upon reboot", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
"OperStatus": voltha.OperStatus_DISCOVERED, "device-id": dh.deviceID})
- if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, voltha.ConnectStatus_REACHABLE,
- voltha.OperStatus_DISCOVERED); err != nil {
+ if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+ DeviceId: dh.deviceID,
+ ConnStatus: voltha.ConnectStatus_REACHABLE,
+ OperStatus: voltha.OperStatus_DISCOVERED,
+ }); err != nil {
//TODO with VOL-3045/VOL-3046: return the error and stop further processing
logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
return
@@ -1294,8 +1225,7 @@
dh.lockUpgradeFsm.RLock()
if dh.pOnuUpradeFsm != nil {
dh.lockUpgradeFsm.RUnlock()
- onuVolthaDevice, getErr := dh.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
- dh.deviceID, dh.deviceID)
+ onuVolthaDevice, getErr := dh.getDeviceFromCore(ctx, dh.deviceID)
if getErr != nil || onuVolthaDevice == nil {
logger.Errorw(ctx, "Failed to fetch Onu device for image activation", log.Fields{"device-id": dh.deviceID, "err": getErr})
return nil, fmt.Errorf("could not fetch device for device-id: %s", dh.deviceID)
@@ -1358,8 +1288,7 @@
dh.lockUpgradeFsm.RLock()
if dh.pOnuUpradeFsm != nil {
dh.lockUpgradeFsm.RUnlock()
- onuVolthaDevice, getErr := dh.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
- dh.deviceID, dh.deviceID)
+ onuVolthaDevice, getErr := dh.getDeviceFromCore(ctx, dh.deviceID)
if getErr != nil || onuVolthaDevice == nil {
logger.Errorw(ctx, "Failed to fetch Onu device for image commitment", log.Fields{"device-id": dh.deviceID, "err": getErr})
return nil, fmt.Errorf("could not fetch device for device-id: %s", dh.deviceID)
@@ -1513,7 +1442,9 @@
if !dh.isReconciling() {
logger.Infow(ctx, "DeviceUpdate", log.Fields{"deviceReason": dh.device.Reason, "device-id": dh.deviceID})
- _ = dh.coreProxy.DeviceUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.device)
+ if err := dh.updateDeviceInCore(ctx, dh.device); err != nil {
+ logger.Errorw(ctx, "device-update-failed", log.Fields{"device-id": dh.device.Id, "error": err})
+ }
//TODO Need to Update Device Reason To CORE as part of device update userstory
} else {
logger.Debugw(ctx, "reconciling - don't notify core about DeviceUpdate",
@@ -1547,6 +1478,7 @@
}
pPonPort := &voltha.Port{
+ DeviceId: dh.deviceID,
PortNo: ponPortNo,
Label: fmt.Sprintf("pon-%d", ponPortNo),
Type: voltha.Port_PON_ONU,
@@ -1554,7 +1486,7 @@
Peers: []*voltha.Port_PeerPort{{DeviceId: dh.parentID, // Peer device is OLT
PortNo: ponPortNo}}, // Peer port is parent's port number
}
- if err = dh.coreProxy.PortCreated(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, pPonPort); err != nil {
+ if err = dh.createPortInCore(ctx, pPonPort); err != nil {
logger.Fatalf(ctx, "Device FSM: PortCreated-failed-%s", err)
e.Cancel(err)
return
@@ -1814,8 +1746,12 @@
}
logger.Debugw(ctx, "call DeviceStateUpdate upon create interface", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
"OperStatus": voltha.OperStatus_ACTIVATING, "device-id": dh.deviceID})
- if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID,
- voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVATING); err != nil {
+
+ if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+ DeviceId: dh.deviceID,
+ OperStatus: voltha.OperStatus_ACTIVATING,
+ ConnStatus: voltha.ConnectStatus_REACHABLE,
+ }); err != nil {
//TODO with VOL-3045/VOL-3046: return the error and stop further processing
logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
}
@@ -2031,8 +1967,11 @@
}
logger.Debugw(ctx, "call DeviceStateUpdate upon update interface", log.Fields{"ConnectStatus": voltha.ConnectStatus_UNREACHABLE,
"OperStatus": voltha.OperStatus_DISCOVERED, "device-id": dh.deviceID})
- if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID,
- voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_DISCOVERED); err != nil {
+ if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+ DeviceId: dh.deviceID,
+ ConnStatus: voltha.ConnectStatus_UNREACHABLE,
+ OperStatus: voltha.OperStatus_DISCOVERED,
+ }); err != nil {
//TODO with VOL-3045/VOL-3046: return the error and stop further processing
logger.Errorw(ctx, "error-updating-device-state unreachable-discovered",
log.Fields{"device-id": dh.deviceID, "error": err})
@@ -2208,8 +2147,11 @@
// in case of adapter restart connected to an ONU upgrade I would not rely on the image quality
// maybe some 'forced' commitment can be done in this situation from system management (or upgrade restarted)
dh.checkOnOnuImageCommit(ctx)
- if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID,
- voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE); err != nil {
+ if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+ DeviceId: dh.deviceID,
+ ConnStatus: voltha.ConnectStatus_REACHABLE,
+ OperStatus: voltha.OperStatus_ACTIVE,
+ }); err != nil {
//TODO with VOL-3045/VOL-3046: return the error and stop further processing
logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
} else {
@@ -2292,8 +2234,12 @@
func (dh *deviceHandler) processUniDisableStateDoneEvent(ctx context.Context, devEvent OnuDeviceEvent) {
logger.Debugw(ctx, "DeviceStateUpdate upon disable", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
"OperStatus": voltha.OperStatus_UNKNOWN, "device-id": dh.deviceID})
- if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx),
- dh.deviceID, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN); err != nil {
+
+ if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+ DeviceId: dh.deviceID,
+ ConnStatus: voltha.ConnectStatus_REACHABLE,
+ OperStatus: voltha.OperStatus_UNKNOWN,
+ }); err != nil {
//TODO with VOL-3045/VOL-3046: return the error and stop further processing
logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
}
@@ -2322,8 +2268,11 @@
func (dh *deviceHandler) processUniEnableStateDoneEvent(ctx context.Context, devEvent OnuDeviceEvent) {
logger.Debugw(ctx, "DeviceStateUpdate upon re-enable", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
"OperStatus": voltha.OperStatus_ACTIVE, "device-id": dh.deviceID})
- if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, voltha.ConnectStatus_REACHABLE,
- voltha.OperStatus_ACTIVE); err != nil {
+ if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+ DeviceId: dh.deviceID,
+ ConnStatus: voltha.ConnectStatus_REACHABLE,
+ OperStatus: voltha.OperStatus_ACTIVE,
+ }); err != nil {
//TODO with VOL-3045/VOL-3046: return the error and stop further processing
logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
}
@@ -2537,7 +2486,16 @@
uniPort.setOperState(vc.OperStatus_ACTIVE)
if !dh.isReconciling() {
//maybe also use getter functions on uniPort - perhaps later ...
- go dh.coreProxy.PortStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, voltha.Port_ETHERNET_UNI, uniPort.portNo, uniPort.operState)
+ go func(port *onuUniPort) {
+ if err := dh.updatePortStateInCore(ctx, &ic.PortState{
+ DeviceId: dh.deviceID,
+ PortType: voltha.Port_ETHERNET_UNI,
+ PortNo: port.portNo,
+ OperStatus: port.operState,
+ }); err != nil {
+ logger.Errorw(ctx, "port-state-update-failed", log.Fields{"error": err, "port-no": uniPort.portNo, "device-id": dh.deviceID})
+ }
+ }(uniPort)
} else {
logger.Debugw(ctx, "reconciling - don't notify core about PortStateUpdate", log.Fields{"device-id": dh.deviceID})
}
@@ -2557,7 +2515,16 @@
uniPort.setOperState(vc.OperStatus_UNKNOWN)
if !dh.isReconciling() {
//maybe also use getter functions on uniPort - perhaps later ...
- go dh.coreProxy.PortStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, voltha.Port_ETHERNET_UNI, uniPort.portNo, uniPort.operState)
+ go func(port *onuUniPort) {
+ if err := dh.updatePortStateInCore(ctx, &ic.PortState{
+ DeviceId: dh.deviceID,
+ PortType: voltha.Port_ETHERNET_UNI,
+ PortNo: port.portNo,
+ OperStatus: port.operState,
+ }); err != nil {
+ logger.Errorw(ctx, "port-state-update-failed", log.Fields{"error": err, "port-no": uniPort.portNo, "device-id": dh.deviceID})
+ }
+ }(uniPort)
} else {
logger.Debugw(ctx, "reconciling - don't notify core about PortStateUpdate", log.Fields{"device-id": dh.deviceID})
}
@@ -2573,7 +2540,7 @@
eventContext := make(map[string]string)
//Populating event context
// assume giving ParentId in GetDevice twice really gives the ParentDevice (there is no GetParentDevice()...)
- parentDevice, err := dh.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx), dh.parentID, dh.parentID)
+ parentDevice, err := dh.getDeviceFromCore(ctx, dh.parentID)
if err != nil || parentDevice == nil {
logger.Errorw(ctx, "Failed to fetch parent device for OnuEvent",
log.Fields{"parentID": dh.parentID, "err": err})
@@ -2853,7 +2820,7 @@
return kvbackend
}
-func (dh *deviceHandler) getFlowOfbFields(ctx context.Context, apFlowItem *ofp.OfpFlowStats, loMatchVlan *uint16,
+func (dh *deviceHandler) getFlowOfbFields(ctx context.Context, apFlowItem *of.OfpFlowStats, loMatchVlan *uint16,
loAddPcp *uint8, loIPProto *uint32) {
for _, field := range flow.GetOfbFields(apFlowItem) {
@@ -2930,7 +2897,7 @@
} //for all OfbFields
}
-func (dh *deviceHandler) getFlowActions(ctx context.Context, apFlowItem *ofp.OfpFlowStats, loSetPcp *uint8, loSetVlan *uint16) {
+func (dh *deviceHandler) getFlowActions(ctx context.Context, apFlowItem *of.OfpFlowStats, loSetPcp *uint8, loSetVlan *uint16) {
for _, action := range flow.GetActions(apFlowItem) {
switch action.Type {
/* not used:
@@ -2976,7 +2943,7 @@
}
//addFlowItemToUniPort parses the actual flow item to add it to the UniPort
-func (dh *deviceHandler) addFlowItemToUniPort(ctx context.Context, apFlowItem *ofp.OfpFlowStats, apUniPort *onuUniPort,
+func (dh *deviceHandler) addFlowItemToUniPort(ctx context.Context, apFlowItem *of.OfpFlowStats, apUniPort *onuUniPort,
apFlowMetaData *voltha.FlowMetadata) error {
var loSetVlan uint16 = uint16(of.OfpVlanId_OFPVID_NONE) //noValidEntry
var loMatchVlan uint16 = uint16(of.OfpVlanId_OFPVID_PRESENT) //reserved VLANID entry
@@ -3067,7 +3034,7 @@
}
//removeFlowItemFromUniPort parses the actual flow item to remove it from the UniPort
-func (dh *deviceHandler) removeFlowItemFromUniPort(ctx context.Context, apFlowItem *ofp.OfpFlowStats, apUniPort *onuUniPort) error {
+func (dh *deviceHandler) removeFlowItemFromUniPort(ctx context.Context, apFlowItem *of.OfpFlowStats, apUniPort *onuUniPort) error {
//optimization and assumption: the flow cookie uniquely identifies the flow and with that the internal rule
//hence only the cookie is used here to find the relevant flow and possibly remove the rule
//no extra check is done on the rule parameters
@@ -3300,7 +3267,10 @@
dh.setDeviceReason(deviceReason)
if notifyCore {
//TODO with VOL-3045/VOL-3046: return the error and stop further processing at calling position
- if err := dh.coreProxy.DeviceReasonUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, deviceReasonMap[deviceReason]); err != nil {
+ if err := dh.updateDeviceReasonInCore(ctx, &ic.DeviceReason{
+ DeviceId: dh.deviceID,
+ Reason: deviceReasonMap[deviceReason],
+ }); err != nil {
logger.Errorf(ctx, "DeviceReasonUpdate error: %s",
log.Fields{"device-id": dh.deviceID, "error": err}, deviceReasonMap[deviceReason])
return err
@@ -3715,7 +3685,11 @@
}
logger.Debugw(ctx, "reconciling has been finished in time",
log.Fields{"device-id": dh.deviceID})
- if err := dh.coreProxy.DeviceStateUpdate(ctx, dh.deviceID, connectStatus, operState); err != nil {
+ if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+ DeviceId: dh.deviceID,
+ ConnStatus: connectStatus,
+ OperStatus: operState,
+ }); err != nil {
logger.Errorw(ctx, "unable to update device state to core",
log.Fields{"device-id": dh.deviceID, "Err": err})
}
@@ -3830,8 +3804,130 @@
}
logger.Debugw(ctx, "Core DeviceStateUpdate", log.Fields{"connectStatus": connectStatus, "operState": voltha.OperStatus_RECONCILING_FAILED})
- if err := dh.coreProxy.DeviceStateUpdate(ctx, dh.deviceID, connectStatus, voltha.OperStatus_RECONCILING_FAILED); err != nil {
+ if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+ DeviceId: dh.deviceID,
+ ConnStatus: connectStatus,
+ OperStatus: voltha.OperStatus_RECONCILING_FAILED,
+ }); err != nil {
logger.Errorw(ctx, "unable to update device state to core",
log.Fields{"device-id": dh.deviceID, "Err": err})
}
}
+
+/*
+Helper functions to communicate with Core
+*/
+
+func (dh *deviceHandler) getDeviceFromCore(ctx context.Context, deviceID string) (*voltha.Device, error) {
+ cClient, err := dh.coreClient.GetCoreServiceClient()
+ if err != nil || cClient == nil {
+ return nil, err
+ }
+ subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+ defer cancel()
+ logger.Debugw(subCtx, "get-device-from-core", log.Fields{"device-id": deviceID})
+ return cClient.GetDevice(subCtx, &vc.ID{Id: deviceID})
+}
+
+func (dh *deviceHandler) updateDeviceStateInCore(ctx context.Context, deviceStateFilter *ic.DeviceStateFilter) error {
+ cClient, err := dh.coreClient.GetCoreServiceClient()
+ if err != nil || cClient == nil {
+ return err
+ }
+ subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+ defer cancel()
+ _, err = cClient.DeviceStateUpdate(subCtx, deviceStateFilter)
+ logger.Debugw(subCtx, "device-updated-in-core", log.Fields{"device-state": deviceStateFilter, "error": err})
+ return err
+}
+
+func (dh *deviceHandler) updatePMConfigInCore(ctx context.Context, pmConfigs *voltha.PmConfigs) error {
+ cClient, err := dh.coreClient.GetCoreServiceClient()
+ if err != nil || cClient == nil {
+ return err
+ }
+ subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+ defer cancel()
+ _, err = cClient.DevicePMConfigUpdate(subCtx, pmConfigs)
+ logger.Debugw(subCtx, "pmconfig-updated-in-core", log.Fields{"pm-configs": pmConfigs, "error": err})
+ return err
+}
+
+func (dh *deviceHandler) updateDeviceInCore(ctx context.Context, device *voltha.Device) error {
+ cClient, err := dh.coreClient.GetCoreServiceClient()
+ if err != nil || cClient == nil {
+ return err
+ }
+ subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+ defer cancel()
+ _, err = cClient.DeviceUpdate(subCtx, device)
+ logger.Debugw(subCtx, "device-updated-in-core", log.Fields{"device-id": device.Id, "error": err})
+ return err
+}
+
+func (dh *deviceHandler) createPortInCore(ctx context.Context, port *voltha.Port) error {
+ cClient, err := dh.coreClient.GetCoreServiceClient()
+ if err != nil || cClient == nil {
+ return err
+ }
+ subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+ defer cancel()
+ _, err = cClient.PortCreated(subCtx, port)
+ logger.Debugw(subCtx, "port-created-in-core", log.Fields{"port": port, "error": err})
+ return err
+}
+
+func (dh *deviceHandler) updatePortStateInCore(ctx context.Context, portState *ic.PortState) error {
+ cClient, err := dh.coreClient.GetCoreServiceClient()
+ if err != nil || cClient == nil {
+ return err
+ }
+ subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+ defer cancel()
+ _, err = cClient.PortStateUpdate(subCtx, portState)
+ logger.Debugw(subCtx, "port-state-updated-in-core", log.Fields{"port-state": portState, "error": err})
+ return err
+}
+
+func (dh *deviceHandler) updateDeviceReasonInCore(ctx context.Context, reason *ic.DeviceReason) error {
+ cClient, err := dh.coreClient.GetCoreServiceClient()
+ if err != nil || cClient == nil {
+ return err
+ }
+ subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+ defer cancel()
+ _, err = cClient.DeviceReasonUpdate(subCtx, reason)
+ logger.Debugw(subCtx, "device-reason-updated-in-core", log.Fields{"reason": reason, "error": err})
+ return err
+}
+
+/*
+Helper functions to communicate with parent adapter
+*/
+
+func (dh *deviceHandler) getTechProfileInstanceFromParentAdapter(ctx context.Context, parentEndpoint string,
+ request *ic.TechProfileInstanceRequestMessage) (*ic.TechProfileDownloadMessage, error) {
+ pgClient, err := dh.pOpenOnuAc.getParentAdapterServiceClient(parentEndpoint)
+ if err != nil || pgClient == nil {
+ return nil, err
+ }
+ subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.MaxTimeoutInterAdapterComm)
+ defer cancel()
+ logger.Debugw(subCtx, "get-tech-profile-instance", log.Fields{"request": request, "parent-endpoint": parentEndpoint})
+ return pgClient.GetTechProfileInstance(subCtx, request)
+}
+
+func (dh *deviceHandler) sendOMCIRequest(ctx context.Context, parentEndpoint string, request *ic.OmciMessage) error {
+ pgClient, err := dh.pOpenOnuAc.getParentAdapterServiceClient(parentEndpoint)
+ if err != nil || pgClient == nil {
+ return err
+ }
+ subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.MaxTimeoutInterAdapterComm)
+ defer cancel()
+ logger.Debugw(subCtx, "send-omci-request", log.Fields{"request": request, "parent-endpoint": parentEndpoint})
+ _, err = pgClient.ProxyOmciRequest(subCtx, request)
+ if err != nil {
+ logger.Errorw(ctx, "omci-failure", log.Fields{"request": request, "error": err, "request-parent": request.ParentDeviceId, "request-child": request.ChildDeviceId, "request-proxy": request.ProxyAddress})
+ }
+ return err
+}
diff --git a/internal/pkg/onuadaptercore/file_download_manager.go b/internal/pkg/onuadaptercore/file_download_manager.go
index 4793e69..5c3226e 100644
--- a/internal/pkg/onuadaptercore/file_download_manager.go
+++ b/internal/pkg/onuadaptercore/file_download_manager.go
@@ -29,7 +29,7 @@
"sync"
"time"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
)
const cDefaultLocalDir = "/tmp" //this is the default local dir to download to
diff --git a/internal/pkg/onuadaptercore/mib_download.go b/internal/pkg/onuadaptercore/mib_download.go
index be63ead..5d77931 100644
--- a/internal/pkg/onuadaptercore/mib_download.go
+++ b/internal/pkg/onuadaptercore/mib_download.go
@@ -26,10 +26,10 @@
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- //ic "github.com/opencord/voltha-protos/v4/go/inter_container"
- //"github.com/opencord/voltha-protos/v4/go/openflow_13"
- //"github.com/opencord/voltha-protos/v4/go/voltha"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ //ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+ //"github.com/opencord/voltha-protos/v5/go/openflow_13"
+ //"github.com/opencord/voltha-protos/v5/go/voltha"
)
func (onuDeviceEntry *OnuDeviceEntry) enterDLStartingState(ctx context.Context, e *fsm.Event) {
diff --git a/internal/pkg/onuadaptercore/mib_sync.go b/internal/pkg/onuadaptercore/mib_sync.go
index 3716352..1660791 100644
--- a/internal/pkg/onuadaptercore/mib_sync.go
+++ b/internal/pkg/onuadaptercore/mib_sync.go
@@ -28,17 +28,12 @@
"github.com/looplab/fsm"
- //"sync"
"time"
- //"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- //ic "github.com/opencord/voltha-protos/v4/go/inter_container"
- //"github.com/opencord/voltha-protos/v4/go/openflow_13"
- //"github.com/opencord/voltha-protos/v4/go/voltha"
+ "github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
)
type sLastTxMeParameter struct {
diff --git a/internal/pkg/onuadaptercore/omci_ani_config.go b/internal/pkg/onuadaptercore/omci_ani_config.go
index f617f41..33e323d 100644
--- a/internal/pkg/onuadaptercore/omci_ani_config.go
+++ b/internal/pkg/onuadaptercore/omci_ani_config.go
@@ -30,10 +30,10 @@
"github.com/looplab/fsm"
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- //ic "github.com/opencord/voltha-protos/v4/go/inter_container"
- //"github.com/opencord/voltha-protos/v4/go/openflow_13"
- //"github.com/opencord/voltha-protos/v4/go/voltha"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ //ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+ //"github.com/opencord/voltha-protos/v5/go/openflow_13"
+ //"github.com/opencord/voltha-protos/v5/go/voltha"
)
const (
diff --git a/internal/pkg/onuadaptercore/omci_cc.go b/internal/pkg/onuadaptercore/omci_cc.go
index 4006283..be930b8 100644
--- a/internal/pkg/onuadaptercore/omci_cc.go
+++ b/internal/pkg/onuadaptercore/omci_cc.go
@@ -33,14 +33,15 @@
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
- "github.com/opencord/voltha-protos/v4/go/common"
- //"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- ic "github.com/opencord/voltha-protos/v4/go/inter_container"
- //"github.com/opencord/voltha-protos/v4/go/openflow_13"
- //"github.com/opencord/voltha-protos/v4/go/voltha"
+ vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+
+ "github.com/opencord/voltha-protos/v5/go/common"
+ //"github.com/opencord/voltha-lib-go/v7/pkg/kafka"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+ //"github.com/opencord/voltha-protos/v5/go/openflow_13"
+ //"github.com/opencord/voltha-protos/v5/go/voltha"
)
// ### OMCI related definitions - retrieved from Python adapter code/trace ####
@@ -99,8 +100,7 @@
pOnuDeviceEntry *OnuDeviceEntry
deviceID string
pBaseDeviceHandler *deviceHandler
- coreProxy adapterif.CoreProxy
- adapterProxy adapterif.AdapterProxy
+ coreClient *vgrpc.Client
supportExtMsg bool
rxOmciFrameError tOmciReceiveError
@@ -139,15 +139,14 @@
//mib_db (as well as not inluded alarm_db not really used in this code? VERIFY!!)
func newOmciCC(ctx context.Context, onuDeviceEntry *OnuDeviceEntry,
deviceID string, deviceHandler *deviceHandler,
- coreProxy adapterif.CoreProxy, adapterProxy adapterif.AdapterProxy) *omciCC {
+ coreClient *vgrpc.Client) *omciCC {
logger.Debugw(ctx, "init-omciCC", log.Fields{"device-id": deviceID})
var omciCC omciCC
omciCC.enabled = false
omciCC.pOnuDeviceEntry = onuDeviceEntry
omciCC.deviceID = deviceID
omciCC.pBaseDeviceHandler = deviceHandler
- omciCC.coreProxy = coreProxy
- omciCC.adapterProxy = adapterProxy
+ omciCC.coreClient = coreClient
omciCC.supportExtMsg = false
omciCC.rxOmciFrameError = cOmciMessageReceiveNoError
omciCC.txFrames = 0
@@ -590,18 +589,18 @@
"TxOmciMessage": hex.EncodeToString(omciTxRequest.txFrame),
"device-id": oo.deviceID,
"toDeviceType": oo.pBaseDeviceHandler.ProxyAddressType,
- "proxyDeviceID": oo.pBaseDeviceHandler.ProxyAddressID})
+ "proxyDeviceID": oo.pBaseDeviceHandler.ProxyAddressID,
+ "proxyAddress": oo.pBaseDeviceHandler.device.ProxyAddress})
}
- omciMsg := &ic.InterAdapterOmciMessage{
- Message: omciTxRequest.txFrame,
- ProxyAddress: oo.pBaseDeviceHandler.device.ProxyAddress,
- ConnectStatus: common.ConnectStatus_REACHABLE, // If we are sending OMCI messages means we are connected, else we should not be here
+ omciMsg := &ic.OmciMessage{
+ ParentDeviceId: oo.pBaseDeviceHandler.ProxyAddressID,
+ ChildDeviceId: oo.deviceID,
+ Message: omciTxRequest.txFrame,
+ ProxyAddress: oo.pBaseDeviceHandler.device.ProxyAddress,
+ ConnectStatus: common.ConnectStatus_REACHABLE, // If we are sending OMCI messages means we are connected, else we should not be here
}
- if sendErr := oo.adapterProxy.SendInterAdapterMessage(log.WithSpanFromContext(context.Background(), ctx), omciMsg,
- ic.InterAdapterMessageType_OMCI_REQUEST,
- //fromTopic,toType,toDevId, ProxyDevId
- oo.pOnuDeviceEntry.baseDeviceHandler.pOpenOnuAc.config.Topic, oo.pBaseDeviceHandler.ProxyAddressType,
- oo.deviceID, oo.pBaseDeviceHandler.ProxyAddressID, ""); sendErr != nil {
+ sendErr := oo.pBaseDeviceHandler.sendOMCIRequest(ctx, oo.pBaseDeviceHandler.device.ProxyAddress.AdapterEndpoint, omciMsg)
+ if sendErr != nil {
logger.Errorw(ctx, "send omci request error", log.Fields{"ChildId": oo.deviceID, "error": sendErr})
return sendErr
}
diff --git a/internal/pkg/onuadaptercore/omci_onu_upgrade.go b/internal/pkg/onuadaptercore/omci_onu_upgrade.go
index 1c86e97..8e8a82e 100644
--- a/internal/pkg/onuadaptercore/omci_onu_upgrade.go
+++ b/internal/pkg/onuadaptercore/omci_onu_upgrade.go
@@ -29,8 +29,8 @@
"github.com/looplab/fsm"
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- "github.com/opencord/voltha-protos/v4/go/voltha"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ "github.com/opencord/voltha-protos/v5/go/voltha"
)
const cMaxUint32 = ^uint32(0)
diff --git a/internal/pkg/onuadaptercore/omci_self_test_handler.go b/internal/pkg/onuadaptercore/omci_self_test_handler.go
index ce82db8..13b6ec3 100644
--- a/internal/pkg/onuadaptercore/omci_self_test_handler.go
+++ b/internal/pkg/onuadaptercore/omci_self_test_handler.go
@@ -20,13 +20,14 @@
import (
"context"
"fmt"
+ "sync"
+ "time"
+
"github.com/looplab/fsm"
"github.com/opencord/omci-lib-go"
"github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- "github.com/opencord/voltha-protos/v4/go/extension"
- "sync"
- "time"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ "github.com/opencord/voltha-protos/v5/go/extension"
)
const (
diff --git a/internal/pkg/onuadaptercore/omci_test_request.go b/internal/pkg/onuadaptercore/omci_test_request.go
index f0b76dc..235f21d 100644
--- a/internal/pkg/onuadaptercore/omci_test_request.go
+++ b/internal/pkg/onuadaptercore/omci_test_request.go
@@ -21,18 +21,11 @@
"context"
"fmt"
- //"sync"
- //"time"
-
gp "github.com/google/gopacket"
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
- //"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- //ic "github.com/opencord/voltha-protos/v4/go/inter_container"
- //"github.com/opencord/voltha-protos/v4/go/openflow_13"
- //"github.com/opencord/voltha-protos/v4/go/voltha"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
)
//omciTestRequest structure holds the information for the OMCI test
diff --git a/internal/pkg/onuadaptercore/omci_vlan_config.go b/internal/pkg/onuadaptercore/omci_vlan_config.go
index 1751cdb..c828804 100644
--- a/internal/pkg/onuadaptercore/omci_vlan_config.go
+++ b/internal/pkg/onuadaptercore/omci_vlan_config.go
@@ -27,15 +27,15 @@
"sync"
"time"
- meters "github.com/opencord/voltha-lib-go/v5/pkg/meters"
- "github.com/opencord/voltha-protos/v4/go/voltha"
+ meters "github.com/opencord/voltha-lib-go/v7/pkg/meters"
+ "github.com/opencord/voltha-protos/v5/go/voltha"
gp "github.com/google/gopacket"
"github.com/looplab/fsm"
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- of "github.com/opencord/voltha-protos/v4/go/openflow_13"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ of "github.com/opencord/voltha-protos/v5/go/openflow_13"
)
const (
diff --git a/internal/pkg/onuadaptercore/onu_device_db.go b/internal/pkg/onuadaptercore/onu_device_db.go
index 83f419b..17f352b 100644
--- a/internal/pkg/onuadaptercore/onu_device_db.go
+++ b/internal/pkg/onuadaptercore/onu_device_db.go
@@ -25,7 +25,7 @@
"sync"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
)
type meDbMap map[me.ClassID]map[uint16]me.AttributeValueMap
diff --git a/internal/pkg/onuadaptercore/onu_device_entry.go b/internal/pkg/onuadaptercore/onu_device_entry.go
index 91510c9..bfb1ab5 100644
--- a/internal/pkg/onuadaptercore/onu_device_entry.go
+++ b/internal/pkg/onuadaptercore/onu_device_entry.go
@@ -25,22 +25,14 @@
"sync"
"time"
+ "github.com/looplab/fsm"
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
+ "github.com/opencord/voltha-lib-go/v7/pkg/db"
+ "github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+ vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
- //"sync"
- //"time"
-
- "github.com/looplab/fsm"
- "github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
- "github.com/opencord/voltha-lib-go/v5/pkg/db"
- "github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
-
- //"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- //ic "github.com/opencord/voltha-protos/v4/go/inter_container"
- //"github.com/opencord/voltha-protos/v4/go/openflow_13"
- //"github.com/opencord/voltha-protos/v4/go/voltha"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
)
const (
@@ -269,8 +261,7 @@
deviceID string
baseDeviceHandler *deviceHandler
pOpenOnuAc *OpenONUAC
- coreProxy adapterif.CoreProxy
- adapterProxy adapterif.AdapterProxy
+ coreClient *vgrpc.Client
PDevOmciCC *omciCC
pOnuDB *onuDeviceDB
mibTemplateKVStore *db.Backend
@@ -319,8 +310,7 @@
onuDeviceEntry.deviceID = dh.deviceID
onuDeviceEntry.baseDeviceHandler = dh
onuDeviceEntry.pOpenOnuAc = dh.pOpenOnuAc
- onuDeviceEntry.coreProxy = dh.coreProxy
- onuDeviceEntry.adapterProxy = dh.AdapterProxy
+ onuDeviceEntry.coreClient = dh.coreClient
onuDeviceEntry.devState = DeviceStatusInit
onuDeviceEntry.sOnuPersistentData.PersUniConfig = make([]uniPersConfig, 0)
onuDeviceEntry.sOnuPersistentData.PersTcontMap = make(map[uint16]uint16)
@@ -520,8 +510,7 @@
func (oo *OnuDeviceEntry) start(ctx context.Context) error {
logger.Debugw(ctx, "OnuDeviceEntry-starting", log.Fields{"for device-id": oo.deviceID})
if oo.PDevOmciCC == nil {
- oo.PDevOmciCC = newOmciCC(ctx, oo, oo.deviceID, oo.baseDeviceHandler,
- oo.coreProxy, oo.adapterProxy)
+ oo.PDevOmciCC = newOmciCC(ctx, oo, oo.deviceID, oo.baseDeviceHandler, oo.coreClient)
if oo.PDevOmciCC == nil {
logger.Errorw(ctx, "Could not create devOmciCc - abort", log.Fields{"for device-id": oo.deviceID})
return fmt.Errorf("could not create devOmciCc %s", oo.deviceID)
diff --git a/internal/pkg/onuadaptercore/onu_image_status.go b/internal/pkg/onuadaptercore/onu_image_status.go
index f36a44e..d226134 100755
--- a/internal/pkg/onuadaptercore/onu_image_status.go
+++ b/internal/pkg/onuadaptercore/onu_image_status.go
@@ -26,8 +26,8 @@
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- "github.com/opencord/voltha-protos/v4/go/voltha"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ "github.com/opencord/voltha-protos/v5/go/voltha"
)
//OnuImageStatus implements methods to get status info of onu images
diff --git a/internal/pkg/onuadaptercore/onu_metrics_manager.go b/internal/pkg/onuadaptercore/onu_metrics_manager.go
index c33fb99..c2a2ffb 100644
--- a/internal/pkg/onuadaptercore/onu_metrics_manager.go
+++ b/internal/pkg/onuadaptercore/onu_metrics_manager.go
@@ -28,11 +28,11 @@
"github.com/looplab/fsm"
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/db"
- "github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- "github.com/opencord/voltha-protos/v4/go/extension"
- "github.com/opencord/voltha-protos/v4/go/voltha"
+ "github.com/opencord/voltha-lib-go/v7/pkg/db"
+ "github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ "github.com/opencord/voltha-protos/v5/go/extension"
+ "github.com/opencord/voltha-protos/v5/go/voltha"
)
const (
diff --git a/internal/pkg/onuadaptercore/onu_uni_port.go b/internal/pkg/onuadaptercore/onu_uni_port.go
index 9cfa7de..f4b255f 100644
--- a/internal/pkg/onuadaptercore/onu_uni_port.go
+++ b/internal/pkg/onuadaptercore/onu_uni_port.go
@@ -27,11 +27,11 @@
//"sync"
//"time"
- //"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- vc "github.com/opencord/voltha-protos/v4/go/common"
- of "github.com/opencord/voltha-protos/v4/go/openflow_13"
- "github.com/opencord/voltha-protos/v4/go/voltha"
+ //"github.com/opencord/voltha-lib-go/v7/pkg/kafka"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ vc "github.com/opencord/voltha-protos/v5/go/common"
+ of "github.com/opencord/voltha-protos/v5/go/openflow_13"
+ "github.com/opencord/voltha-protos/v5/go/voltha"
)
type uniPortType uint8
@@ -114,6 +114,7 @@
"name": name, "hwAddr": ofHwAddr, "OperState": ofUniPortState})
pUniPort := &voltha.Port{
+ DeviceId: apDeviceHandler.deviceID,
PortNo: oo.portNo,
Label: oo.name,
Type: voltha.Port_ETHERNET_UNI,
@@ -136,8 +137,7 @@
retryCnt := 0
var err error
for retryCnt = 0; retryCnt < maxRetry; retryCnt++ {
- if err = apDeviceHandler.coreProxy.PortCreated(log.WithSpanFromContext(context.TODO(), ctx),
- apDeviceHandler.deviceID, pUniPort); err != nil {
+ if err = apDeviceHandler.createPortInCore(ctx, pUniPort); err != nil {
logger.Errorf(ctx, "Device FSM: PortCreated-failed-%s, retrying after a delay", err)
// retry after a sleep
time.Sleep(2 * time.Second)
diff --git a/internal/pkg/onuadaptercore/onu_uni_tp.go b/internal/pkg/onuadaptercore/onu_uni_tp.go
index 6e365ab..1e3f9af 100644
--- a/internal/pkg/onuadaptercore/onu_uni_tp.go
+++ b/internal/pkg/onuadaptercore/onu_uni_tp.go
@@ -24,9 +24,8 @@
"strings"
"sync"
- "github.com/opencord/voltha-protos/v4/go/tech_profile"
-
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ "github.com/opencord/voltha-protos/v5/go/tech_profile"
)
//definitions for TechProfileProcessing - copied from OltAdapter:openolt_flowmgr.go
diff --git a/internal/pkg/onuadaptercore/openonu.go b/internal/pkg/onuadaptercore/openonu.go
index cfa2661..eecd002 100644
--- a/internal/pkg/onuadaptercore/openonu.go
+++ b/internal/pkg/onuadaptercore/openonu.go
@@ -24,19 +24,20 @@
"sync"
"time"
- conf "github.com/opencord/voltha-lib-go/v5/pkg/config"
+ vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+ "github.com/opencord/voltha-protos/v5/go/adapter_services"
- "github.com/golang/protobuf/ptypes"
- "github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
- "github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
- "github.com/opencord/voltha-lib-go/v5/pkg/events/eventif"
- "github.com/opencord/voltha-lib-go/v5/pkg/kafka"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- "github.com/opencord/voltha-protos/v4/go/extension"
- ic "github.com/opencord/voltha-protos/v4/go/inter_container"
- "github.com/opencord/voltha-protos/v4/go/openflow_13"
- oop "github.com/opencord/voltha-protos/v4/go/openolt"
- "github.com/opencord/voltha-protos/v4/go/voltha"
+ conf "github.com/opencord/voltha-lib-go/v7/pkg/config"
+ "github.com/opencord/voltha-protos/v5/go/common"
+ "google.golang.org/grpc"
+
+ "github.com/golang/protobuf/ptypes/empty"
+ "github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+ "github.com/opencord/voltha-lib-go/v7/pkg/events/eventif"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ "github.com/opencord/voltha-protos/v5/go/extension"
+ ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+ "github.com/opencord/voltha-protos/v5/go/voltha"
"github.com/opencord/voltha-openonu-adapter-go/internal/pkg/config"
)
@@ -46,10 +47,10 @@
deviceHandlers map[string]*deviceHandler
deviceHandlersCreateChan map[string]chan bool //channels for deviceHandler create events
lockDeviceHandlersMap sync.RWMutex
- coreProxy adapterif.CoreProxy
- adapterProxy adapterif.AdapterProxy
+ coreClient *vgrpc.Client
+ parentAdapterClients map[string]*vgrpc.Client
+ lockParentAdapterClients sync.RWMutex
eventProxy eventif.EventProxy
- kafkaICProxy kafka.InterContainerProxy
kvClient kvstore.Client
cm *conf.ConfigManager
config *config.AdapterFlags
@@ -74,23 +75,22 @@
omciTimeout int // in seconds
alarmAuditInterval time.Duration
dlToOnuTimeout4M time.Duration
+ rpcTimeout time.Duration
}
//NewOpenONUAC returns a new instance of OpenONU_AC
-func NewOpenONUAC(ctx context.Context, kafkaICProxy kafka.InterContainerProxy,
- coreProxy adapterif.CoreProxy, adapterProxy adapterif.AdapterProxy,
- eventProxy eventif.EventProxy, kvClient kvstore.Client, cfg *config.AdapterFlags, cm *conf.ConfigManager) *OpenONUAC {
+func NewOpenONUAC(ctx context.Context, coreClient *vgrpc.Client, eventProxy eventif.EventProxy,
+ kvClient kvstore.Client, cfg *config.AdapterFlags, cm *conf.ConfigManager) *OpenONUAC {
var openOnuAc OpenONUAC
openOnuAc.exitChannel = make(chan int, 1)
openOnuAc.deviceHandlers = make(map[string]*deviceHandler)
openOnuAc.deviceHandlersCreateChan = make(map[string]chan bool)
+ openOnuAc.parentAdapterClients = make(map[string]*vgrpc.Client)
openOnuAc.lockDeviceHandlersMap = sync.RWMutex{}
- openOnuAc.kafkaICProxy = kafkaICProxy
openOnuAc.config = cfg
openOnuAc.cm = cm
+ openOnuAc.coreClient = coreClient
openOnuAc.numOnus = cfg.OnuNumber
- openOnuAc.coreProxy = coreProxy
- openOnuAc.adapterProxy = adapterProxy
openOnuAc.eventProxy = eventProxy
openOnuAc.kvClient = kvClient
openOnuAc.KVStoreAddress = cfg.KVStoreAddress
@@ -110,6 +110,7 @@
openOnuAc.omciTimeout = int(cfg.OmciTimeout.Seconds())
openOnuAc.alarmAuditInterval = cfg.AlarmAuditInterval
openOnuAc.dlToOnuTimeout4M = cfg.DownloadToOnuTimeout4MB
+ openOnuAc.rpcTimeout = cfg.RPCTimeout
openOnuAc.pSupportedFsms = &OmciDeviceFsms{
"mib-synchronizer": {
@@ -202,200 +203,106 @@
return agent
}
-func (oo *OpenONUAC) processInterAdapterONUIndReqMessage(ctx context.Context, msg *ic.InterAdapterMessage) error {
- msgBody := msg.GetBody()
- onuIndication := &oop.OnuIndication{}
- if err := ptypes.UnmarshalAny(msgBody, onuIndication); err != nil {
- logger.Warnw(ctx, "onu-ind-request-cannot-unmarshal-msg-body", log.Fields{"error": err})
- return err
- }
- //ToDeviceId should address a DeviceHandler instance
- targetDevice := msg.Header.ToDeviceId
-
- onuOperstate := onuIndication.GetOperState()
- waitForDhInstPresent := false
- if onuOperstate == "up" {
- //Race condition (relevant in BBSIM-environment only): Due to unsynchronized processing of olt-adapter and rw_core,
- //ONU_IND_REQUEST msg by olt-adapter could arrive a little bit earlier than rw_core was able to announce the corresponding
- //ONU by RPC of Adopt_device(). Therefore it could be necessary to wait with processing of ONU_IND_REQUEST until call of
- //Adopt_device() arrived and DeviceHandler instance was created
- waitForDhInstPresent = true
- }
- if handler := oo.getDeviceHandler(ctx, targetDevice, waitForDhInstPresent); handler != nil {
- logger.Infow(ctx, "onu-ind-request", log.Fields{"device-id": targetDevice,
- "OnuId": onuIndication.GetOnuId(),
- "AdminState": onuIndication.GetAdminState(), "OperState": onuOperstate,
- "SNR": onuIndication.GetSerialNumber()})
-
- if onuOperstate == "up" {
- return handler.createInterface(ctx, onuIndication)
- } else if (onuOperstate == "down") || (onuOperstate == "unreachable") {
- return handler.updateInterface(ctx, onuIndication)
- } else {
- logger.Errorw(ctx, "unknown-onu-ind-request operState", log.Fields{"OnuId": onuIndication.GetOnuId()})
- return fmt.Errorf("invalidOperState: %s, %s", onuOperstate, targetDevice)
- }
- }
- logger.Warnw(ctx, "no handler found for received onu-ind-request", log.Fields{
- "msgToDeviceId": targetDevice})
- return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", targetDevice))
+// GetHealthStatus is used as a service readiness validation as a grpc connection
+func (oo *OpenONUAC) GetHealthStatus(ctx context.Context, empty *empty.Empty) (*voltha.HealthStatus, error) {
+ return &voltha.HealthStatus{State: voltha.HealthStatus_HEALTHY}, nil
}
-// Adapter interface required methods ############## begin #########
-// #################################################################
-
-// for original content compare: (needs according deviceHandler methods)
-// /voltha-openolt-adapter/adaptercore/openolt.go
-
-// Adopt_device creates a new device handler if not present already and then adopts the device
-func (oo *OpenONUAC) Adopt_device(ctx context.Context, device *voltha.Device) error {
+// AdoptDevice creates a new device handler if not present already and then adopts the device
+func (oo *OpenONUAC) AdoptDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
if device == nil {
logger.Warn(ctx, "voltha-device-is-nil")
- return errors.New("nil-device")
+ return nil, errors.New("nil-device")
}
logger.Infow(ctx, "adopt-device", log.Fields{"device-id": device.Id})
var handler *deviceHandler
if handler = oo.getDeviceHandler(ctx, device.Id, false); handler == nil {
- handler := newDeviceHandler(ctx, oo.coreProxy, oo.adapterProxy, oo.eventProxy, device, oo)
+ handler := newDeviceHandler(ctx, oo.coreClient, oo.eventProxy, device, oo)
oo.addDeviceHandlerToMap(ctx, handler)
- go handler.adoptOrReconcileDevice(ctx, device)
- // Launch the creation of the device topic
- // go oo.createDeviceTopic(device)
+
+ // Setup the grpc communication with the parent adapter
+ if err := oo.setupParentInterAdapterClient(ctx, device.ProxyAddress.AdapterEndpoint); err != nil {
+ // TODO: Cleanup on failure needed
+ return nil, err
+ }
+
+ go handler.adoptOrReconcileDevice(log.WithSpanFromContext(context.Background(), ctx), device)
}
- return nil
+ return &empty.Empty{}, nil
}
-//Get_ofp_device_info returns OFP information for the given device
-func (oo *OpenONUAC) Get_ofp_device_info(ctx context.Context, device *voltha.Device) (*ic.SwitchCapability, error) {
- logger.Errorw(ctx, "device-handler-not-set", log.Fields{"device-id": device.Id})
- return nil, fmt.Errorf("device-handler-not-set %s", device.Id)
-}
-
-//Get_ofp_port_info returns OFP port information for the given device
-//200630: method removed as per [VOL-3202]: OF port info is now to be delivered within UniPort create
-// cmp changes in onu_uni_port.go::CreateVolthaPort()
-
-//Process_inter_adapter_message sends messages to a target device (between adapters)
-func (oo *OpenONUAC) Process_inter_adapter_message(ctx context.Context, msg *ic.InterAdapterMessage) error {
- logger.Debugw(ctx, "Process_inter_adapter_message", log.Fields{"msgId": msg.Header.Id,
- "msgProxyDeviceId": msg.Header.ProxyDeviceId, "msgToDeviceId": msg.Header.ToDeviceId, "Type": msg.Header.Type})
-
- if msg.Header.Type == ic.InterAdapterMessageType_ONU_IND_REQUEST {
- // we have to handle ONU_IND_REQUEST already here - see comments in processInterAdapterONUIndReqMessage()
- return oo.processInterAdapterONUIndReqMessage(ctx, msg)
- }
- //ToDeviceId should address a DeviceHandler instance
- targetDevice := msg.Header.ToDeviceId
- if handler := oo.getDeviceHandler(ctx, targetDevice, false); handler != nil {
- /* 200724: modification towards synchronous implementation - possible errors within processing shall be
- * in the accordingly delayed response, some timing effect might result in Techprofile processing for multiple UNI's
- */
- return handler.processInterAdapterMessage(ctx, msg)
- /* so far the processing has been in background with according commented error treatment restrictions:
- go handler.ProcessInterAdapterMessage(msg)
- // error treatment might be more sophisticated
- // by now let's just accept the message on 'communication layer'
- // message content problems have to be evaluated then in the handler
- // and are by now not reported to the calling party (to force what reaction there?)
- return nil
- */
- }
- logger.Warnw(ctx, "no handler found for received Inter-Proxy-message", log.Fields{
- "msgToDeviceId": targetDevice})
- return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", targetDevice))
-}
-
-//Process_tech_profile_instance_request not implemented
-func (oo *OpenONUAC) Process_tech_profile_instance_request(ctx context.Context, msg *ic.InterAdapterTechProfileInstanceRequestMessage) *ic.InterAdapterTechProfileDownloadMessage {
- logger.Error(ctx, "unImplemented")
- return nil
-}
-
-//Adapter_descriptor not implemented
-func (oo *OpenONUAC) Adapter_descriptor(ctx context.Context) error {
- return errors.New("unImplemented")
-}
-
-//Device_types unimplemented
-func (oo *OpenONUAC) Device_types(ctx context.Context) (*voltha.DeviceTypes, error) {
- return nil, errors.New("unImplemented")
-}
-
-//Health returns unimplemented
-func (oo *OpenONUAC) Health(ctx context.Context) (*voltha.HealthStatus, error) {
- return nil, errors.New("unImplemented")
-}
-
-//Reconcile_device is called once when the adapter needs to re-create device - usually on core restart
-func (oo *OpenONUAC) Reconcile_device(ctx context.Context, device *voltha.Device) error {
+//ReconcileDevice is called once when the adapter needs to re-create device - usually on core restart
+func (oo *OpenONUAC) ReconcileDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
if device == nil {
logger.Warn(ctx, "reconcile-device-voltha-device-is-nil")
- return errors.New("nil-device")
+ return nil, errors.New("nil-device")
}
logger.Infow(ctx, "reconcile-device", log.Fields{"device-id": device.Id})
var handler *deviceHandler
if handler = oo.getDeviceHandler(ctx, device.Id, false); handler == nil {
- handler := newDeviceHandler(ctx, oo.coreProxy, oo.adapterProxy, oo.eventProxy, device, oo)
+ handler := newDeviceHandler(ctx, oo.coreClient, oo.eventProxy, device, oo)
oo.addDeviceHandlerToMap(ctx, handler)
handler.device = device
- if err := handler.coreProxy.DeviceStateUpdate(ctx, device.Id, device.ConnectStatus, voltha.OperStatus_RECONCILING); err != nil {
- return fmt.Errorf("not able to update device state to reconciling. Err : %s", err.Error())
+ if err := handler.updateDeviceStateInCore(log.WithSpanFromContext(context.Background(), ctx), &ic.DeviceStateFilter{
+ DeviceId: device.Id,
+ OperStatus: voltha.OperStatus_RECONCILING,
+ ConnStatus: device.ConnectStatus,
+ }); err != nil {
+ return nil, fmt.Errorf("not able to update device state to reconciling. Err : %s", err.Error())
}
- handler.startReconciling(ctx, false)
- go handler.adoptOrReconcileDevice(ctx, handler.device)
+ // Setup the grpc communication with the parent adapter
+ if err := oo.setupParentInterAdapterClient(ctx, device.ProxyAddress.AdapterEndpoint); err != nil {
+ // TODO: Cleanup on failure needed
+ return nil, err
+ }
+
+ handler.startReconciling(log.WithSpanFromContext(context.Background(), ctx), false)
+ go handler.adoptOrReconcileDevice(log.WithSpanFromContext(context.Background(), ctx), handler.device)
// reconcilement will be continued after onu-device entry is added
} else {
- return fmt.Errorf(fmt.Sprintf("device-already-reconciled-or-active-%s", device.Id))
+ return nil, fmt.Errorf(fmt.Sprintf("device-already-reconciled-or-active-%s", device.Id))
}
- return nil
+ return &empty.Empty{}, nil
}
-//Abandon_device unimplemented
-func (oo *OpenONUAC) Abandon_device(ctx context.Context, device *voltha.Device) error {
- return errors.New("unImplemented")
-}
-
-//Disable_device disables the given device
-func (oo *OpenONUAC) Disable_device(ctx context.Context, device *voltha.Device) error {
+//DisableDevice disables the given device
+func (oo *OpenONUAC) DisableDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
logger.Infow(ctx, "disable-device", log.Fields{"device-id": device.Id})
if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
- go handler.disableDevice(ctx, device)
- return nil
+ go handler.disableDevice(log.WithSpanFromContext(context.Background(), ctx), device)
+ return &empty.Empty{}, nil
}
logger.Warnw(ctx, "no handler found for device-disable", log.Fields{"device-id": device.Id})
- return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+ return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
}
-//Reenable_device enables the onu device after disable
-func (oo *OpenONUAC) Reenable_device(ctx context.Context, device *voltha.Device) error {
+//ReEnableDevice enables the onu device after disable
+func (oo *OpenONUAC) ReEnableDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
logger.Infow(ctx, "reenable-device", log.Fields{"device-id": device.Id})
if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
- go handler.reEnableDevice(ctx, device)
- return nil
+ go handler.reEnableDevice(log.WithSpanFromContext(context.Background(), ctx), device)
+ return &empty.Empty{}, nil
}
logger.Warnw(ctx, "no handler found for device-reenable", log.Fields{"device-id": device.Id})
- return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+ return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
}
-//Reboot_device reboots the given device
-func (oo *OpenONUAC) Reboot_device(ctx context.Context, device *voltha.Device) error {
+//RebootDevice reboots the given device
+func (oo *OpenONUAC) RebootDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
logger.Infow(ctx, "reboot-device", log.Fields{"device-id": device.Id})
if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
- go handler.rebootDevice(ctx, true, device) //reboot request with device checking
- return nil
+ go handler.rebootDevice(log.WithSpanFromContext(context.Background(), ctx), true, device) //reboot request with device checking
+ return &empty.Empty{}, nil
}
logger.Warnw(ctx, "no handler found for device-reboot", log.Fields{"device-id": device.Id})
- return fmt.Errorf("handler-not-found-for-device: %s", device.Id)
+ return nil, fmt.Errorf("handler-not-found-for-device: %s", device.Id)
}
-//Self_test_device unimplemented
-func (oo *OpenONUAC) Self_test_device(ctx context.Context, device *voltha.Device) error {
- return errors.New("unImplemented")
-}
+// DeleteDevice deletes the given device
+func (oo *OpenONUAC) DeleteDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
+ nctx := log.WithSpanFromContext(context.Background(), ctx)
-// Delete_device deletes the given device
-func (oo *OpenONUAC) Delete_device(ctx context.Context, device *voltha.Device) error {
- logger.Infow(ctx, "delete-device", log.Fields{"device-id": device.Id, "SerialNumber": device.SerialNumber})
+ logger.Infow(ctx, "delete-device", log.Fields{"device-id": device.Id, "SerialNumber": device.SerialNumber, "ctx": ctx, "nctx": nctx})
if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
var errorsList []error
@@ -433,162 +340,104 @@
oo.deleteDeviceHandlerToMap(handler)
if len(errorsList) > 0 {
logger.Errorw(ctx, "one-or-more-error-during-device-delete", log.Fields{"device-id": device.Id})
- return fmt.Errorf("one-or-more-error-during-device-delete, errors:%v", errorsList)
+ return nil, fmt.Errorf("one-or-more-error-during-device-delete, errors:%v", errorsList)
}
- return nil
+ return &empty.Empty{}, nil
}
logger.Warnw(ctx, "no handler found for device-deletion", log.Fields{"device-id": device.Id})
- return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+ return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
}
-//Get_device_details unimplemented
-func (oo *OpenONUAC) Get_device_details(ctx context.Context, device *voltha.Device) error {
- return errors.New("unImplemented")
-}
+//UpdateFlowsIncrementally updates (add/remove) the flows on a given device
+func (oo *OpenONUAC) UpdateFlowsIncrementally(ctx context.Context, incrFlows *ic.IncrementalFlows) (*empty.Empty, error) {
+ logger.Infow(ctx, "update-flows-incrementally", log.Fields{"device-id": incrFlows.Device.Id})
-//Update_flows_bulk returns
-func (oo *OpenONUAC) Update_flows_bulk(ctx context.Context, device *voltha.Device, flows *voltha.Flows, groups *voltha.FlowGroups, flowMetadata *voltha.FlowMetadata) error {
- return errors.New("unImplemented")
-}
-
-//Update_flows_incrementally updates (add/remove) the flows on a given device
-func (oo *OpenONUAC) Update_flows_incrementally(ctx context.Context, device *voltha.Device,
- flows *openflow_13.FlowChanges, groups *openflow_13.FlowGroupChanges, flowMetadata *voltha.FlowMetadata) error {
-
- logger.Infow(ctx, "update-flows-incrementally", log.Fields{"device-id": device.Id})
//flow config is relayed to handler even if the device might be in some 'inactive' state
// let the handler or related FSM's decide, what to do with the modified flow state info
// at least the flow-remove must be done in respect to internal data, while OMCI activity might not be needed here
// For now, there is no support for group changes (as in the actual Py-adapter code)
// but processing is continued for flowUpdate possibly also set in the request
- if groups.ToAdd != nil && groups.ToAdd.Items != nil {
- logger.Warnw(ctx, "Update-flow-incr: group add not supported (ignored)", log.Fields{"device-id": device.Id})
+ if incrFlows.Groups.ToAdd != nil && incrFlows.Groups.ToAdd.Items != nil {
+ logger.Warnw(ctx, "Update-flow-incr: group add not supported (ignored)", log.Fields{"device-id": incrFlows.Device.Id})
}
- if groups.ToRemove != nil && groups.ToRemove.Items != nil {
- logger.Warnw(ctx, "Update-flow-incr: group remove not supported (ignored)", log.Fields{"device-id": device.Id})
+ if incrFlows.Groups.ToRemove != nil && incrFlows.Groups.ToRemove.Items != nil {
+ logger.Warnw(ctx, "Update-flow-incr: group remove not supported (ignored)", log.Fields{"device-id": incrFlows.Device.Id})
}
- if groups.ToUpdate != nil && groups.ToUpdate.Items != nil {
- logger.Warnw(ctx, "Update-flow-incr: group update not supported (ignored)", log.Fields{"device-id": device.Id})
+ if incrFlows.Groups.ToUpdate != nil && incrFlows.Groups.ToUpdate.Items != nil {
+ logger.Warnw(ctx, "Update-flow-incr: group update not supported (ignored)", log.Fields{"device-id": incrFlows.Device.Id})
}
- if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
- err := handler.FlowUpdateIncremental(ctx, flows, groups, flowMetadata)
- return err
+ if handler := oo.getDeviceHandler(ctx, incrFlows.Device.Id, false); handler != nil {
+ if err := handler.FlowUpdateIncremental(log.WithSpanFromContext(context.Background(), ctx), incrFlows.Flows, incrFlows.Groups, incrFlows.FlowMetadata); err != nil {
+ return nil, err
+ }
+ return &empty.Empty{}, nil
}
- logger.Warnw(ctx, "no handler found for incremental flow update", log.Fields{"device-id": device.Id})
- return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+ logger.Warnw(ctx, "no handler found for incremental flow update", log.Fields{"device-id": incrFlows.Device.Id})
+ return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", incrFlows.Device.Id))
}
-//Update_pm_config returns PmConfigs nil or error
-func (oo *OpenONUAC) Update_pm_config(ctx context.Context, device *voltha.Device, pmConfigs *voltha.PmConfigs) error {
- logger.Infow(ctx, "update-pm-config", log.Fields{"device-id": device.Id})
- if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
- return handler.updatePmConfig(ctx, pmConfigs)
+//UpdatePmConfig returns PmConfigs nil or error
+func (oo *OpenONUAC) UpdatePmConfig(ctx context.Context, configs *ic.PmConfigsInfo) (*empty.Empty, error) {
+ logger.Infow(ctx, "update-pm-config", log.Fields{"device-id": configs.DeviceId})
+ if handler := oo.getDeviceHandler(ctx, configs.DeviceId, false); handler != nil {
+ if err := handler.updatePmConfig(log.WithSpanFromContext(context.Background(), ctx), configs.PmConfigs); err != nil {
+ return nil, err
+ }
+ return &empty.Empty{}, nil
}
- logger.Warnw(ctx, "no handler found for update-pm-config", log.Fields{"device-id": device.Id})
- return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+ logger.Warnw(ctx, "no handler found for update-pm-config", log.Fields{"device-id": configs.DeviceId})
+ return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", configs.DeviceId))
}
-//Receive_packet_out sends packet out to the device
-func (oo *OpenONUAC) Receive_packet_out(ctx context.Context, deviceID string, egressPortNo int, packet *openflow_13.OfpPacketOut) error {
- return errors.New("unImplemented")
-}
-
-//Suppress_event unimplemented
-func (oo *OpenONUAC) Suppress_event(ctx context.Context, filter *voltha.EventFilter) error {
- return errors.New("unImplemented")
-}
-
-//Unsuppress_event unimplemented
-func (oo *OpenONUAC) Unsuppress_event(ctx context.Context, filter *voltha.EventFilter) error {
- return errors.New("unImplemented")
-}
-
-//Download_image requests downloading some image according to indications as given in request
+//DownloadImage requests downloading some image according to indications as given in request
//The ImageDownload needs to be called `request`due to library reflection requirements
-func (oo *OpenONUAC) Download_image(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
- if request != nil && (*request).Name != "" {
- if !oo.pDownloadManager.imageExists(ctx, request) {
- logger.Debugw(ctx, "start image download", log.Fields{"image-description": request})
+func (oo *OpenONUAC) DownloadImage(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+ ctx = log.WithSpanFromContext(context.Background(), ctx)
+ if imageInfo != nil && imageInfo.Image != nil && imageInfo.Image.Name != "" {
+ if !oo.pDownloadManager.imageExists(ctx, imageInfo.Image) {
+ logger.Debugw(ctx, "start image download", log.Fields{"image-description": imageInfo.Image})
// Download_image is not supposed to be blocking, anyway let's call the DownloadManager still synchronously to detect 'fast' problems
// the download itself is later done in background
- err := oo.pDownloadManager.startDownload(ctx, request)
- return request, err
+ if err := oo.pDownloadManager.startDownload(ctx, imageInfo.Image); err != nil {
+ return nil, err
+ }
+ return imageInfo.Image, nil
}
// image already exists
- logger.Debugw(ctx, "image already downloaded", log.Fields{"image-description": request})
- return request, nil
+ logger.Debugw(ctx, "image already downloaded", log.Fields{"image-description": imageInfo.Image})
+ return imageInfo.Image, nil
}
- return request, errors.New("invalid image definition")
+
+ return nil, errors.New("invalid image definition")
}
-//Get_image_download_status unimplemented
-//The ImageDownload needs to be called `request`due to library reflection requirements
-func (oo *OpenONUAC) Get_image_download_status(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
- return nil, errors.New("unImplemented")
-}
-
-//Cancel_image_download unimplemented
-//The ImageDownload needs to be called `request`due to library reflection requirements
-func (oo *OpenONUAC) Cancel_image_download(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
- return nil, errors.New("unImplemented")
-}
-
-//Activate_image_update requests downloading some Onu Software image to the ONU via OMCI
+//ActivateImageUpdate requests downloading some Onu Software image to the INU via OMCI
// according to indications as given in request and on success activate the image on the ONU
//The ImageDownload needs to be called `request`due to library reflection requirements
-func (oo *OpenONUAC) Activate_image_update(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
- if request != nil && (*request).Name != "" {
- if oo.pDownloadManager.imageLocallyDownloaded(ctx, request) {
- if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
+func (oo *OpenONUAC) ActivateImageUpdate(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+ if imageInfo != nil && imageInfo.Image != nil && imageInfo.Image.Name != "" {
+ if oo.pDownloadManager.imageLocallyDownloaded(ctx, imageInfo.Image) {
+ if handler := oo.getDeviceHandler(ctx, imageInfo.Device.Id, false); handler != nil {
logger.Debugw(ctx, "image download on omci requested", log.Fields{
- "image-description": request, "device-id": device.Id})
- err := handler.doOnuSwUpgrade(ctx, request, oo.pDownloadManager)
- return request, err
+ "image-description": imageInfo.Image, "device-id": imageInfo.Device.Id})
+ if err := handler.doOnuSwUpgrade(ctx, imageInfo.Image, oo.pDownloadManager); err != nil {
+ return nil, err
+ }
+ return imageInfo.Image, nil
}
- logger.Warnw(ctx, "no handler found for image activation", log.Fields{"device-id": device.Id})
- return request, fmt.Errorf(fmt.Sprintf("handler-not-found - device-id: %s", device.Id))
+ logger.Warnw(ctx, "no handler found for image activation", log.Fields{"device-id": imageInfo.Device.Id})
+ return nil, fmt.Errorf(fmt.Sprintf("handler-not-found - device-id: %s", imageInfo.Device.Id))
}
- logger.Debugw(ctx, "image not yet downloaded on activate request", log.Fields{"image-description": request})
- return request, fmt.Errorf(fmt.Sprintf("image-not-yet-downloaded - device-id: %s", device.Id))
+ logger.Debugw(ctx, "image not yet downloaded on activate request", log.Fields{"image-description": imageInfo.Image})
+ return nil, fmt.Errorf(fmt.Sprintf("image-not-yet-downloaded - device-id: %s", imageInfo.Device.Id))
}
- return request, errors.New("invalid image definition")
+ return nil, errors.New("invalid image definition")
}
-//Revert_image_update unimplemented
-func (oo *OpenONUAC) Revert_image_update(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
- return nil, errors.New("unImplemented")
-}
-
-// Enable_port to Enable PON/NNI interface - seems not to be used/required according to python code
-func (oo *OpenONUAC) Enable_port(ctx context.Context, deviceID string, port *voltha.Port) error {
- return errors.New("unImplemented")
-}
-
-// Disable_port to Disable pon/nni interface - seems not to be used/required according to python code
-func (oo *OpenONUAC) Disable_port(ctx context.Context, deviceID string, port *voltha.Port) error {
- return errors.New("unImplemented")
-}
-
-//Child_device_lost - unimplemented
-//needed for if update >= 3.1.x
-func (oo *OpenONUAC) Child_device_lost(ctx context.Context, device *voltha.Device) error {
- return errors.New("unImplemented")
-}
-
-// Start_omci_test unimplemented
-func (oo *OpenONUAC) Start_omci_test(ctx context.Context, device *voltha.Device, request *voltha.OmciTestRequest) (*voltha.TestResponse, error) {
- return nil, errors.New("unImplemented")
-}
-
-// Get_ext_value - unimplemented
-func (oo *OpenONUAC) Get_ext_value(ctx context.Context, deviceID string, device *voltha.Device, valueparam voltha.ValueType_Type) (*voltha.ReturnValues, error) {
- return nil, errors.New("unImplemented")
-}
-
-//Single_get_value_request handles the core request to retrieve uni status
-func (oo *OpenONUAC) Single_get_value_request(ctx context.Context, request extension.SingleGetValueRequest) (*extension.SingleGetValueResponse, error) {
+//GetSingleValue handles the core request to retrieve uni status
+func (oo *OpenONUAC) GetSingleValue(ctx context.Context, request *extension.SingleGetValueRequest) (*extension.SingleGetValueResponse, error) {
logger.Infow(ctx, "Single_get_value_request", log.Fields{"request": request})
if handler := oo.getDeviceHandler(ctx, request.TargetId, false); handler != nil {
@@ -599,7 +448,7 @@
commChan := make(chan Message)
respChan := make(chan extension.SingleGetValueResponse)
// Initiate the self test request
- if err := handler.pSelfTestHdlr.SelfTestRequestStart(ctx, request, commChan, respChan); err != nil {
+ if err := handler.pSelfTestHdlr.SelfTestRequestStart(ctx, *request, commChan, respChan); err != nil {
return &extension.SingleGetValueResponse{
Response: &extension.GetValueResponse{
Status: extension.GetValueResponse_ERROR,
@@ -627,9 +476,9 @@
// The reason for that was never finally investigated.
// To be on the safe side argument names are left here always as defined in iAdapter.go .
-// Download_onu_image downloads (and optionally activates and commits) the indicated ONU image to the requested ONU(s)
+// DownloadOnuImage downloads (and optionally activates and commits) the indicated ONU image to the requested ONU(s)
// if the image is not yet present on the adapter it has to be automatically downloaded
-func (oo *OpenONUAC) Download_onu_image(ctx context.Context, request *voltha.DeviceImageDownloadRequest) (*voltha.DeviceImageResponse, error) {
+func (oo *OpenONUAC) DownloadOnuImage(ctx context.Context, request *voltha.DeviceImageDownloadRequest) (*voltha.DeviceImageResponse, error) {
if request != nil && len((*request).DeviceId) > 0 && (*request).Image.Version != "" {
loResponse := voltha.DeviceImageResponse{}
imageIdentifier := (*request).Image.Version
@@ -645,8 +494,18 @@
loDeviceImageState.ImageState = &loImageState
loDeviceImageState.ImageState.Version = (*request).Image.Version
- onuVolthaDevice, err := oo.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
- loDeviceID, loDeviceID)
+ handler := oo.getDeviceHandler(ctx, loDeviceID, false)
+ if handler == nil {
+ //cannot start ONU download for requested device
+ logger.Warnw(ctx, "no handler found for image activation", log.Fields{"device-id": loDeviceID})
+ loDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_FAILED
+ loDeviceImageState.ImageState.Reason = voltha.ImageState_UNKNOWN_ERROR
+ loDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
+ loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, &loDeviceImageState)
+ continue
+ }
+
+ onuVolthaDevice, err := handler.getDeviceFromCore(ctx, loDeviceID)
if err != nil || onuVolthaDevice == nil {
logger.Warnw(ctx, "Failed to fetch Onu device for image download",
log.Fields{"device-id": loDeviceID, "err": err})
@@ -718,6 +577,17 @@
loDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
}
}
+
+ // start the ONU download activity for each possible device
+ logger.Debugw(ctx, "image download on omci requested", log.Fields{
+ "image-id": imageIdentifier, "device-id": loDeviceID})
+ //onu upgrade handling called in background without immediate error evaluation here
+ // as the processing can be done for multiple ONU's and an error on one ONU should not stop processing for others
+ // state/progress/success of the request has to be verified using the Get_onu_image_status() API
+ go handler.onuSwUpgradeAfterDownload(ctx, request, oo.pFileManager, imageIdentifier)
+ loDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_STARTED
+ loDeviceImageState.ImageState.Reason = voltha.ImageState_NO_ERROR
+ loDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, &loDeviceImageState)
}
pImageResp := &loResponse
@@ -726,9 +596,9 @@
return nil, errors.New("invalid image download parameters")
}
-// Get_onu_image_status delivers the adapter-related information about the download/activation/commitment
+// GetOnuImageStatus delivers the adapter-related information about the download/activation/commitment
// status for the requested image
-func (oo *OpenONUAC) Get_onu_image_status(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+func (oo *OpenONUAC) GetOnuImageStatus(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
if in != nil && len((*in).DeviceId) > 0 && (*in).Version != "" {
loResponse := voltha.DeviceImageResponse{}
imageIdentifier := (*in).Version
@@ -737,12 +607,20 @@
var vendorID string
for _, pCommonID := range (*in).DeviceId {
loDeviceID := (*pCommonID).Id
- pDeviceImageState := &voltha.DeviceImageState{
- DeviceId: loDeviceID,
+ pDeviceImageState := &voltha.DeviceImageState{DeviceId: loDeviceID}
+ handler := oo.getDeviceHandler(ctx, loDeviceID, false)
+ if handler == nil {
+ //cannot get the handler
+ logger.Warnw(ctx, "no handler found for image status request ", log.Fields{"device-id": loDeviceID})
+ pDeviceImageState.DeviceId = loDeviceID
+ pDeviceImageState.ImageState.Version = (*in).Version
+ pDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_FAILED
+ pDeviceImageState.ImageState.Reason = voltha.ImageState_UNKNOWN_ERROR
+ pDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
+ loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, pDeviceImageState)
+ continue
}
- vendorIDSet = false
- onuVolthaDevice, err := oo.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
- loDeviceID, loDeviceID)
+ onuVolthaDevice, err := handler.getDeviceFromCore(ctx, loDeviceID)
if err != nil || onuVolthaDevice == nil {
logger.Warnw(ctx, "Failed to fetch Onu device to get image status",
log.Fields{"device-id": loDeviceID, "err": err})
@@ -780,23 +658,10 @@
}
pDeviceImageState.ImageState = pImageState
} else {
- // get the handler for the device
- if handler := oo.getDeviceHandler(ctx, loDeviceID, false); handler != nil {
- logger.Debugw(ctx, "image status request for", log.Fields{
- "image-id": imageIdentifier, "device-id": loDeviceID})
- //status request is called synchronously to collect the indications for all concerned devices
- pDeviceImageState.ImageState = handler.requestOnuSwUpgradeState(ctx, imageIdentifier, (*in).Version)
- } else {
- //cannot get the handler
- logger.Warnw(ctx, "no handler found for image status request ", log.Fields{"device-id": loDeviceID})
- pImageState := &voltha.ImageState{
- Version: (*in).Version,
- DownloadState: voltha.ImageState_DOWNLOAD_UNKNOWN, //no statement about last activity possible
- Reason: voltha.ImageState_UNKNOWN_ERROR, //something like "DEVICE_NOT_EXISTS" would be better (proto def)
- ImageState: voltha.ImageState_IMAGE_UNKNOWN,
- }
- pDeviceImageState.ImageState = pImageState
- }
+ logger.Debugw(ctx, "image status request for", log.Fields{
+ "image-id": imageIdentifier, "device-id": loDeviceID})
+ //status request is called synchronously to collect the indications for all concerned devices
+ pDeviceImageState.ImageState = handler.requestOnuSwUpgradeState(ctx, imageIdentifier, (*in).Version)
}
}
loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, pDeviceImageState)
@@ -807,8 +672,8 @@
return nil, errors.New("invalid image status request parameters")
}
-// Abort_onu_image_upgrade stops the actual download/activation/commitment process (on next possibly step)
-func (oo *OpenONUAC) Abort_onu_image_upgrade(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+// AbortOnuImageUpgrade stops the actual download/activation/commitment process (on next possibly step)
+func (oo *OpenONUAC) AbortOnuImageUpgrade(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
if in != nil && len((*in).DeviceId) > 0 && (*in).Version != "" {
loResponse := voltha.DeviceImageResponse{}
imageIdentifier := (*in).Version
@@ -816,16 +681,30 @@
var vendorID string
for _, pCommonID := range (*in).DeviceId {
loDeviceID := (*pCommonID).Id
- onuVolthaDevice, err := oo.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
- loDeviceID, loDeviceID)
+ pDeviceImageState := &voltha.DeviceImageState{}
+ loImageState := voltha.ImageState{}
+ pDeviceImageState.ImageState = &loImageState
+
+ handler := oo.getDeviceHandler(ctx, loDeviceID, false)
+ if handler == nil {
+ //cannot start ONU download for requested device
+ logger.Warnw(ctx, "no handler found for aborting upgrade ", log.Fields{"device-id": loDeviceID})
+ pDeviceImageState.DeviceId = loDeviceID
+ pDeviceImageState.ImageState.Version = (*in).Version
+ //nolint:misspell
+ pDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_CANCELLED
+ //nolint:misspell
+ pDeviceImageState.ImageState.Reason = voltha.ImageState_CANCELLED_ON_REQUEST
+ pDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
+ loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, pDeviceImageState)
+ continue
+ }
+ onuVolthaDevice, err := handler.getDeviceFromCore(ctx, loDeviceID)
if err != nil || onuVolthaDevice == nil {
logger.Warnw(ctx, "Failed to fetch Onu device to abort its download",
log.Fields{"device-id": loDeviceID, "err": err})
continue //try the work with next deviceId
}
- pDeviceImageState := &voltha.DeviceImageState{}
- loImageState := voltha.ImageState{}
- pDeviceImageState.ImageState = &loImageState
if firstDevice {
//start/verify download of the image to the adapter based on first found device only
@@ -844,20 +723,10 @@
}
// cancel the ONU upgrade activity for each possible device
- if handler := oo.getDeviceHandler(ctx, loDeviceID, false); handler != nil {
- logger.Debugw(ctx, "image upgrade abort requested", log.Fields{
- "image-id": imageIdentifier, "device-id": loDeviceID})
- //upgrade cancel is called synchronously to collect the imageResponse indications for all concerned devices
- handler.cancelOnuSwUpgrade(ctx, imageIdentifier, (*in).Version, pDeviceImageState)
- } else {
- //cannot start aborting ONU download for requested device
- logger.Warnw(ctx, "no handler found for aborting upgrade ", log.Fields{"device-id": loDeviceID})
- pDeviceImageState.DeviceId = loDeviceID
- pDeviceImageState.ImageState.Version = (*in).Version
- pDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_CANCELLED
- pDeviceImageState.ImageState.Reason = voltha.ImageState_CANCELLED_ON_REQUEST //something better would be possible with protos modification
- pDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
- }
+ logger.Debugw(ctx, "image upgrade abort requested", log.Fields{
+ "image-id": imageIdentifier, "device-id": loDeviceID})
+ //upgrade cancel is called synchronously to collect the imageResponse indications for all concerned devices
+ handler.cancelOnuSwUpgrade(ctx, imageIdentifier, (*in).Version, pDeviceImageState)
loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, pDeviceImageState)
}
if !firstDevice {
@@ -871,23 +740,23 @@
return nil, errors.New("invalid image upgrade abort parameters")
}
-// Get_onu_images retrieves the ONU SW image status information via OMCI
-func (oo *OpenONUAC) Get_onu_images(ctx context.Context, deviceID string) (*voltha.OnuImages, error) {
- logger.Infow(ctx, "Get_onu_images", log.Fields{"device-id": deviceID})
- if handler := oo.getDeviceHandler(ctx, deviceID, false); handler != nil {
+// GetOnuImages retrieves the ONU SW image status information via OMCI
+func (oo *OpenONUAC) GetOnuImages(ctx context.Context, id *common.ID) (*voltha.OnuImages, error) {
+ logger.Infow(ctx, "Get_onu_images", log.Fields{"device-id": id.Id})
+ if handler := oo.getDeviceHandler(ctx, id.Id, false); handler != nil {
images, err := handler.getOnuImages(ctx)
if err == nil {
return images, nil
}
- return nil, fmt.Errorf(fmt.Sprintf("%s-%s", err, deviceID))
+ return nil, fmt.Errorf(fmt.Sprintf("%s-%s", err, id.Id))
}
- logger.Warnw(ctx, "no handler found for Get_onu_images", log.Fields{"device-id": deviceID})
- return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", deviceID))
+ logger.Warnw(ctx, "no handler found for Get_onu_images", log.Fields{"device-id": id.Id})
+ return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", id.Id))
}
-// Activate_onu_image initiates the activation of the image for the requested ONU(s)
+// ActivateOnuImage initiates the activation of the image for the requested ONU(s)
// precondition: image downloaded and not yet activated or image refers to current inactive image
-func (oo *OpenONUAC) Activate_onu_image(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+func (oo *OpenONUAC) ActivateOnuImage(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
if in != nil && len((*in).DeviceId) > 0 && (*in).Version != "" {
loResponse := voltha.DeviceImageResponse{}
imageIdentifier := (*in).Version
@@ -933,9 +802,9 @@
return nil, errors.New("invalid image activation parameters")
}
-// Commit_onu_image enforces the commitment of the image for the requested ONU(s)
+// CommitOnuImage enforces the commitment of the image for the requested ONU(s)
// precondition: image activated and not yet committed
-func (oo *OpenONUAC) Commit_onu_image(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+func (oo *OpenONUAC) CommitOnuImage(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
if in != nil && len((*in).DeviceId) > 0 && (*in).Version != "" {
loResponse := voltha.DeviceImageResponse{}
imageIdentifier := (*in).Version
@@ -983,3 +852,281 @@
// Adapter interface required methods ################ end #########
// #################################################################
+
+/*
+ *
+ * ONU inter adapter service
+ *
+ */
+
+// OnuIndication is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) OnuIndication(ctx context.Context, onuInd *ic.OnuIndicationMessage) (*empty.Empty, error) {
+ logger.Debugw(ctx, "onu-indication", log.Fields{"onu-indication": onuInd})
+
+ if onuInd == nil || onuInd.OnuIndication == nil {
+ return nil, fmt.Errorf("invalid-onu-indication-%v", onuInd)
+ }
+
+ onuIndication := onuInd.OnuIndication
+ onuOperstate := onuIndication.GetOperState()
+ waitForDhInstPresent := false
+ if onuOperstate == "up" {
+ //Race condition (relevant in BBSIM-environment only): Due to unsynchronized processing of olt-adapter and rw_core,
+ //ONU_IND_REQUEST msg by olt-adapter could arrive a little bit earlier than rw_core was able to announce the corresponding
+ //ONU by RPC of Adopt_device(). Therefore it could be necessary to wait with processing of ONU_IND_REQUEST until call of
+ //Adopt_device() arrived and DeviceHandler instance was created
+ waitForDhInstPresent = true
+ }
+ if handler := oo.getDeviceHandler(ctx, onuInd.DeviceId, waitForDhInstPresent); handler != nil {
+ logger.Infow(ctx, "onu-ind-request", log.Fields{"device-id": onuInd.DeviceId,
+ "OnuId": onuIndication.GetOnuId(),
+ "AdminState": onuIndication.GetAdminState(), "OperState": onuOperstate,
+ "SNR": onuIndication.GetSerialNumber()})
+
+ if onuOperstate == "up" {
+ if err := handler.createInterface(ctx, onuIndication); err != nil {
+ return nil, err
+ }
+ return &empty.Empty{}, nil
+ } else if (onuOperstate == "down") || (onuOperstate == "unreachable") {
+ return nil, handler.updateInterface(ctx, onuIndication)
+ } else {
+ logger.Errorw(ctx, "unknown-onu-ind-request operState", log.Fields{"OnuId": onuIndication.GetOnuId()})
+ return nil, fmt.Errorf("invalidOperState: %s, %s", onuOperstate, onuInd.DeviceId)
+ }
+ }
+ logger.Warnw(ctx, "no handler found for received onu-ind-request", log.Fields{
+ "msgToDeviceId": onuInd.DeviceId})
+ return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", onuInd.DeviceId))
+}
+
+// OmciIndication is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) OmciIndication(ctx context.Context, msg *ic.OmciMessage) (*empty.Empty, error) {
+ logger.Debugw(ctx, "omci-response", log.Fields{"parent-device-id": msg.ParentDeviceId, "child-device-id": msg.ChildDeviceId})
+
+ if handler := oo.getDeviceHandler(ctx, msg.ChildDeviceId, false); handler != nil {
+ if err := handler.handleOMCIIndication(log.WithSpanFromContext(context.Background(), ctx), msg); err != nil {
+ return nil, err
+ }
+ return &empty.Empty{}, nil
+ }
+ return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", msg.ChildDeviceId))
+}
+
+// DownloadTechProfile is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) DownloadTechProfile(ctx context.Context, tProfile *ic.TechProfileDownloadMessage) (*empty.Empty, error) {
+ logger.Debugw(ctx, "download-tech-profile", log.Fields{"uni-id": tProfile.UniId})
+
+ if handler := oo.getDeviceHandler(ctx, tProfile.DeviceId, false); handler != nil {
+ if err := handler.handleTechProfileDownloadRequest(log.WithSpanFromContext(context.Background(), ctx), tProfile); err != nil {
+ return nil, err
+ }
+ return &empty.Empty{}, nil
+ }
+ return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", tProfile.DeviceId))
+}
+
+// DeleteGemPort is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) DeleteGemPort(ctx context.Context, gPort *ic.DeleteGemPortMessage) (*empty.Empty, error) {
+ logger.Debugw(ctx, "delete-gem-port", log.Fields{"device-id": gPort.DeviceId, "uni-id": gPort.UniId})
+
+ if handler := oo.getDeviceHandler(ctx, gPort.DeviceId, false); handler != nil {
+ if err := handler.handleDeleteGemPortRequest(log.WithSpanFromContext(context.Background(), ctx), gPort); err != nil {
+ return nil, err
+ }
+ return &empty.Empty{}, nil
+ }
+ return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", gPort.DeviceId))
+}
+
+// DeleteTCont is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) DeleteTCont(ctx context.Context, tConf *ic.DeleteTcontMessage) (*empty.Empty, error) {
+ logger.Debugw(ctx, "delete-tcont", log.Fields{"tconf": tConf})
+
+ if handler := oo.getDeviceHandler(ctx, tConf.DeviceId, false); handler != nil {
+ if err := handler.handleDeleteTcontRequest(log.WithSpanFromContext(context.Background(), ctx), tConf); err != nil {
+ return nil, err
+ }
+ return &empty.Empty{}, nil
+ }
+ return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", tConf.DeviceId))
+}
+
+/*
+ * Parent GRPC clients
+ */
+
+func (oo *OpenONUAC) setupParentInterAdapterClient(ctx context.Context, endpoint string) error {
+ logger.Infow(ctx, "setting-parent-adapter-connection", log.Fields{"parent-endpoint": endpoint})
+ oo.lockParentAdapterClients.Lock()
+ defer oo.lockParentAdapterClients.Unlock()
+ if _, ok := oo.parentAdapterClients[endpoint]; ok {
+ return nil
+ }
+
+ childClient, err := vgrpc.NewClient(endpoint,
+ oo.oltAdapterRestarted,
+ vgrpc.ActivityCheck(true))
+
+ if err != nil {
+ return err
+ }
+
+ oo.parentAdapterClients[endpoint] = childClient
+
+ go oo.parentAdapterClients[endpoint].Start(log.WithSpanFromContext(context.TODO(), ctx), setAndTestAdapterServiceHandler)
+
+ // Wait until we have a connection to the child adapter.
+ // Unlimited retries or until context expires
+ subCtx := log.WithSpanFromContext(context.TODO(), ctx)
+ backoff := vgrpc.NewBackoff(oo.config.MinBackoffRetryDelay, oo.config.MaxBackoffRetryDelay, 0)
+ for {
+ client, err := oo.parentAdapterClients[endpoint].GetOltInterAdapterServiceClient()
+ if err == nil && client != nil {
+ logger.Infow(subCtx, "connected-to-parent-adapter", log.Fields{"parent-endpoint": endpoint})
+ break
+ }
+ logger.Warnw(subCtx, "connection-to-parent-adapter-not-ready", log.Fields{"error": err, "parent-endpoint": endpoint})
+ // Backoff
+ if err = backoff.Backoff(subCtx); err != nil {
+ logger.Errorw(subCtx, "received-error-on-backoff", log.Fields{"error": err, "parent-endpoint": endpoint})
+ break
+ }
+ }
+ return nil
+}
+
+func (oo *OpenONUAC) getParentAdapterServiceClient(endpoint string) (adapter_services.OltInterAdapterServiceClient, error) {
+ // First check from cache
+ oo.lockParentAdapterClients.RLock()
+ if pgClient, ok := oo.parentAdapterClients[endpoint]; ok {
+ oo.lockParentAdapterClients.RUnlock()
+ return pgClient.GetOltInterAdapterServiceClient()
+ }
+ oo.lockParentAdapterClients.RUnlock()
+
+ // Set the parent connection - can occur on restarts
+ ctx, cancel := context.WithTimeout(context.Background(), oo.config.RPCTimeout)
+ err := oo.setupParentInterAdapterClient(ctx, endpoint)
+ cancel()
+ if err != nil {
+ return nil, err
+ }
+
+ // Get the parent client now
+ oo.lockParentAdapterClients.RLock()
+ defer oo.lockParentAdapterClients.RUnlock()
+ if pgClient, ok := oo.parentAdapterClients[endpoint]; ok {
+ return pgClient.GetOltInterAdapterServiceClient()
+ }
+
+ return nil, fmt.Errorf("no-client-for-endpoint-%s", endpoint)
+}
+
+// TODO: Any action the adapter needs to do following an olt adapter restart?
+func (oo *OpenONUAC) oltAdapterRestarted(ctx context.Context, endPoint string) error {
+ logger.Errorw(ctx, "olt-adapter-restarted", log.Fields{"endpoint": endPoint})
+ return nil
+}
+
+// setAndTestAdapterServiceHandler is used to test whether the remote gRPC service is up
+func setAndTestAdapterServiceHandler(ctx context.Context, conn *grpc.ClientConn) interface{} {
+ svc := adapter_services.NewOltInterAdapterServiceClient(conn)
+ if h, err := svc.GetHealthStatus(ctx, &empty.Empty{}); err != nil || h.State != voltha.HealthStatus_HEALTHY {
+ return nil
+ }
+ return svc
+}
+
+/*
+ *
+ * Unimplemented APIs
+ *
+ */
+
+//GetOfpDeviceInfo returns OFP information for the given device. Method not implemented as per [VOL-3202].
+// OF port info is now to be delivered within UniPort create cmp changes in onu_uni_port.go::CreateVolthaPort()
+//
+func (oo *OpenONUAC) GetOfpDeviceInfo(ctx context.Context, device *voltha.Device) (*ic.SwitchCapability, error) {
+ return nil, errors.New("unImplemented")
+}
+
+//SimulateAlarm is unimplemented
+func (oo *OpenONUAC) SimulateAlarm(context.Context, *ic.SimulateAlarmMessage) (*common.OperationResp, error) {
+ return nil, errors.New("unImplemented")
+}
+
+//SetExtValue is unimplemented
+func (oo *OpenONUAC) SetExtValue(context.Context, *ic.SetExtValueMessage) (*empty.Empty, error) {
+ return nil, errors.New("unImplemented")
+}
+
+//SetSingleValue is unimplemented
+func (oo *OpenONUAC) SetSingleValue(context.Context, *extension.SingleSetValueRequest) (*extension.SingleSetValueResponse, error) {
+ return nil, errors.New("unImplemented")
+}
+
+//StartOmciTest not implemented
+func (oo *OpenONUAC) StartOmciTest(ctx context.Context, test *ic.OMCITest) (*voltha.TestResponse, error) {
+ return nil, errors.New("unImplemented")
+}
+
+//SuppressEvent unimplemented
+func (oo *OpenONUAC) SuppressEvent(ctx context.Context, filter *voltha.EventFilter) (*empty.Empty, error) {
+ return nil, errors.New("unImplemented")
+}
+
+//UnSuppressEvent unimplemented
+func (oo *OpenONUAC) UnSuppressEvent(ctx context.Context, filter *voltha.EventFilter) (*empty.Empty, error) {
+ return nil, errors.New("unImplemented")
+}
+
+//GetImageDownloadStatus is unimplemented
+func (oo *OpenONUAC) GetImageDownloadStatus(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+ return nil, errors.New("unImplemented")
+}
+
+//CancelImageDownload is unimplemented
+func (oo *OpenONUAC) CancelImageDownload(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+ return nil, errors.New("unImplemented")
+}
+
+//RevertImageUpdate is unimplemented
+func (oo *OpenONUAC) RevertImageUpdate(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+ return nil, errors.New("unImplemented")
+}
+
+// UpdateFlowsBulk is unimplemented
+func (oo *OpenONUAC) UpdateFlowsBulk(ctx context.Context, flows *ic.BulkFlows) (*empty.Empty, error) {
+ return nil, errors.New("unImplemented")
+}
+
+//SelfTestDevice unimplented
+func (oo *OpenONUAC) SelfTestDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
+ return nil, errors.New("unImplemented")
+}
+
+//SendPacketOut sends packet out to the device
+func (oo *OpenONUAC) SendPacketOut(ctx context.Context, packet *ic.PacketOut) (*empty.Empty, error) {
+ return nil, errors.New("unImplemented")
+}
+
+// EnablePort to Enable PON/NNI interface - seems not to be used/required according to python code
+func (oo *OpenONUAC) EnablePort(ctx context.Context, port *voltha.Port) (*empty.Empty, error) {
+ return nil, errors.New("unImplemented")
+}
+
+// DisablePort to Disable pon/nni interface - seems not to be used/required according to python code
+func (oo *OpenONUAC) DisablePort(ctx context.Context, port *voltha.Port) (*empty.Empty, error) {
+ return nil, errors.New("unImplemented")
+}
+
+// GetExtValue - unimplemented
+func (oo *OpenONUAC) GetExtValue(ctx context.Context, extInfo *ic.GetExtValueMessage) (*voltha.ReturnValues, error) {
+ return nil, errors.New("unImplemented")
+}
+
+// ChildDeviceLost - unimplemented
+func (oo *OpenONUAC) ChildDeviceLost(ctx context.Context, childDevice *voltha.Device) (*empty.Empty, error) {
+ return nil, errors.New("unImplemented")
+}
diff --git a/internal/pkg/onuadaptercore/openonuimpl.go b/internal/pkg/onuadaptercore/openonuimpl.go
index 8e6e187..efa4e96 100644
--- a/internal/pkg/onuadaptercore/openonuimpl.go
+++ b/internal/pkg/onuadaptercore/openonuimpl.go
@@ -20,7 +20,6 @@
import (
"context"
"errors"
- //"github.com/opencord/voltha-lib-go/v5/pkg/log"
)
/*
diff --git a/internal/pkg/onuadaptercore/uniportadmin.go b/internal/pkg/onuadaptercore/uniportadmin.go
index 0c22500..2ceea25 100644
--- a/internal/pkg/onuadaptercore/uniportadmin.go
+++ b/internal/pkg/onuadaptercore/uniportadmin.go
@@ -27,9 +27,7 @@
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- //ic "github.com/opencord/voltha-protos/v4/go/inter_container"
- //"github.com/opencord/voltha-protos/v4/go/openflow_13"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
)
//lockStateFsm defines the structure for the state machine to lock/unlock the ONU UNI ports via OMCI
diff --git a/internal/pkg/onuadaptercore/uniportstatus.go b/internal/pkg/onuadaptercore/uniportstatus.go
index fe9af4c..f7cfe4b 100644
--- a/internal/pkg/onuadaptercore/uniportstatus.go
+++ b/internal/pkg/onuadaptercore/uniportstatus.go
@@ -19,11 +19,12 @@
import (
"context"
+ "time"
+
"github.com/opencord/omci-lib-go"
me "github.com/opencord/omci-lib-go/generated"
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
- "github.com/opencord/voltha-protos/v4/go/extension"
- "time"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
+ "github.com/opencord/voltha-protos/v5/go/extension"
)
const (
diff --git a/pkg/mocks/common.go b/pkg/mocks/common.go
index 72a4c30..be78c34 100644
--- a/pkg/mocks/common.go
+++ b/pkg/mocks/common.go
@@ -18,7 +18,7 @@
package mocks
import (
- "github.com/opencord/voltha-lib-go/v5/pkg/log"
+ "github.com/opencord/voltha-lib-go/v7/pkg/log"
)
func init() {
diff --git a/vendor/github.com/DataDog/zstd/.travis.yml b/vendor/github.com/DataDog/zstd/.travis.yml
deleted file mode 100644
index 629470c..0000000
--- a/vendor/github.com/DataDog/zstd/.travis.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-dist: xenial
-language: go
-
-go:
- - 1.10.x
- - 1.11.x
- - 1.12.x
-
-os:
- - linux
- - osx
-
-matrix:
- include:
- name: "Go 1.11.x CentOS 32bits"
- language: go
- go: 1.11.x
- os: linux
- services:
- - docker
- script:
- # Please update Go version in travis_test_32 as needed
- - "docker run -i -v \"${PWD}:/zstd\" toopher/centos-i386:centos6 /bin/bash -c \"linux32 --32bit i386 /zstd/travis_test_32.sh\""
-
-install:
- - "wget https://github.com/DataDog/zstd/files/2246767/mr.zip"
- - "unzip mr.zip"
-script:
- - "go build"
- - "PAYLOAD=`pwd`/mr go test -v"
- - "PAYLOAD=`pwd`/mr go test -bench ."
diff --git a/vendor/github.com/DataDog/zstd/LICENSE b/vendor/github.com/DataDog/zstd/LICENSE
deleted file mode 100644
index 345c1eb..0000000
--- a/vendor/github.com/DataDog/zstd/LICENSE
+++ /dev/null
@@ -1,27 +0,0 @@
-Simplified BSD License
-
-Copyright (c) 2016, Datadog <info@datadoghq.com>
-All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are met:
-
- * Redistributions of source code must retain the above copyright notice,
- this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above copyright notice,
- this list of conditions and the following disclaimer in the documentation
- and/or other materials provided with the distribution.
- * Neither the name of the copyright holder nor the names of its contributors
- may be used to endorse or promote products derived from this software
- without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
-AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
-IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
-DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE
-FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
-DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
-SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
-CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
-OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/vendor/github.com/DataDog/zstd/README.md b/vendor/github.com/DataDog/zstd/README.md
deleted file mode 100644
index b32c3e7..0000000
--- a/vendor/github.com/DataDog/zstd/README.md
+++ /dev/null
@@ -1,120 +0,0 @@
-# Zstd Go Wrapper
-
-[C Zstd Homepage](https://github.com/Cyan4973/zstd)
-
-The current headers and C files are from *v1.4.1* (Commit
-[52181f8](https://github.com/facebook/zstd/releases/tag/v1.4.1)).
-
-## Usage
-
-There are two main APIs:
-
-* simple Compress/Decompress
-* streaming API (io.Reader/io.Writer)
-
-The compress/decompress APIs mirror that of lz4, while the streaming API was
-designed to be a drop-in replacement for zlib.
-
-### Simple `Compress/Decompress`
-
-
-```go
-// Compress compresses the byte array given in src and writes it to dst.
-// If you already have a buffer allocated, you can pass it to prevent allocation
-// If not, you can pass nil as dst.
-// If the buffer is too small, it will be reallocated, resized, and returned bu the function
-// If dst is nil, this will allocate the worst case size (CompressBound(src))
-Compress(dst, src []byte) ([]byte, error)
-```
-
-```go
-// CompressLevel is the same as Compress but you can pass another compression level
-CompressLevel(dst, src []byte, level int) ([]byte, error)
-```
-
-```go
-// Decompress will decompress your payload into dst.
-// If you already have a buffer allocated, you can pass it to prevent allocation
-// If not, you can pass nil as dst (allocates a 4*src size as default).
-// If the buffer is too small, it will retry 3 times by doubling the dst size
-// After max retries, it will switch to the slower stream API to be sure to be able
-// to decompress. Currently switches if compression ratio > 4*2**3=32.
-Decompress(dst, src []byte) ([]byte, error)
-```
-
-### Stream API
-
-```go
-// NewWriter creates a new object that can optionally be initialized with
-// a precomputed dictionary. If dict is nil, compress without a dictionary.
-// The dictionary array should not be changed during the use of this object.
-// You MUST CALL Close() to write the last bytes of a zstd stream and free C objects.
-NewWriter(w io.Writer) *Writer
-NewWriterLevel(w io.Writer, level int) *Writer
-NewWriterLevelDict(w io.Writer, level int, dict []byte) *Writer
-
-// Write compresses the input data and write it to the underlying writer
-(w *Writer) Write(p []byte) (int, error)
-
-// Close flushes the buffer and frees C zstd objects
-(w *Writer) Close() error
-```
-
-```go
-// NewReader returns a new io.ReadCloser that will decompress data from the
-// underlying reader. If a dictionary is provided to NewReaderDict, it must
-// not be modified until Close is called. It is the caller's responsibility
-// to call Close, which frees up C objects.
-NewReader(r io.Reader) io.ReadCloser
-NewReaderDict(r io.Reader, dict []byte) io.ReadCloser
-```
-
-### Benchmarks (benchmarked with v0.5.0)
-
-The author of Zstd also wrote lz4. Zstd is intended to occupy a speed/ratio
-level similar to what zlib currently provides. In our tests, the can always
-be made to be better than zlib by chosing an appropriate level while still
-keeping compression and decompression time faster than zlib.
-
-You can run the benchmarks against your own payloads by using the Go benchmarks tool.
-Just export your payload filepath as the `PAYLOAD` environment variable and run the benchmarks:
-
-```go
-go test -bench .
-```
-
-Compression of a 7Mb pdf zstd (this wrapper) vs [czlib](https://github.com/DataDog/czlib):
-```
-BenchmarkCompression 5 221056624 ns/op 67.34 MB/s
-BenchmarkDecompression 100 18370416 ns/op 810.32 MB/s
-
-BenchmarkFzlibCompress 2 610156603 ns/op 24.40 MB/s
-BenchmarkFzlibDecompress 20 81195246 ns/op 183.33 MB/s
-```
-
-Ratio is also better by a margin of ~20%.
-Compression speed is always better than zlib on all the payloads we tested;
-However, [czlib](https://github.com/DataDog/czlib) has optimisations that make it
-faster at decompressiong small payloads:
-
-```
-Testing with size: 11... czlib: 8.97 MB/s, zstd: 3.26 MB/s
-Testing with size: 27... czlib: 23.3 MB/s, zstd: 8.22 MB/s
-Testing with size: 62... czlib: 31.6 MB/s, zstd: 19.49 MB/s
-Testing with size: 141... czlib: 74.54 MB/s, zstd: 42.55 MB/s
-Testing with size: 323... czlib: 155.14 MB/s, zstd: 99.39 MB/s
-Testing with size: 739... czlib: 235.9 MB/s, zstd: 216.45 MB/s
-Testing with size: 1689... czlib: 116.45 MB/s, zstd: 345.64 MB/s
-Testing with size: 3858... czlib: 176.39 MB/s, zstd: 617.56 MB/s
-Testing with size: 8811... czlib: 254.11 MB/s, zstd: 824.34 MB/s
-Testing with size: 20121... czlib: 197.43 MB/s, zstd: 1339.11 MB/s
-Testing with size: 45951... czlib: 201.62 MB/s, zstd: 1951.57 MB/s
-```
-
-zstd starts to shine with payloads > 1KB
-
-### Stability - Current state: STABLE
-
-The C library seems to be pretty stable and according to the author has been tested and fuzzed.
-
-For the Go wrapper, the test cover most usual cases and we have succesfully tested it on all staging and prod data.
diff --git a/vendor/github.com/DataDog/zstd/ZSTD_LICENSE b/vendor/github.com/DataDog/zstd/ZSTD_LICENSE
deleted file mode 100644
index a793a80..0000000
--- a/vendor/github.com/DataDog/zstd/ZSTD_LICENSE
+++ /dev/null
@@ -1,30 +0,0 @@
-BSD License
-
-For Zstandard software
-
-Copyright (c) 2016-present, Facebook, Inc. All rights reserved.
-
-Redistribution and use in source and binary forms, with or without modification,
-are permitted provided that the following conditions are met:
-
- * Redistributions of source code must retain the above copyright notice, this
- list of conditions and the following disclaimer.
-
- * Redistributions in binary form must reproduce the above copyright notice,
- this list of conditions and the following disclaimer in the documentation
- and/or other materials provided with the distribution.
-
- * Neither the name Facebook nor the names of its contributors may be used to
- endorse or promote products derived from this software without specific
- prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
-ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
-WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
-DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
-ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
-(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
-LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
-ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
-SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/vendor/github.com/DataDog/zstd/bitstream.h b/vendor/github.com/DataDog/zstd/bitstream.h
deleted file mode 100644
index d955bd6..0000000
--- a/vendor/github.com/DataDog/zstd/bitstream.h
+++ /dev/null
@@ -1,455 +0,0 @@
-/* ******************************************************************
- bitstream
- Part of FSE library
- Copyright (C) 2013-present, Yann Collet.
-
- BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
- * Redistributions of source code must retain the above copyright
- notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
- copyright notice, this list of conditions and the following disclaimer
- in the documentation and/or other materials provided with the
- distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
- "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
- LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
- A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
- OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
- OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
- You can contact the author at :
- - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-#ifndef BITSTREAM_H_MODULE
-#define BITSTREAM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/*
-* This API consists of small unitary functions, which must be inlined for best performance.
-* Since link-time-optimization is not available for all compilers,
-* these functions are defined into a .h to be included.
-*/
-
-/*-****************************************
-* Dependencies
-******************************************/
-#include "mem.h" /* unaligned access routines */
-#include "debug.h" /* assert(), DEBUGLOG(), RAWLOG() */
-#include "error_private.h" /* error codes and messages */
-
-
-/*=========================================
-* Target specific
-=========================================*/
-#if defined(__BMI__) && defined(__GNUC__)
-# include <immintrin.h> /* support for bextr (experimental) */
-#endif
-
-#define STREAM_ACCUMULATOR_MIN_32 25
-#define STREAM_ACCUMULATOR_MIN_64 57
-#define STREAM_ACCUMULATOR_MIN ((U32)(MEM_32bits() ? STREAM_ACCUMULATOR_MIN_32 : STREAM_ACCUMULATOR_MIN_64))
-
-
-/*-******************************************
-* bitStream encoding API (write forward)
-********************************************/
-/* bitStream can mix input from multiple sources.
- * A critical property of these streams is that they encode and decode in **reverse** direction.
- * So the first bit sequence you add will be the last to be read, like a LIFO stack.
- */
-typedef struct {
- size_t bitContainer;
- unsigned bitPos;
- char* startPtr;
- char* ptr;
- char* endPtr;
-} BIT_CStream_t;
-
-MEM_STATIC size_t BIT_initCStream(BIT_CStream_t* bitC, void* dstBuffer, size_t dstCapacity);
-MEM_STATIC void BIT_addBits(BIT_CStream_t* bitC, size_t value, unsigned nbBits);
-MEM_STATIC void BIT_flushBits(BIT_CStream_t* bitC);
-MEM_STATIC size_t BIT_closeCStream(BIT_CStream_t* bitC);
-
-/* Start with initCStream, providing the size of buffer to write into.
-* bitStream will never write outside of this buffer.
-* `dstCapacity` must be >= sizeof(bitD->bitContainer), otherwise @return will be an error code.
-*
-* bits are first added to a local register.
-* Local register is size_t, hence 64-bits on 64-bits systems, or 32-bits on 32-bits systems.
-* Writing data into memory is an explicit operation, performed by the flushBits function.
-* Hence keep track how many bits are potentially stored into local register to avoid register overflow.
-* After a flushBits, a maximum of 7 bits might still be stored into local register.
-*
-* Avoid storing elements of more than 24 bits if you want compatibility with 32-bits bitstream readers.
-*
-* Last operation is to close the bitStream.
-* The function returns the final size of CStream in bytes.
-* If data couldn't fit into `dstBuffer`, it will return a 0 ( == not storable)
-*/
-
-
-/*-********************************************
-* bitStream decoding API (read backward)
-**********************************************/
-typedef struct {
- size_t bitContainer;
- unsigned bitsConsumed;
- const char* ptr;
- const char* start;
- const char* limitPtr;
-} BIT_DStream_t;
-
-typedef enum { BIT_DStream_unfinished = 0,
- BIT_DStream_endOfBuffer = 1,
- BIT_DStream_completed = 2,
- BIT_DStream_overflow = 3 } BIT_DStream_status; /* result of BIT_reloadDStream() */
- /* 1,2,4,8 would be better for bitmap combinations, but slows down performance a bit ... :( */
-
-MEM_STATIC size_t BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize);
-MEM_STATIC size_t BIT_readBits(BIT_DStream_t* bitD, unsigned nbBits);
-MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD);
-MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* bitD);
-
-
-/* Start by invoking BIT_initDStream().
-* A chunk of the bitStream is then stored into a local register.
-* Local register size is 64-bits on 64-bits systems, 32-bits on 32-bits systems (size_t).
-* You can then retrieve bitFields stored into the local register, **in reverse order**.
-* Local register is explicitly reloaded from memory by the BIT_reloadDStream() method.
-* A reload guarantee a minimum of ((8*sizeof(bitD->bitContainer))-7) bits when its result is BIT_DStream_unfinished.
-* Otherwise, it can be less than that, so proceed accordingly.
-* Checking if DStream has reached its end can be performed with BIT_endOfDStream().
-*/
-
-
-/*-****************************************
-* unsafe API
-******************************************/
-MEM_STATIC void BIT_addBitsFast(BIT_CStream_t* bitC, size_t value, unsigned nbBits);
-/* faster, but works only if value is "clean", meaning all high bits above nbBits are 0 */
-
-MEM_STATIC void BIT_flushBitsFast(BIT_CStream_t* bitC);
-/* unsafe version; does not check buffer overflow */
-
-MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, unsigned nbBits);
-/* faster, but works only if nbBits >= 1 */
-
-
-
-/*-**************************************************************
-* Internal functions
-****************************************************************/
-MEM_STATIC unsigned BIT_highbit32 (U32 val)
-{
- assert(val != 0);
- {
-# if defined(_MSC_VER) /* Visual */
- unsigned long r=0;
- _BitScanReverse ( &r, val );
- return (unsigned) r;
-# elif defined(__GNUC__) && (__GNUC__ >= 3) /* Use GCC Intrinsic */
- return 31 - __builtin_clz (val);
-# else /* Software version */
- static const unsigned DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29,
- 11, 14, 16, 18, 22, 25, 3, 30,
- 8, 12, 20, 28, 15, 17, 24, 7,
- 19, 27, 23, 6, 26, 5, 4, 31 };
- U32 v = val;
- v |= v >> 1;
- v |= v >> 2;
- v |= v >> 4;
- v |= v >> 8;
- v |= v >> 16;
- return DeBruijnClz[ (U32) (v * 0x07C4ACDDU) >> 27];
-# endif
- }
-}
-
-/*===== Local Constants =====*/
-static const unsigned BIT_mask[] = {
- 0, 1, 3, 7, 0xF, 0x1F,
- 0x3F, 0x7F, 0xFF, 0x1FF, 0x3FF, 0x7FF,
- 0xFFF, 0x1FFF, 0x3FFF, 0x7FFF, 0xFFFF, 0x1FFFF,
- 0x3FFFF, 0x7FFFF, 0xFFFFF, 0x1FFFFF, 0x3FFFFF, 0x7FFFFF,
- 0xFFFFFF, 0x1FFFFFF, 0x3FFFFFF, 0x7FFFFFF, 0xFFFFFFF, 0x1FFFFFFF,
- 0x3FFFFFFF, 0x7FFFFFFF}; /* up to 31 bits */
-#define BIT_MASK_SIZE (sizeof(BIT_mask) / sizeof(BIT_mask[0]))
-
-/*-**************************************************************
-* bitStream encoding
-****************************************************************/
-/*! BIT_initCStream() :
- * `dstCapacity` must be > sizeof(size_t)
- * @return : 0 if success,
- * otherwise an error code (can be tested using ERR_isError()) */
-MEM_STATIC size_t BIT_initCStream(BIT_CStream_t* bitC,
- void* startPtr, size_t dstCapacity)
-{
- bitC->bitContainer = 0;
- bitC->bitPos = 0;
- bitC->startPtr = (char*)startPtr;
- bitC->ptr = bitC->startPtr;
- bitC->endPtr = bitC->startPtr + dstCapacity - sizeof(bitC->bitContainer);
- if (dstCapacity <= sizeof(bitC->bitContainer)) return ERROR(dstSize_tooSmall);
- return 0;
-}
-
-/*! BIT_addBits() :
- * can add up to 31 bits into `bitC`.
- * Note : does not check for register overflow ! */
-MEM_STATIC void BIT_addBits(BIT_CStream_t* bitC,
- size_t value, unsigned nbBits)
-{
- MEM_STATIC_ASSERT(BIT_MASK_SIZE == 32);
- assert(nbBits < BIT_MASK_SIZE);
- assert(nbBits + bitC->bitPos < sizeof(bitC->bitContainer) * 8);
- bitC->bitContainer |= (value & BIT_mask[nbBits]) << bitC->bitPos;
- bitC->bitPos += nbBits;
-}
-
-/*! BIT_addBitsFast() :
- * works only if `value` is _clean_,
- * meaning all high bits above nbBits are 0 */
-MEM_STATIC void BIT_addBitsFast(BIT_CStream_t* bitC,
- size_t value, unsigned nbBits)
-{
- assert((value>>nbBits) == 0);
- assert(nbBits + bitC->bitPos < sizeof(bitC->bitContainer) * 8);
- bitC->bitContainer |= value << bitC->bitPos;
- bitC->bitPos += nbBits;
-}
-
-/*! BIT_flushBitsFast() :
- * assumption : bitContainer has not overflowed
- * unsafe version; does not check buffer overflow */
-MEM_STATIC void BIT_flushBitsFast(BIT_CStream_t* bitC)
-{
- size_t const nbBytes = bitC->bitPos >> 3;
- assert(bitC->bitPos < sizeof(bitC->bitContainer) * 8);
- MEM_writeLEST(bitC->ptr, bitC->bitContainer);
- bitC->ptr += nbBytes;
- assert(bitC->ptr <= bitC->endPtr);
- bitC->bitPos &= 7;
- bitC->bitContainer >>= nbBytes*8;
-}
-
-/*! BIT_flushBits() :
- * assumption : bitContainer has not overflowed
- * safe version; check for buffer overflow, and prevents it.
- * note : does not signal buffer overflow.
- * overflow will be revealed later on using BIT_closeCStream() */
-MEM_STATIC void BIT_flushBits(BIT_CStream_t* bitC)
-{
- size_t const nbBytes = bitC->bitPos >> 3;
- assert(bitC->bitPos < sizeof(bitC->bitContainer) * 8);
- MEM_writeLEST(bitC->ptr, bitC->bitContainer);
- bitC->ptr += nbBytes;
- if (bitC->ptr > bitC->endPtr) bitC->ptr = bitC->endPtr;
- bitC->bitPos &= 7;
- bitC->bitContainer >>= nbBytes*8;
-}
-
-/*! BIT_closeCStream() :
- * @return : size of CStream, in bytes,
- * or 0 if it could not fit into dstBuffer */
-MEM_STATIC size_t BIT_closeCStream(BIT_CStream_t* bitC)
-{
- BIT_addBitsFast(bitC, 1, 1); /* endMark */
- BIT_flushBits(bitC);
- if (bitC->ptr >= bitC->endPtr) return 0; /* overflow detected */
- return (bitC->ptr - bitC->startPtr) + (bitC->bitPos > 0);
-}
-
-
-/*-********************************************************
-* bitStream decoding
-**********************************************************/
-/*! BIT_initDStream() :
- * Initialize a BIT_DStream_t.
- * `bitD` : a pointer to an already allocated BIT_DStream_t structure.
- * `srcSize` must be the *exact* size of the bitStream, in bytes.
- * @return : size of stream (== srcSize), or an errorCode if a problem is detected
- */
-MEM_STATIC size_t BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize)
-{
- if (srcSize < 1) { memset(bitD, 0, sizeof(*bitD)); return ERROR(srcSize_wrong); }
-
- bitD->start = (const char*)srcBuffer;
- bitD->limitPtr = bitD->start + sizeof(bitD->bitContainer);
-
- if (srcSize >= sizeof(bitD->bitContainer)) { /* normal case */
- bitD->ptr = (const char*)srcBuffer + srcSize - sizeof(bitD->bitContainer);
- bitD->bitContainer = MEM_readLEST(bitD->ptr);
- { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1];
- bitD->bitsConsumed = lastByte ? 8 - BIT_highbit32(lastByte) : 0; /* ensures bitsConsumed is always set */
- if (lastByte == 0) return ERROR(GENERIC); /* endMark not present */ }
- } else {
- bitD->ptr = bitD->start;
- bitD->bitContainer = *(const BYTE*)(bitD->start);
- switch(srcSize)
- {
- case 7: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[6]) << (sizeof(bitD->bitContainer)*8 - 16);
- /* fall-through */
-
- case 6: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[5]) << (sizeof(bitD->bitContainer)*8 - 24);
- /* fall-through */
-
- case 5: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[4]) << (sizeof(bitD->bitContainer)*8 - 32);
- /* fall-through */
-
- case 4: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[3]) << 24;
- /* fall-through */
-
- case 3: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[2]) << 16;
- /* fall-through */
-
- case 2: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[1]) << 8;
- /* fall-through */
-
- default: break;
- }
- { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1];
- bitD->bitsConsumed = lastByte ? 8 - BIT_highbit32(lastByte) : 0;
- if (lastByte == 0) return ERROR(corruption_detected); /* endMark not present */
- }
- bitD->bitsConsumed += (U32)(sizeof(bitD->bitContainer) - srcSize)*8;
- }
-
- return srcSize;
-}
-
-MEM_STATIC size_t BIT_getUpperBits(size_t bitContainer, U32 const start)
-{
- return bitContainer >> start;
-}
-
-MEM_STATIC size_t BIT_getMiddleBits(size_t bitContainer, U32 const start, U32 const nbBits)
-{
- U32 const regMask = sizeof(bitContainer)*8 - 1;
- /* if start > regMask, bitstream is corrupted, and result is undefined */
- assert(nbBits < BIT_MASK_SIZE);
- return (bitContainer >> (start & regMask)) & BIT_mask[nbBits];
-}
-
-MEM_STATIC size_t BIT_getLowerBits(size_t bitContainer, U32 const nbBits)
-{
- assert(nbBits < BIT_MASK_SIZE);
- return bitContainer & BIT_mask[nbBits];
-}
-
-/*! BIT_lookBits() :
- * Provides next n bits from local register.
- * local register is not modified.
- * On 32-bits, maxNbBits==24.
- * On 64-bits, maxNbBits==56.
- * @return : value extracted */
-MEM_STATIC size_t BIT_lookBits(const BIT_DStream_t* bitD, U32 nbBits)
-{
- /* arbitrate between double-shift and shift+mask */
-#if 1
- /* if bitD->bitsConsumed + nbBits > sizeof(bitD->bitContainer)*8,
- * bitstream is likely corrupted, and result is undefined */
- return BIT_getMiddleBits(bitD->bitContainer, (sizeof(bitD->bitContainer)*8) - bitD->bitsConsumed - nbBits, nbBits);
-#else
- /* this code path is slower on my os-x laptop */
- U32 const regMask = sizeof(bitD->bitContainer)*8 - 1;
- return ((bitD->bitContainer << (bitD->bitsConsumed & regMask)) >> 1) >> ((regMask-nbBits) & regMask);
-#endif
-}
-
-/*! BIT_lookBitsFast() :
- * unsafe version; only works if nbBits >= 1 */
-MEM_STATIC size_t BIT_lookBitsFast(const BIT_DStream_t* bitD, U32 nbBits)
-{
- U32 const regMask = sizeof(bitD->bitContainer)*8 - 1;
- assert(nbBits >= 1);
- return (bitD->bitContainer << (bitD->bitsConsumed & regMask)) >> (((regMask+1)-nbBits) & regMask);
-}
-
-MEM_STATIC void BIT_skipBits(BIT_DStream_t* bitD, U32 nbBits)
-{
- bitD->bitsConsumed += nbBits;
-}
-
-/*! BIT_readBits() :
- * Read (consume) next n bits from local register and update.
- * Pay attention to not read more than nbBits contained into local register.
- * @return : extracted value. */
-MEM_STATIC size_t BIT_readBits(BIT_DStream_t* bitD, unsigned nbBits)
-{
- size_t const value = BIT_lookBits(bitD, nbBits);
- BIT_skipBits(bitD, nbBits);
- return value;
-}
-
-/*! BIT_readBitsFast() :
- * unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, unsigned nbBits)
-{
- size_t const value = BIT_lookBitsFast(bitD, nbBits);
- assert(nbBits >= 1);
- BIT_skipBits(bitD, nbBits);
- return value;
-}
-
-/*! BIT_reloadDStream() :
- * Refill `bitD` from buffer previously set in BIT_initDStream() .
- * This function is safe, it guarantees it will not read beyond src buffer.
- * @return : status of `BIT_DStream_t` internal register.
- * when status == BIT_DStream_unfinished, internal register is filled with at least 25 or 57 bits */
-MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD)
-{
- if (bitD->bitsConsumed > (sizeof(bitD->bitContainer)*8)) /* overflow detected, like end of stream */
- return BIT_DStream_overflow;
-
- if (bitD->ptr >= bitD->limitPtr) {
- bitD->ptr -= bitD->bitsConsumed >> 3;
- bitD->bitsConsumed &= 7;
- bitD->bitContainer = MEM_readLEST(bitD->ptr);
- return BIT_DStream_unfinished;
- }
- if (bitD->ptr == bitD->start) {
- if (bitD->bitsConsumed < sizeof(bitD->bitContainer)*8) return BIT_DStream_endOfBuffer;
- return BIT_DStream_completed;
- }
- /* start < ptr < limitPtr */
- { U32 nbBytes = bitD->bitsConsumed >> 3;
- BIT_DStream_status result = BIT_DStream_unfinished;
- if (bitD->ptr - nbBytes < bitD->start) {
- nbBytes = (U32)(bitD->ptr - bitD->start); /* ptr > start */
- result = BIT_DStream_endOfBuffer;
- }
- bitD->ptr -= nbBytes;
- bitD->bitsConsumed -= nbBytes*8;
- bitD->bitContainer = MEM_readLEST(bitD->ptr); /* reminder : srcSize > sizeof(bitD->bitContainer), otherwise bitD->ptr == bitD->start */
- return result;
- }
-}
-
-/*! BIT_endOfDStream() :
- * @return : 1 if DStream has _exactly_ reached its end (all bits consumed).
- */
-MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* DStream)
-{
- return ((DStream->ptr == DStream->start) && (DStream->bitsConsumed == sizeof(DStream->bitContainer)*8));
-}
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* BITSTREAM_H_MODULE */
diff --git a/vendor/github.com/DataDog/zstd/compiler.h b/vendor/github.com/DataDog/zstd/compiler.h
deleted file mode 100644
index 87bf51a..0000000
--- a/vendor/github.com/DataDog/zstd/compiler.h
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_COMPILER_H
-#define ZSTD_COMPILER_H
-
-/*-*******************************************************
-* Compiler specifics
-*********************************************************/
-/* force inlining */
-
-#if !defined(ZSTD_NO_INLINE)
-#if defined (__GNUC__) || defined(__cplusplus) || defined(__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */
-# define INLINE_KEYWORD inline
-#else
-# define INLINE_KEYWORD
-#endif
-
-#if defined(__GNUC__)
-# define FORCE_INLINE_ATTR __attribute__((always_inline))
-#elif defined(_MSC_VER)
-# define FORCE_INLINE_ATTR __forceinline
-#else
-# define FORCE_INLINE_ATTR
-#endif
-
-#else
-
-#define INLINE_KEYWORD
-#define FORCE_INLINE_ATTR
-
-#endif
-
-/**
- * FORCE_INLINE_TEMPLATE is used to define C "templates", which take constant
- * parameters. They must be inlined for the compiler to eliminate the constant
- * branches.
- */
-#define FORCE_INLINE_TEMPLATE static INLINE_KEYWORD FORCE_INLINE_ATTR
-/**
- * HINT_INLINE is used to help the compiler generate better code. It is *not*
- * used for "templates", so it can be tweaked based on the compilers
- * performance.
- *
- * gcc-4.8 and gcc-4.9 have been shown to benefit from leaving off the
- * always_inline attribute.
- *
- * clang up to 5.0.0 (trunk) benefit tremendously from the always_inline
- * attribute.
- */
-#if !defined(__clang__) && defined(__GNUC__) && __GNUC__ >= 4 && __GNUC_MINOR__ >= 8 && __GNUC__ < 5
-# define HINT_INLINE static INLINE_KEYWORD
-#else
-# define HINT_INLINE static INLINE_KEYWORD FORCE_INLINE_ATTR
-#endif
-
-/* force no inlining */
-#ifdef _MSC_VER
-# define FORCE_NOINLINE static __declspec(noinline)
-#else
-# ifdef __GNUC__
-# define FORCE_NOINLINE static __attribute__((__noinline__))
-# else
-# define FORCE_NOINLINE static
-# endif
-#endif
-
-/* target attribute */
-#ifndef __has_attribute
- #define __has_attribute(x) 0 /* Compatibility with non-clang compilers. */
-#endif
-#if defined(__GNUC__)
-# define TARGET_ATTRIBUTE(target) __attribute__((__target__(target)))
-#else
-# define TARGET_ATTRIBUTE(target)
-#endif
-
-/* Enable runtime BMI2 dispatch based on the CPU.
- * Enabled for clang & gcc >=4.8 on x86 when BMI2 isn't enabled by default.
- */
-#ifndef DYNAMIC_BMI2
- #if ((defined(__clang__) && __has_attribute(__target__)) \
- || (defined(__GNUC__) \
- && (__GNUC__ >= 5 || (__GNUC__ == 4 && __GNUC_MINOR__ >= 8)))) \
- && (defined(__x86_64__) || defined(_M_X86)) \
- && !defined(__BMI2__)
- # define DYNAMIC_BMI2 1
- #else
- # define DYNAMIC_BMI2 0
- #endif
-#endif
-
-/* prefetch
- * can be disabled, by declaring NO_PREFETCH build macro */
-#if defined(NO_PREFETCH)
-# define PREFETCH_L1(ptr) (void)(ptr) /* disabled */
-# define PREFETCH_L2(ptr) (void)(ptr) /* disabled */
-#else
-# if defined(_MSC_VER) && (defined(_M_X64) || defined(_M_I86)) /* _mm_prefetch() is not defined outside of x86/x64 */
-# include <mmintrin.h> /* https://msdn.microsoft.com/fr-fr/library/84szxsww(v=vs.90).aspx */
-# define PREFETCH_L1(ptr) _mm_prefetch((const char*)(ptr), _MM_HINT_T0)
-# define PREFETCH_L2(ptr) _mm_prefetch((const char*)(ptr), _MM_HINT_T1)
-# elif defined(__GNUC__) && ( (__GNUC__ >= 4) || ( (__GNUC__ == 3) && (__GNUC_MINOR__ >= 1) ) )
-# define PREFETCH_L1(ptr) __builtin_prefetch((ptr), 0 /* rw==read */, 3 /* locality */)
-# define PREFETCH_L2(ptr) __builtin_prefetch((ptr), 0 /* rw==read */, 2 /* locality */)
-# else
-# define PREFETCH_L1(ptr) (void)(ptr) /* disabled */
-# define PREFETCH_L2(ptr) (void)(ptr) /* disabled */
-# endif
-#endif /* NO_PREFETCH */
-
-#define CACHELINE_SIZE 64
-
-#define PREFETCH_AREA(p, s) { \
- const char* const _ptr = (const char*)(p); \
- size_t const _size = (size_t)(s); \
- size_t _pos; \
- for (_pos=0; _pos<_size; _pos+=CACHELINE_SIZE) { \
- PREFETCH_L2(_ptr + _pos); \
- } \
-}
-
-/* vectorization */
-#if !defined(__clang__) && defined(__GNUC__)
-# define DONT_VECTORIZE __attribute__((optimize("no-tree-vectorize")))
-#else
-# define DONT_VECTORIZE
-#endif
-
-/* disable warnings */
-#ifdef _MSC_VER /* Visual Studio */
-# include <intrin.h> /* For Visual 2005 */
-# pragma warning(disable : 4100) /* disable: C4100: unreferenced formal parameter */
-# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */
-# pragma warning(disable : 4204) /* disable: C4204: non-constant aggregate initializer */
-# pragma warning(disable : 4214) /* disable: C4214: non-int bitfields */
-# pragma warning(disable : 4324) /* disable: C4324: padded structure */
-#endif
-
-#endif /* ZSTD_COMPILER_H */
diff --git a/vendor/github.com/DataDog/zstd/cover.c b/vendor/github.com/DataDog/zstd/cover.c
deleted file mode 100644
index 6219967..0000000
--- a/vendor/github.com/DataDog/zstd/cover.c
+++ /dev/null
@@ -1,1237 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-/* *****************************************************************************
- * Constructs a dictionary using a heuristic based on the following paper:
- *
- * Liao, Petri, Moffat, Wirth
- * Effective Construction of Relative Lempel-Ziv Dictionaries
- * Published in WWW 2016.
- *
- * Adapted from code originally written by @ot (Giuseppe Ottaviano).
- ******************************************************************************/
-
-/*-*************************************
-* Dependencies
-***************************************/
-#include <stdio.h> /* fprintf */
-#include <stdlib.h> /* malloc, free, qsort */
-#include <string.h> /* memset */
-#include <time.h> /* clock */
-
-#include "mem.h" /* read */
-#include "pool.h"
-#include "threading.h"
-#include "cover.h"
-#include "zstd_internal.h" /* includes zstd.h */
-#ifndef ZDICT_STATIC_LINKING_ONLY
-#define ZDICT_STATIC_LINKING_ONLY
-#endif
-#include "zdict.h"
-
-/*-*************************************
-* Constants
-***************************************/
-#define COVER_MAX_SAMPLES_SIZE (sizeof(size_t) == 8 ? ((unsigned)-1) : ((unsigned)1 GB))
-#define DEFAULT_SPLITPOINT 1.0
-
-/*-*************************************
-* Console display
-***************************************/
-static int g_displayLevel = 2;
-#define DISPLAY(...) \
- { \
- fprintf(stderr, __VA_ARGS__); \
- fflush(stderr); \
- }
-#define LOCALDISPLAYLEVEL(displayLevel, l, ...) \
- if (displayLevel >= l) { \
- DISPLAY(__VA_ARGS__); \
- } /* 0 : no display; 1: errors; 2: default; 3: details; 4: debug */
-#define DISPLAYLEVEL(l, ...) LOCALDISPLAYLEVEL(g_displayLevel, l, __VA_ARGS__)
-
-#define LOCALDISPLAYUPDATE(displayLevel, l, ...) \
- if (displayLevel >= l) { \
- if ((clock() - g_time > refreshRate) || (displayLevel >= 4)) { \
- g_time = clock(); \
- DISPLAY(__VA_ARGS__); \
- } \
- }
-#define DISPLAYUPDATE(l, ...) LOCALDISPLAYUPDATE(g_displayLevel, l, __VA_ARGS__)
-static const clock_t refreshRate = CLOCKS_PER_SEC * 15 / 100;
-static clock_t g_time = 0;
-
-/*-*************************************
-* Hash table
-***************************************
-* A small specialized hash map for storing activeDmers.
-* The map does not resize, so if it becomes full it will loop forever.
-* Thus, the map must be large enough to store every value.
-* The map implements linear probing and keeps its load less than 0.5.
-*/
-
-#define MAP_EMPTY_VALUE ((U32)-1)
-typedef struct COVER_map_pair_t_s {
- U32 key;
- U32 value;
-} COVER_map_pair_t;
-
-typedef struct COVER_map_s {
- COVER_map_pair_t *data;
- U32 sizeLog;
- U32 size;
- U32 sizeMask;
-} COVER_map_t;
-
-/**
- * Clear the map.
- */
-static void COVER_map_clear(COVER_map_t *map) {
- memset(map->data, MAP_EMPTY_VALUE, map->size * sizeof(COVER_map_pair_t));
-}
-
-/**
- * Initializes a map of the given size.
- * Returns 1 on success and 0 on failure.
- * The map must be destroyed with COVER_map_destroy().
- * The map is only guaranteed to be large enough to hold size elements.
- */
-static int COVER_map_init(COVER_map_t *map, U32 size) {
- map->sizeLog = ZSTD_highbit32(size) + 2;
- map->size = (U32)1 << map->sizeLog;
- map->sizeMask = map->size - 1;
- map->data = (COVER_map_pair_t *)malloc(map->size * sizeof(COVER_map_pair_t));
- if (!map->data) {
- map->sizeLog = 0;
- map->size = 0;
- return 0;
- }
- COVER_map_clear(map);
- return 1;
-}
-
-/**
- * Internal hash function
- */
-static const U32 prime4bytes = 2654435761U;
-static U32 COVER_map_hash(COVER_map_t *map, U32 key) {
- return (key * prime4bytes) >> (32 - map->sizeLog);
-}
-
-/**
- * Helper function that returns the index that a key should be placed into.
- */
-static U32 COVER_map_index(COVER_map_t *map, U32 key) {
- const U32 hash = COVER_map_hash(map, key);
- U32 i;
- for (i = hash;; i = (i + 1) & map->sizeMask) {
- COVER_map_pair_t *pos = &map->data[i];
- if (pos->value == MAP_EMPTY_VALUE) {
- return i;
- }
- if (pos->key == key) {
- return i;
- }
- }
-}
-
-/**
- * Returns the pointer to the value for key.
- * If key is not in the map, it is inserted and the value is set to 0.
- * The map must not be full.
- */
-static U32 *COVER_map_at(COVER_map_t *map, U32 key) {
- COVER_map_pair_t *pos = &map->data[COVER_map_index(map, key)];
- if (pos->value == MAP_EMPTY_VALUE) {
- pos->key = key;
- pos->value = 0;
- }
- return &pos->value;
-}
-
-/**
- * Deletes key from the map if present.
- */
-static void COVER_map_remove(COVER_map_t *map, U32 key) {
- U32 i = COVER_map_index(map, key);
- COVER_map_pair_t *del = &map->data[i];
- U32 shift = 1;
- if (del->value == MAP_EMPTY_VALUE) {
- return;
- }
- for (i = (i + 1) & map->sizeMask;; i = (i + 1) & map->sizeMask) {
- COVER_map_pair_t *const pos = &map->data[i];
- /* If the position is empty we are done */
- if (pos->value == MAP_EMPTY_VALUE) {
- del->value = MAP_EMPTY_VALUE;
- return;
- }
- /* If pos can be moved to del do so */
- if (((i - COVER_map_hash(map, pos->key)) & map->sizeMask) >= shift) {
- del->key = pos->key;
- del->value = pos->value;
- del = pos;
- shift = 1;
- } else {
- ++shift;
- }
- }
-}
-
-/**
- * Destroys a map that is inited with COVER_map_init().
- */
-static void COVER_map_destroy(COVER_map_t *map) {
- if (map->data) {
- free(map->data);
- }
- map->data = NULL;
- map->size = 0;
-}
-
-/*-*************************************
-* Context
-***************************************/
-
-typedef struct {
- const BYTE *samples;
- size_t *offsets;
- const size_t *samplesSizes;
- size_t nbSamples;
- size_t nbTrainSamples;
- size_t nbTestSamples;
- U32 *suffix;
- size_t suffixSize;
- U32 *freqs;
- U32 *dmerAt;
- unsigned d;
-} COVER_ctx_t;
-
-/* We need a global context for qsort... */
-static COVER_ctx_t *g_ctx = NULL;
-
-/*-*************************************
-* Helper functions
-***************************************/
-
-/**
- * Returns the sum of the sample sizes.
- */
-size_t COVER_sum(const size_t *samplesSizes, unsigned nbSamples) {
- size_t sum = 0;
- unsigned i;
- for (i = 0; i < nbSamples; ++i) {
- sum += samplesSizes[i];
- }
- return sum;
-}
-
-/**
- * Returns -1 if the dmer at lp is less than the dmer at rp.
- * Return 0 if the dmers at lp and rp are equal.
- * Returns 1 if the dmer at lp is greater than the dmer at rp.
- */
-static int COVER_cmp(COVER_ctx_t *ctx, const void *lp, const void *rp) {
- U32 const lhs = *(U32 const *)lp;
- U32 const rhs = *(U32 const *)rp;
- return memcmp(ctx->samples + lhs, ctx->samples + rhs, ctx->d);
-}
-/**
- * Faster version for d <= 8.
- */
-static int COVER_cmp8(COVER_ctx_t *ctx, const void *lp, const void *rp) {
- U64 const mask = (ctx->d == 8) ? (U64)-1 : (((U64)1 << (8 * ctx->d)) - 1);
- U64 const lhs = MEM_readLE64(ctx->samples + *(U32 const *)lp) & mask;
- U64 const rhs = MEM_readLE64(ctx->samples + *(U32 const *)rp) & mask;
- if (lhs < rhs) {
- return -1;
- }
- return (lhs > rhs);
-}
-
-/**
- * Same as COVER_cmp() except ties are broken by pointer value
- * NOTE: g_ctx must be set to call this function. A global is required because
- * qsort doesn't take an opaque pointer.
- */
-static int COVER_strict_cmp(const void *lp, const void *rp) {
- int result = COVER_cmp(g_ctx, lp, rp);
- if (result == 0) {
- result = lp < rp ? -1 : 1;
- }
- return result;
-}
-/**
- * Faster version for d <= 8.
- */
-static int COVER_strict_cmp8(const void *lp, const void *rp) {
- int result = COVER_cmp8(g_ctx, lp, rp);
- if (result == 0) {
- result = lp < rp ? -1 : 1;
- }
- return result;
-}
-
-/**
- * Returns the first pointer in [first, last) whose element does not compare
- * less than value. If no such element exists it returns last.
- */
-static const size_t *COVER_lower_bound(const size_t *first, const size_t *last,
- size_t value) {
- size_t count = last - first;
- while (count != 0) {
- size_t step = count / 2;
- const size_t *ptr = first;
- ptr += step;
- if (*ptr < value) {
- first = ++ptr;
- count -= step + 1;
- } else {
- count = step;
- }
- }
- return first;
-}
-
-/**
- * Generic groupBy function.
- * Groups an array sorted by cmp into groups with equivalent values.
- * Calls grp for each group.
- */
-static void
-COVER_groupBy(const void *data, size_t count, size_t size, COVER_ctx_t *ctx,
- int (*cmp)(COVER_ctx_t *, const void *, const void *),
- void (*grp)(COVER_ctx_t *, const void *, const void *)) {
- const BYTE *ptr = (const BYTE *)data;
- size_t num = 0;
- while (num < count) {
- const BYTE *grpEnd = ptr + size;
- ++num;
- while (num < count && cmp(ctx, ptr, grpEnd) == 0) {
- grpEnd += size;
- ++num;
- }
- grp(ctx, ptr, grpEnd);
- ptr = grpEnd;
- }
-}
-
-/*-*************************************
-* Cover functions
-***************************************/
-
-/**
- * Called on each group of positions with the same dmer.
- * Counts the frequency of each dmer and saves it in the suffix array.
- * Fills `ctx->dmerAt`.
- */
-static void COVER_group(COVER_ctx_t *ctx, const void *group,
- const void *groupEnd) {
- /* The group consists of all the positions with the same first d bytes. */
- const U32 *grpPtr = (const U32 *)group;
- const U32 *grpEnd = (const U32 *)groupEnd;
- /* The dmerId is how we will reference this dmer.
- * This allows us to map the whole dmer space to a much smaller space, the
- * size of the suffix array.
- */
- const U32 dmerId = (U32)(grpPtr - ctx->suffix);
- /* Count the number of samples this dmer shows up in */
- U32 freq = 0;
- /* Details */
- const size_t *curOffsetPtr = ctx->offsets;
- const size_t *offsetsEnd = ctx->offsets + ctx->nbSamples;
- /* Once *grpPtr >= curSampleEnd this occurrence of the dmer is in a
- * different sample than the last.
- */
- size_t curSampleEnd = ctx->offsets[0];
- for (; grpPtr != grpEnd; ++grpPtr) {
- /* Save the dmerId for this position so we can get back to it. */
- ctx->dmerAt[*grpPtr] = dmerId;
- /* Dictionaries only help for the first reference to the dmer.
- * After that zstd can reference the match from the previous reference.
- * So only count each dmer once for each sample it is in.
- */
- if (*grpPtr < curSampleEnd) {
- continue;
- }
- freq += 1;
- /* Binary search to find the end of the sample *grpPtr is in.
- * In the common case that grpPtr + 1 == grpEnd we can skip the binary
- * search because the loop is over.
- */
- if (grpPtr + 1 != grpEnd) {
- const size_t *sampleEndPtr =
- COVER_lower_bound(curOffsetPtr, offsetsEnd, *grpPtr);
- curSampleEnd = *sampleEndPtr;
- curOffsetPtr = sampleEndPtr + 1;
- }
- }
- /* At this point we are never going to look at this segment of the suffix
- * array again. We take advantage of this fact to save memory.
- * We store the frequency of the dmer in the first position of the group,
- * which is dmerId.
- */
- ctx->suffix[dmerId] = freq;
-}
-
-
-/**
- * Selects the best segment in an epoch.
- * Segments of are scored according to the function:
- *
- * Let F(d) be the frequency of dmer d.
- * Let S_i be the dmer at position i of segment S which has length k.
- *
- * Score(S) = F(S_1) + F(S_2) + ... + F(S_{k-d+1})
- *
- * Once the dmer d is in the dictionary we set F(d) = 0.
- */
-static COVER_segment_t COVER_selectSegment(const COVER_ctx_t *ctx, U32 *freqs,
- COVER_map_t *activeDmers, U32 begin,
- U32 end,
- ZDICT_cover_params_t parameters) {
- /* Constants */
- const U32 k = parameters.k;
- const U32 d = parameters.d;
- const U32 dmersInK = k - d + 1;
- /* Try each segment (activeSegment) and save the best (bestSegment) */
- COVER_segment_t bestSegment = {0, 0, 0};
- COVER_segment_t activeSegment;
- /* Reset the activeDmers in the segment */
- COVER_map_clear(activeDmers);
- /* The activeSegment starts at the beginning of the epoch. */
- activeSegment.begin = begin;
- activeSegment.end = begin;
- activeSegment.score = 0;
- /* Slide the activeSegment through the whole epoch.
- * Save the best segment in bestSegment.
- */
- while (activeSegment.end < end) {
- /* The dmerId for the dmer at the next position */
- U32 newDmer = ctx->dmerAt[activeSegment.end];
- /* The entry in activeDmers for this dmerId */
- U32 *newDmerOcc = COVER_map_at(activeDmers, newDmer);
- /* If the dmer isn't already present in the segment add its score. */
- if (*newDmerOcc == 0) {
- /* The paper suggest using the L-0.5 norm, but experiments show that it
- * doesn't help.
- */
- activeSegment.score += freqs[newDmer];
- }
- /* Add the dmer to the segment */
- activeSegment.end += 1;
- *newDmerOcc += 1;
-
- /* If the window is now too large, drop the first position */
- if (activeSegment.end - activeSegment.begin == dmersInK + 1) {
- U32 delDmer = ctx->dmerAt[activeSegment.begin];
- U32 *delDmerOcc = COVER_map_at(activeDmers, delDmer);
- activeSegment.begin += 1;
- *delDmerOcc -= 1;
- /* If this is the last occurrence of the dmer, subtract its score */
- if (*delDmerOcc == 0) {
- COVER_map_remove(activeDmers, delDmer);
- activeSegment.score -= freqs[delDmer];
- }
- }
-
- /* If this segment is the best so far save it */
- if (activeSegment.score > bestSegment.score) {
- bestSegment = activeSegment;
- }
- }
- {
- /* Trim off the zero frequency head and tail from the segment. */
- U32 newBegin = bestSegment.end;
- U32 newEnd = bestSegment.begin;
- U32 pos;
- for (pos = bestSegment.begin; pos != bestSegment.end; ++pos) {
- U32 freq = freqs[ctx->dmerAt[pos]];
- if (freq != 0) {
- newBegin = MIN(newBegin, pos);
- newEnd = pos + 1;
- }
- }
- bestSegment.begin = newBegin;
- bestSegment.end = newEnd;
- }
- {
- /* Zero out the frequency of each dmer covered by the chosen segment. */
- U32 pos;
- for (pos = bestSegment.begin; pos != bestSegment.end; ++pos) {
- freqs[ctx->dmerAt[pos]] = 0;
- }
- }
- return bestSegment;
-}
-
-/**
- * Check the validity of the parameters.
- * Returns non-zero if the parameters are valid and 0 otherwise.
- */
-static int COVER_checkParameters(ZDICT_cover_params_t parameters,
- size_t maxDictSize) {
- /* k and d are required parameters */
- if (parameters.d == 0 || parameters.k == 0) {
- return 0;
- }
- /* k <= maxDictSize */
- if (parameters.k > maxDictSize) {
- return 0;
- }
- /* d <= k */
- if (parameters.d > parameters.k) {
- return 0;
- }
- /* 0 < splitPoint <= 1 */
- if (parameters.splitPoint <= 0 || parameters.splitPoint > 1){
- return 0;
- }
- return 1;
-}
-
-/**
- * Clean up a context initialized with `COVER_ctx_init()`.
- */
-static void COVER_ctx_destroy(COVER_ctx_t *ctx) {
- if (!ctx) {
- return;
- }
- if (ctx->suffix) {
- free(ctx->suffix);
- ctx->suffix = NULL;
- }
- if (ctx->freqs) {
- free(ctx->freqs);
- ctx->freqs = NULL;
- }
- if (ctx->dmerAt) {
- free(ctx->dmerAt);
- ctx->dmerAt = NULL;
- }
- if (ctx->offsets) {
- free(ctx->offsets);
- ctx->offsets = NULL;
- }
-}
-
-/**
- * Prepare a context for dictionary building.
- * The context is only dependent on the parameter `d` and can used multiple
- * times.
- * Returns 0 on success or error code on error.
- * The context must be destroyed with `COVER_ctx_destroy()`.
- */
-static size_t COVER_ctx_init(COVER_ctx_t *ctx, const void *samplesBuffer,
- const size_t *samplesSizes, unsigned nbSamples,
- unsigned d, double splitPoint) {
- const BYTE *const samples = (const BYTE *)samplesBuffer;
- const size_t totalSamplesSize = COVER_sum(samplesSizes, nbSamples);
- /* Split samples into testing and training sets */
- const unsigned nbTrainSamples = splitPoint < 1.0 ? (unsigned)((double)nbSamples * splitPoint) : nbSamples;
- const unsigned nbTestSamples = splitPoint < 1.0 ? nbSamples - nbTrainSamples : nbSamples;
- const size_t trainingSamplesSize = splitPoint < 1.0 ? COVER_sum(samplesSizes, nbTrainSamples) : totalSamplesSize;
- const size_t testSamplesSize = splitPoint < 1.0 ? COVER_sum(samplesSizes + nbTrainSamples, nbTestSamples) : totalSamplesSize;
- /* Checks */
- if (totalSamplesSize < MAX(d, sizeof(U64)) ||
- totalSamplesSize >= (size_t)COVER_MAX_SAMPLES_SIZE) {
- DISPLAYLEVEL(1, "Total samples size is too large (%u MB), maximum size is %u MB\n",
- (unsigned)(totalSamplesSize>>20), (COVER_MAX_SAMPLES_SIZE >> 20));
- return ERROR(srcSize_wrong);
- }
- /* Check if there are at least 5 training samples */
- if (nbTrainSamples < 5) {
- DISPLAYLEVEL(1, "Total number of training samples is %u and is invalid.", nbTrainSamples);
- return ERROR(srcSize_wrong);
- }
- /* Check if there's testing sample */
- if (nbTestSamples < 1) {
- DISPLAYLEVEL(1, "Total number of testing samples is %u and is invalid.", nbTestSamples);
- return ERROR(srcSize_wrong);
- }
- /* Zero the context */
- memset(ctx, 0, sizeof(*ctx));
- DISPLAYLEVEL(2, "Training on %u samples of total size %u\n", nbTrainSamples,
- (unsigned)trainingSamplesSize);
- DISPLAYLEVEL(2, "Testing on %u samples of total size %u\n", nbTestSamples,
- (unsigned)testSamplesSize);
- ctx->samples = samples;
- ctx->samplesSizes = samplesSizes;
- ctx->nbSamples = nbSamples;
- ctx->nbTrainSamples = nbTrainSamples;
- ctx->nbTestSamples = nbTestSamples;
- /* Partial suffix array */
- ctx->suffixSize = trainingSamplesSize - MAX(d, sizeof(U64)) + 1;
- ctx->suffix = (U32 *)malloc(ctx->suffixSize * sizeof(U32));
- /* Maps index to the dmerID */
- ctx->dmerAt = (U32 *)malloc(ctx->suffixSize * sizeof(U32));
- /* The offsets of each file */
- ctx->offsets = (size_t *)malloc((nbSamples + 1) * sizeof(size_t));
- if (!ctx->suffix || !ctx->dmerAt || !ctx->offsets) {
- DISPLAYLEVEL(1, "Failed to allocate scratch buffers\n");
- COVER_ctx_destroy(ctx);
- return ERROR(memory_allocation);
- }
- ctx->freqs = NULL;
- ctx->d = d;
-
- /* Fill offsets from the samplesSizes */
- {
- U32 i;
- ctx->offsets[0] = 0;
- for (i = 1; i <= nbSamples; ++i) {
- ctx->offsets[i] = ctx->offsets[i - 1] + samplesSizes[i - 1];
- }
- }
- DISPLAYLEVEL(2, "Constructing partial suffix array\n");
- {
- /* suffix is a partial suffix array.
- * It only sorts suffixes by their first parameters.d bytes.
- * The sort is stable, so each dmer group is sorted by position in input.
- */
- U32 i;
- for (i = 0; i < ctx->suffixSize; ++i) {
- ctx->suffix[i] = i;
- }
- /* qsort doesn't take an opaque pointer, so pass as a global.
- * On OpenBSD qsort() is not guaranteed to be stable, their mergesort() is.
- */
- g_ctx = ctx;
-#if defined(__OpenBSD__)
- mergesort(ctx->suffix, ctx->suffixSize, sizeof(U32),
- (ctx->d <= 8 ? &COVER_strict_cmp8 : &COVER_strict_cmp));
-#else
- qsort(ctx->suffix, ctx->suffixSize, sizeof(U32),
- (ctx->d <= 8 ? &COVER_strict_cmp8 : &COVER_strict_cmp));
-#endif
- }
- DISPLAYLEVEL(2, "Computing frequencies\n");
- /* For each dmer group (group of positions with the same first d bytes):
- * 1. For each position we set dmerAt[position] = dmerID. The dmerID is
- * (groupBeginPtr - suffix). This allows us to go from position to
- * dmerID so we can look up values in freq.
- * 2. We calculate how many samples the dmer occurs in and save it in
- * freqs[dmerId].
- */
- COVER_groupBy(ctx->suffix, ctx->suffixSize, sizeof(U32), ctx,
- (ctx->d <= 8 ? &COVER_cmp8 : &COVER_cmp), &COVER_group);
- ctx->freqs = ctx->suffix;
- ctx->suffix = NULL;
- return 0;
-}
-
-void COVER_warnOnSmallCorpus(size_t maxDictSize, size_t nbDmers, int displayLevel)
-{
- const double ratio = (double)nbDmers / maxDictSize;
- if (ratio >= 10) {
- return;
- }
- LOCALDISPLAYLEVEL(displayLevel, 1,
- "WARNING: The maximum dictionary size %u is too large "
- "compared to the source size %u! "
- "size(source)/size(dictionary) = %f, but it should be >= "
- "10! This may lead to a subpar dictionary! We recommend "
- "training on sources at least 10x, and up to 100x the "
- "size of the dictionary!\n", (U32)maxDictSize,
- (U32)nbDmers, ratio);
-}
-
-COVER_epoch_info_t COVER_computeEpochs(U32 maxDictSize,
- U32 nbDmers, U32 k, U32 passes)
-{
- const U32 minEpochSize = k * 10;
- COVER_epoch_info_t epochs;
- epochs.num = MAX(1, maxDictSize / k / passes);
- epochs.size = nbDmers / epochs.num;
- if (epochs.size >= minEpochSize) {
- assert(epochs.size * epochs.num <= nbDmers);
- return epochs;
- }
- epochs.size = MIN(minEpochSize, nbDmers);
- epochs.num = nbDmers / epochs.size;
- assert(epochs.size * epochs.num <= nbDmers);
- return epochs;
-}
-
-/**
- * Given the prepared context build the dictionary.
- */
-static size_t COVER_buildDictionary(const COVER_ctx_t *ctx, U32 *freqs,
- COVER_map_t *activeDmers, void *dictBuffer,
- size_t dictBufferCapacity,
- ZDICT_cover_params_t parameters) {
- BYTE *const dict = (BYTE *)dictBuffer;
- size_t tail = dictBufferCapacity;
- /* Divide the data into epochs. We will select one segment from each epoch. */
- const COVER_epoch_info_t epochs = COVER_computeEpochs(
- (U32)dictBufferCapacity, (U32)ctx->suffixSize, parameters.k, 4);
- const size_t maxZeroScoreRun = MAX(10, MIN(100, epochs.num >> 3));
- size_t zeroScoreRun = 0;
- size_t epoch;
- DISPLAYLEVEL(2, "Breaking content into %u epochs of size %u\n",
- (U32)epochs.num, (U32)epochs.size);
- /* Loop through the epochs until there are no more segments or the dictionary
- * is full.
- */
- for (epoch = 0; tail > 0; epoch = (epoch + 1) % epochs.num) {
- const U32 epochBegin = (U32)(epoch * epochs.size);
- const U32 epochEnd = epochBegin + epochs.size;
- size_t segmentSize;
- /* Select a segment */
- COVER_segment_t segment = COVER_selectSegment(
- ctx, freqs, activeDmers, epochBegin, epochEnd, parameters);
- /* If the segment covers no dmers, then we are out of content.
- * There may be new content in other epochs, for continue for some time.
- */
- if (segment.score == 0) {
- if (++zeroScoreRun >= maxZeroScoreRun) {
- break;
- }
- continue;
- }
- zeroScoreRun = 0;
- /* Trim the segment if necessary and if it is too small then we are done */
- segmentSize = MIN(segment.end - segment.begin + parameters.d - 1, tail);
- if (segmentSize < parameters.d) {
- break;
- }
- /* We fill the dictionary from the back to allow the best segments to be
- * referenced with the smallest offsets.
- */
- tail -= segmentSize;
- memcpy(dict + tail, ctx->samples + segment.begin, segmentSize);
- DISPLAYUPDATE(
- 2, "\r%u%% ",
- (unsigned)(((dictBufferCapacity - tail) * 100) / dictBufferCapacity));
- }
- DISPLAYLEVEL(2, "\r%79s\r", "");
- return tail;
-}
-
-ZDICTLIB_API size_t ZDICT_trainFromBuffer_cover(
- void *dictBuffer, size_t dictBufferCapacity,
- const void *samplesBuffer, const size_t *samplesSizes, unsigned nbSamples,
- ZDICT_cover_params_t parameters)
-{
- BYTE* const dict = (BYTE*)dictBuffer;
- COVER_ctx_t ctx;
- COVER_map_t activeDmers;
- parameters.splitPoint = 1.0;
- /* Initialize global data */
- g_displayLevel = parameters.zParams.notificationLevel;
- /* Checks */
- if (!COVER_checkParameters(parameters, dictBufferCapacity)) {
- DISPLAYLEVEL(1, "Cover parameters incorrect\n");
- return ERROR(parameter_outOfBound);
- }
- if (nbSamples == 0) {
- DISPLAYLEVEL(1, "Cover must have at least one input file\n");
- return ERROR(srcSize_wrong);
- }
- if (dictBufferCapacity < ZDICT_DICTSIZE_MIN) {
- DISPLAYLEVEL(1, "dictBufferCapacity must be at least %u\n",
- ZDICT_DICTSIZE_MIN);
- return ERROR(dstSize_tooSmall);
- }
- /* Initialize context and activeDmers */
- {
- size_t const initVal = COVER_ctx_init(&ctx, samplesBuffer, samplesSizes, nbSamples,
- parameters.d, parameters.splitPoint);
- if (ZSTD_isError(initVal)) {
- return initVal;
- }
- }
- COVER_warnOnSmallCorpus(dictBufferCapacity, ctx.suffixSize, g_displayLevel);
- if (!COVER_map_init(&activeDmers, parameters.k - parameters.d + 1)) {
- DISPLAYLEVEL(1, "Failed to allocate dmer map: out of memory\n");
- COVER_ctx_destroy(&ctx);
- return ERROR(memory_allocation);
- }
-
- DISPLAYLEVEL(2, "Building dictionary\n");
- {
- const size_t tail =
- COVER_buildDictionary(&ctx, ctx.freqs, &activeDmers, dictBuffer,
- dictBufferCapacity, parameters);
- const size_t dictionarySize = ZDICT_finalizeDictionary(
- dict, dictBufferCapacity, dict + tail, dictBufferCapacity - tail,
- samplesBuffer, samplesSizes, nbSamples, parameters.zParams);
- if (!ZSTD_isError(dictionarySize)) {
- DISPLAYLEVEL(2, "Constructed dictionary of size %u\n",
- (unsigned)dictionarySize);
- }
- COVER_ctx_destroy(&ctx);
- COVER_map_destroy(&activeDmers);
- return dictionarySize;
- }
-}
-
-
-
-size_t COVER_checkTotalCompressedSize(const ZDICT_cover_params_t parameters,
- const size_t *samplesSizes, const BYTE *samples,
- size_t *offsets,
- size_t nbTrainSamples, size_t nbSamples,
- BYTE *const dict, size_t dictBufferCapacity) {
- size_t totalCompressedSize = ERROR(GENERIC);
- /* Pointers */
- ZSTD_CCtx *cctx;
- ZSTD_CDict *cdict;
- void *dst;
- /* Local variables */
- size_t dstCapacity;
- size_t i;
- /* Allocate dst with enough space to compress the maximum sized sample */
- {
- size_t maxSampleSize = 0;
- i = parameters.splitPoint < 1.0 ? nbTrainSamples : 0;
- for (; i < nbSamples; ++i) {
- maxSampleSize = MAX(samplesSizes[i], maxSampleSize);
- }
- dstCapacity = ZSTD_compressBound(maxSampleSize);
- dst = malloc(dstCapacity);
- }
- /* Create the cctx and cdict */
- cctx = ZSTD_createCCtx();
- cdict = ZSTD_createCDict(dict, dictBufferCapacity,
- parameters.zParams.compressionLevel);
- if (!dst || !cctx || !cdict) {
- goto _compressCleanup;
- }
- /* Compress each sample and sum their sizes (or error) */
- totalCompressedSize = dictBufferCapacity;
- i = parameters.splitPoint < 1.0 ? nbTrainSamples : 0;
- for (; i < nbSamples; ++i) {
- const size_t size = ZSTD_compress_usingCDict(
- cctx, dst, dstCapacity, samples + offsets[i],
- samplesSizes[i], cdict);
- if (ZSTD_isError(size)) {
- totalCompressedSize = size;
- goto _compressCleanup;
- }
- totalCompressedSize += size;
- }
-_compressCleanup:
- ZSTD_freeCCtx(cctx);
- ZSTD_freeCDict(cdict);
- if (dst) {
- free(dst);
- }
- return totalCompressedSize;
-}
-
-
-/**
- * Initialize the `COVER_best_t`.
- */
-void COVER_best_init(COVER_best_t *best) {
- if (best==NULL) return; /* compatible with init on NULL */
- (void)ZSTD_pthread_mutex_init(&best->mutex, NULL);
- (void)ZSTD_pthread_cond_init(&best->cond, NULL);
- best->liveJobs = 0;
- best->dict = NULL;
- best->dictSize = 0;
- best->compressedSize = (size_t)-1;
- memset(&best->parameters, 0, sizeof(best->parameters));
-}
-
-/**
- * Wait until liveJobs == 0.
- */
-void COVER_best_wait(COVER_best_t *best) {
- if (!best) {
- return;
- }
- ZSTD_pthread_mutex_lock(&best->mutex);
- while (best->liveJobs != 0) {
- ZSTD_pthread_cond_wait(&best->cond, &best->mutex);
- }
- ZSTD_pthread_mutex_unlock(&best->mutex);
-}
-
-/**
- * Call COVER_best_wait() and then destroy the COVER_best_t.
- */
-void COVER_best_destroy(COVER_best_t *best) {
- if (!best) {
- return;
- }
- COVER_best_wait(best);
- if (best->dict) {
- free(best->dict);
- }
- ZSTD_pthread_mutex_destroy(&best->mutex);
- ZSTD_pthread_cond_destroy(&best->cond);
-}
-
-/**
- * Called when a thread is about to be launched.
- * Increments liveJobs.
- */
-void COVER_best_start(COVER_best_t *best) {
- if (!best) {
- return;
- }
- ZSTD_pthread_mutex_lock(&best->mutex);
- ++best->liveJobs;
- ZSTD_pthread_mutex_unlock(&best->mutex);
-}
-
-/**
- * Called when a thread finishes executing, both on error or success.
- * Decrements liveJobs and signals any waiting threads if liveJobs == 0.
- * If this dictionary is the best so far save it and its parameters.
- */
-void COVER_best_finish(COVER_best_t *best, ZDICT_cover_params_t parameters,
- COVER_dictSelection_t selection) {
- void* dict = selection.dictContent;
- size_t compressedSize = selection.totalCompressedSize;
- size_t dictSize = selection.dictSize;
- if (!best) {
- return;
- }
- {
- size_t liveJobs;
- ZSTD_pthread_mutex_lock(&best->mutex);
- --best->liveJobs;
- liveJobs = best->liveJobs;
- /* If the new dictionary is better */
- if (compressedSize < best->compressedSize) {
- /* Allocate space if necessary */
- if (!best->dict || best->dictSize < dictSize) {
- if (best->dict) {
- free(best->dict);
- }
- best->dict = malloc(dictSize);
- if (!best->dict) {
- best->compressedSize = ERROR(GENERIC);
- best->dictSize = 0;
- ZSTD_pthread_cond_signal(&best->cond);
- ZSTD_pthread_mutex_unlock(&best->mutex);
- return;
- }
- }
- /* Save the dictionary, parameters, and size */
- if (!dict) {
- return;
- }
- memcpy(best->dict, dict, dictSize);
- best->dictSize = dictSize;
- best->parameters = parameters;
- best->compressedSize = compressedSize;
- }
- if (liveJobs == 0) {
- ZSTD_pthread_cond_broadcast(&best->cond);
- }
- ZSTD_pthread_mutex_unlock(&best->mutex);
- }
-}
-
-COVER_dictSelection_t COVER_dictSelectionError(size_t error) {
- COVER_dictSelection_t selection = { NULL, 0, error };
- return selection;
-}
-
-unsigned COVER_dictSelectionIsError(COVER_dictSelection_t selection) {
- return (ZSTD_isError(selection.totalCompressedSize) || !selection.dictContent);
-}
-
-void COVER_dictSelectionFree(COVER_dictSelection_t selection){
- free(selection.dictContent);
-}
-
-COVER_dictSelection_t COVER_selectDict(BYTE* customDictContent,
- size_t dictContentSize, const BYTE* samplesBuffer, const size_t* samplesSizes, unsigned nbFinalizeSamples,
- size_t nbCheckSamples, size_t nbSamples, ZDICT_cover_params_t params, size_t* offsets, size_t totalCompressedSize) {
-
- size_t largestDict = 0;
- size_t largestCompressed = 0;
- BYTE* customDictContentEnd = customDictContent + dictContentSize;
-
- BYTE * largestDictbuffer = (BYTE *)malloc(dictContentSize);
- BYTE * candidateDictBuffer = (BYTE *)malloc(dictContentSize);
- double regressionTolerance = ((double)params.shrinkDictMaxRegression / 100.0) + 1.00;
-
- if (!largestDictbuffer || !candidateDictBuffer) {
- free(largestDictbuffer);
- free(candidateDictBuffer);
- return COVER_dictSelectionError(dictContentSize);
- }
-
- /* Initial dictionary size and compressed size */
- memcpy(largestDictbuffer, customDictContent, dictContentSize);
- dictContentSize = ZDICT_finalizeDictionary(
- largestDictbuffer, dictContentSize, customDictContent, dictContentSize,
- samplesBuffer, samplesSizes, nbFinalizeSamples, params.zParams);
-
- if (ZDICT_isError(dictContentSize)) {
- free(largestDictbuffer);
- free(candidateDictBuffer);
- return COVER_dictSelectionError(dictContentSize);
- }
-
- totalCompressedSize = COVER_checkTotalCompressedSize(params, samplesSizes,
- samplesBuffer, offsets,
- nbCheckSamples, nbSamples,
- largestDictbuffer, dictContentSize);
-
- if (ZSTD_isError(totalCompressedSize)) {
- free(largestDictbuffer);
- free(candidateDictBuffer);
- return COVER_dictSelectionError(totalCompressedSize);
- }
-
- if (params.shrinkDict == 0) {
- COVER_dictSelection_t selection = { largestDictbuffer, dictContentSize, totalCompressedSize };
- free(candidateDictBuffer);
- return selection;
- }
-
- largestDict = dictContentSize;
- largestCompressed = totalCompressedSize;
- dictContentSize = ZDICT_DICTSIZE_MIN;
-
- /* Largest dict is initially at least ZDICT_DICTSIZE_MIN */
- while (dictContentSize < largestDict) {
- memcpy(candidateDictBuffer, largestDictbuffer, largestDict);
- dictContentSize = ZDICT_finalizeDictionary(
- candidateDictBuffer, dictContentSize, customDictContentEnd - dictContentSize, dictContentSize,
- samplesBuffer, samplesSizes, nbFinalizeSamples, params.zParams);
-
- if (ZDICT_isError(dictContentSize)) {
- free(largestDictbuffer);
- free(candidateDictBuffer);
- return COVER_dictSelectionError(dictContentSize);
-
- }
-
- totalCompressedSize = COVER_checkTotalCompressedSize(params, samplesSizes,
- samplesBuffer, offsets,
- nbCheckSamples, nbSamples,
- candidateDictBuffer, dictContentSize);
-
- if (ZSTD_isError(totalCompressedSize)) {
- free(largestDictbuffer);
- free(candidateDictBuffer);
- return COVER_dictSelectionError(totalCompressedSize);
- }
-
- if (totalCompressedSize <= largestCompressed * regressionTolerance) {
- COVER_dictSelection_t selection = { candidateDictBuffer, dictContentSize, totalCompressedSize };
- free(largestDictbuffer);
- return selection;
- }
- dictContentSize *= 2;
- }
- dictContentSize = largestDict;
- totalCompressedSize = largestCompressed;
- {
- COVER_dictSelection_t selection = { largestDictbuffer, dictContentSize, totalCompressedSize };
- free(candidateDictBuffer);
- return selection;
- }
-}
-
-/**
- * Parameters for COVER_tryParameters().
- */
-typedef struct COVER_tryParameters_data_s {
- const COVER_ctx_t *ctx;
- COVER_best_t *best;
- size_t dictBufferCapacity;
- ZDICT_cover_params_t parameters;
-} COVER_tryParameters_data_t;
-
-/**
- * Tries a set of parameters and updates the COVER_best_t with the results.
- * This function is thread safe if zstd is compiled with multithreaded support.
- * It takes its parameters as an *OWNING* opaque pointer to support threading.
- */
-static void COVER_tryParameters(void *opaque) {
- /* Save parameters as local variables */
- COVER_tryParameters_data_t *const data = (COVER_tryParameters_data_t *)opaque;
- const COVER_ctx_t *const ctx = data->ctx;
- const ZDICT_cover_params_t parameters = data->parameters;
- size_t dictBufferCapacity = data->dictBufferCapacity;
- size_t totalCompressedSize = ERROR(GENERIC);
- /* Allocate space for hash table, dict, and freqs */
- COVER_map_t activeDmers;
- BYTE *const dict = (BYTE * const)malloc(dictBufferCapacity);
- COVER_dictSelection_t selection = COVER_dictSelectionError(ERROR(GENERIC));
- U32 *freqs = (U32 *)malloc(ctx->suffixSize * sizeof(U32));
- if (!COVER_map_init(&activeDmers, parameters.k - parameters.d + 1)) {
- DISPLAYLEVEL(1, "Failed to allocate dmer map: out of memory\n");
- goto _cleanup;
- }
- if (!dict || !freqs) {
- DISPLAYLEVEL(1, "Failed to allocate buffers: out of memory\n");
- goto _cleanup;
- }
- /* Copy the frequencies because we need to modify them */
- memcpy(freqs, ctx->freqs, ctx->suffixSize * sizeof(U32));
- /* Build the dictionary */
- {
- const size_t tail = COVER_buildDictionary(ctx, freqs, &activeDmers, dict,
- dictBufferCapacity, parameters);
- selection = COVER_selectDict(dict + tail, dictBufferCapacity - tail,
- ctx->samples, ctx->samplesSizes, (unsigned)ctx->nbTrainSamples, ctx->nbTrainSamples, ctx->nbSamples, parameters, ctx->offsets,
- totalCompressedSize);
-
- if (COVER_dictSelectionIsError(selection)) {
- DISPLAYLEVEL(1, "Failed to select dictionary\n");
- goto _cleanup;
- }
- }
-_cleanup:
- free(dict);
- COVER_best_finish(data->best, parameters, selection);
- free(data);
- COVER_map_destroy(&activeDmers);
- COVER_dictSelectionFree(selection);
- if (freqs) {
- free(freqs);
- }
-}
-
-ZDICTLIB_API size_t ZDICT_optimizeTrainFromBuffer_cover(
- void *dictBuffer, size_t dictBufferCapacity, const void *samplesBuffer,
- const size_t *samplesSizes, unsigned nbSamples,
- ZDICT_cover_params_t *parameters) {
- /* constants */
- const unsigned nbThreads = parameters->nbThreads;
- const double splitPoint =
- parameters->splitPoint <= 0.0 ? DEFAULT_SPLITPOINT : parameters->splitPoint;
- const unsigned kMinD = parameters->d == 0 ? 6 : parameters->d;
- const unsigned kMaxD = parameters->d == 0 ? 8 : parameters->d;
- const unsigned kMinK = parameters->k == 0 ? 50 : parameters->k;
- const unsigned kMaxK = parameters->k == 0 ? 2000 : parameters->k;
- const unsigned kSteps = parameters->steps == 0 ? 40 : parameters->steps;
- const unsigned kStepSize = MAX((kMaxK - kMinK) / kSteps, 1);
- const unsigned kIterations =
- (1 + (kMaxD - kMinD) / 2) * (1 + (kMaxK - kMinK) / kStepSize);
- const unsigned shrinkDict = 0;
- /* Local variables */
- const int displayLevel = parameters->zParams.notificationLevel;
- unsigned iteration = 1;
- unsigned d;
- unsigned k;
- COVER_best_t best;
- POOL_ctx *pool = NULL;
- int warned = 0;
-
- /* Checks */
- if (splitPoint <= 0 || splitPoint > 1) {
- LOCALDISPLAYLEVEL(displayLevel, 1, "Incorrect parameters\n");
- return ERROR(parameter_outOfBound);
- }
- if (kMinK < kMaxD || kMaxK < kMinK) {
- LOCALDISPLAYLEVEL(displayLevel, 1, "Incorrect parameters\n");
- return ERROR(parameter_outOfBound);
- }
- if (nbSamples == 0) {
- DISPLAYLEVEL(1, "Cover must have at least one input file\n");
- return ERROR(srcSize_wrong);
- }
- if (dictBufferCapacity < ZDICT_DICTSIZE_MIN) {
- DISPLAYLEVEL(1, "dictBufferCapacity must be at least %u\n",
- ZDICT_DICTSIZE_MIN);
- return ERROR(dstSize_tooSmall);
- }
- if (nbThreads > 1) {
- pool = POOL_create(nbThreads, 1);
- if (!pool) {
- return ERROR(memory_allocation);
- }
- }
- /* Initialization */
- COVER_best_init(&best);
- /* Turn down global display level to clean up display at level 2 and below */
- g_displayLevel = displayLevel == 0 ? 0 : displayLevel - 1;
- /* Loop through d first because each new value needs a new context */
- LOCALDISPLAYLEVEL(displayLevel, 2, "Trying %u different sets of parameters\n",
- kIterations);
- for (d = kMinD; d <= kMaxD; d += 2) {
- /* Initialize the context for this value of d */
- COVER_ctx_t ctx;
- LOCALDISPLAYLEVEL(displayLevel, 3, "d=%u\n", d);
- {
- const size_t initVal = COVER_ctx_init(&ctx, samplesBuffer, samplesSizes, nbSamples, d, splitPoint);
- if (ZSTD_isError(initVal)) {
- LOCALDISPLAYLEVEL(displayLevel, 1, "Failed to initialize context\n");
- COVER_best_destroy(&best);
- POOL_free(pool);
- return initVal;
- }
- }
- if (!warned) {
- COVER_warnOnSmallCorpus(dictBufferCapacity, ctx.suffixSize, displayLevel);
- warned = 1;
- }
- /* Loop through k reusing the same context */
- for (k = kMinK; k <= kMaxK; k += kStepSize) {
- /* Prepare the arguments */
- COVER_tryParameters_data_t *data = (COVER_tryParameters_data_t *)malloc(
- sizeof(COVER_tryParameters_data_t));
- LOCALDISPLAYLEVEL(displayLevel, 3, "k=%u\n", k);
- if (!data) {
- LOCALDISPLAYLEVEL(displayLevel, 1, "Failed to allocate parameters\n");
- COVER_best_destroy(&best);
- COVER_ctx_destroy(&ctx);
- POOL_free(pool);
- return ERROR(memory_allocation);
- }
- data->ctx = &ctx;
- data->best = &best;
- data->dictBufferCapacity = dictBufferCapacity;
- data->parameters = *parameters;
- data->parameters.k = k;
- data->parameters.d = d;
- data->parameters.splitPoint = splitPoint;
- data->parameters.steps = kSteps;
- data->parameters.shrinkDict = shrinkDict;
- data->parameters.zParams.notificationLevel = g_displayLevel;
- /* Check the parameters */
- if (!COVER_checkParameters(data->parameters, dictBufferCapacity)) {
- DISPLAYLEVEL(1, "Cover parameters incorrect\n");
- free(data);
- continue;
- }
- /* Call the function and pass ownership of data to it */
- COVER_best_start(&best);
- if (pool) {
- POOL_add(pool, &COVER_tryParameters, data);
- } else {
- COVER_tryParameters(data);
- }
- /* Print status */
- LOCALDISPLAYUPDATE(displayLevel, 2, "\r%u%% ",
- (unsigned)((iteration * 100) / kIterations));
- ++iteration;
- }
- COVER_best_wait(&best);
- COVER_ctx_destroy(&ctx);
- }
- LOCALDISPLAYLEVEL(displayLevel, 2, "\r%79s\r", "");
- /* Fill the output buffer and parameters with output of the best parameters */
- {
- const size_t dictSize = best.dictSize;
- if (ZSTD_isError(best.compressedSize)) {
- const size_t compressedSize = best.compressedSize;
- COVER_best_destroy(&best);
- POOL_free(pool);
- return compressedSize;
- }
- *parameters = best.parameters;
- memcpy(dictBuffer, best.dict, dictSize);
- COVER_best_destroy(&best);
- POOL_free(pool);
- return dictSize;
- }
-}
diff --git a/vendor/github.com/DataDog/zstd/cover.h b/vendor/github.com/DataDog/zstd/cover.h
deleted file mode 100644
index d9e0636..0000000
--- a/vendor/github.com/DataDog/zstd/cover.h
+++ /dev/null
@@ -1,147 +0,0 @@
-#include <stdio.h> /* fprintf */
-#include <stdlib.h> /* malloc, free, qsort */
-#include <string.h> /* memset */
-#include <time.h> /* clock */
-#include "mem.h" /* read */
-#include "pool.h"
-#include "threading.h"
-#include "zstd_internal.h" /* includes zstd.h */
-#ifndef ZDICT_STATIC_LINKING_ONLY
-#define ZDICT_STATIC_LINKING_ONLY
-#endif
-#include "zdict.h"
-
-/**
- * COVER_best_t is used for two purposes:
- * 1. Synchronizing threads.
- * 2. Saving the best parameters and dictionary.
- *
- * All of the methods except COVER_best_init() are thread safe if zstd is
- * compiled with multithreaded support.
- */
-typedef struct COVER_best_s {
- ZSTD_pthread_mutex_t mutex;
- ZSTD_pthread_cond_t cond;
- size_t liveJobs;
- void *dict;
- size_t dictSize;
- ZDICT_cover_params_t parameters;
- size_t compressedSize;
-} COVER_best_t;
-
-/**
- * A segment is a range in the source as well as the score of the segment.
- */
-typedef struct {
- U32 begin;
- U32 end;
- U32 score;
-} COVER_segment_t;
-
-/**
- *Number of epochs and size of each epoch.
- */
-typedef struct {
- U32 num;
- U32 size;
-} COVER_epoch_info_t;
-
-/**
- * Struct used for the dictionary selection function.
- */
-typedef struct COVER_dictSelection {
- BYTE* dictContent;
- size_t dictSize;
- size_t totalCompressedSize;
-} COVER_dictSelection_t;
-
-/**
- * Computes the number of epochs and the size of each epoch.
- * We will make sure that each epoch gets at least 10 * k bytes.
- *
- * The COVER algorithms divide the data up into epochs of equal size and
- * select one segment from each epoch.
- *
- * @param maxDictSize The maximum allowed dictionary size.
- * @param nbDmers The number of dmers we are training on.
- * @param k The parameter k (segment size).
- * @param passes The target number of passes over the dmer corpus.
- * More passes means a better dictionary.
- */
-COVER_epoch_info_t COVER_computeEpochs(U32 maxDictSize, U32 nbDmers,
- U32 k, U32 passes);
-
-/**
- * Warns the user when their corpus is too small.
- */
-void COVER_warnOnSmallCorpus(size_t maxDictSize, size_t nbDmers, int displayLevel);
-
-/**
- * Checks total compressed size of a dictionary
- */
-size_t COVER_checkTotalCompressedSize(const ZDICT_cover_params_t parameters,
- const size_t *samplesSizes, const BYTE *samples,
- size_t *offsets,
- size_t nbTrainSamples, size_t nbSamples,
- BYTE *const dict, size_t dictBufferCapacity);
-
-/**
- * Returns the sum of the sample sizes.
- */
-size_t COVER_sum(const size_t *samplesSizes, unsigned nbSamples) ;
-
-/**
- * Initialize the `COVER_best_t`.
- */
-void COVER_best_init(COVER_best_t *best);
-
-/**
- * Wait until liveJobs == 0.
- */
-void COVER_best_wait(COVER_best_t *best);
-
-/**
- * Call COVER_best_wait() and then destroy the COVER_best_t.
- */
-void COVER_best_destroy(COVER_best_t *best);
-
-/**
- * Called when a thread is about to be launched.
- * Increments liveJobs.
- */
-void COVER_best_start(COVER_best_t *best);
-
-/**
- * Called when a thread finishes executing, both on error or success.
- * Decrements liveJobs and signals any waiting threads if liveJobs == 0.
- * If this dictionary is the best so far save it and its parameters.
- */
-void COVER_best_finish(COVER_best_t *best, ZDICT_cover_params_t parameters,
- COVER_dictSelection_t selection);
-/**
- * Error function for COVER_selectDict function. Checks if the return
- * value is an error.
- */
-unsigned COVER_dictSelectionIsError(COVER_dictSelection_t selection);
-
- /**
- * Error function for COVER_selectDict function. Returns a struct where
- * return.totalCompressedSize is a ZSTD error.
- */
-COVER_dictSelection_t COVER_dictSelectionError(size_t error);
-
-/**
- * Always call after selectDict is called to free up used memory from
- * newly created dictionary.
- */
-void COVER_dictSelectionFree(COVER_dictSelection_t selection);
-
-/**
- * Called to finalize the dictionary and select one based on whether or not
- * the shrink-dict flag was enabled. If enabled the dictionary used is the
- * smallest dictionary within a specified regression of the compressed size
- * from the largest dictionary.
- */
- COVER_dictSelection_t COVER_selectDict(BYTE* customDictContent,
- size_t dictContentSize, const BYTE* samplesBuffer, const size_t* samplesSizes, unsigned nbFinalizeSamples,
- size_t nbCheckSamples, size_t nbSamples, ZDICT_cover_params_t params, size_t* offsets, size_t totalCompressedSize);
diff --git a/vendor/github.com/DataDog/zstd/cpu.h b/vendor/github.com/DataDog/zstd/cpu.h
deleted file mode 100644
index 5f0923f..0000000
--- a/vendor/github.com/DataDog/zstd/cpu.h
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Copyright (c) 2018-present, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_COMMON_CPU_H
-#define ZSTD_COMMON_CPU_H
-
-/**
- * Implementation taken from folly/CpuId.h
- * https://github.com/facebook/folly/blob/master/folly/CpuId.h
- */
-
-#include <string.h>
-
-#include "mem.h"
-
-#ifdef _MSC_VER
-#include <intrin.h>
-#endif
-
-typedef struct {
- U32 f1c;
- U32 f1d;
- U32 f7b;
- U32 f7c;
-} ZSTD_cpuid_t;
-
-MEM_STATIC ZSTD_cpuid_t ZSTD_cpuid(void) {
- U32 f1c = 0;
- U32 f1d = 0;
- U32 f7b = 0;
- U32 f7c = 0;
-#if defined(_MSC_VER) && (defined(_M_X64) || defined(_M_IX86))
- int reg[4];
- __cpuid((int*)reg, 0);
- {
- int const n = reg[0];
- if (n >= 1) {
- __cpuid((int*)reg, 1);
- f1c = (U32)reg[2];
- f1d = (U32)reg[3];
- }
- if (n >= 7) {
- __cpuidex((int*)reg, 7, 0);
- f7b = (U32)reg[1];
- f7c = (U32)reg[2];
- }
- }
-#elif defined(__i386__) && defined(__PIC__) && !defined(__clang__) && defined(__GNUC__)
- /* The following block like the normal cpuid branch below, but gcc
- * reserves ebx for use of its pic register so we must specially
- * handle the save and restore to avoid clobbering the register
- */
- U32 n;
- __asm__(
- "pushl %%ebx\n\t"
- "cpuid\n\t"
- "popl %%ebx\n\t"
- : "=a"(n)
- : "a"(0)
- : "ecx", "edx");
- if (n >= 1) {
- U32 f1a;
- __asm__(
- "pushl %%ebx\n\t"
- "cpuid\n\t"
- "popl %%ebx\n\t"
- : "=a"(f1a), "=c"(f1c), "=d"(f1d)
- : "a"(1));
- }
- if (n >= 7) {
- __asm__(
- "pushl %%ebx\n\t"
- "cpuid\n\t"
- "movl %%ebx, %%eax\n\t"
- "popl %%ebx"
- : "=a"(f7b), "=c"(f7c)
- : "a"(7), "c"(0)
- : "edx");
- }
-#elif defined(__x86_64__) || defined(_M_X64) || defined(__i386__)
- U32 n;
- __asm__("cpuid" : "=a"(n) : "a"(0) : "ebx", "ecx", "edx");
- if (n >= 1) {
- U32 f1a;
- __asm__("cpuid" : "=a"(f1a), "=c"(f1c), "=d"(f1d) : "a"(1) : "ebx");
- }
- if (n >= 7) {
- U32 f7a;
- __asm__("cpuid"
- : "=a"(f7a), "=b"(f7b), "=c"(f7c)
- : "a"(7), "c"(0)
- : "edx");
- }
-#endif
- {
- ZSTD_cpuid_t cpuid;
- cpuid.f1c = f1c;
- cpuid.f1d = f1d;
- cpuid.f7b = f7b;
- cpuid.f7c = f7c;
- return cpuid;
- }
-}
-
-#define X(name, r, bit) \
- MEM_STATIC int ZSTD_cpuid_##name(ZSTD_cpuid_t const cpuid) { \
- return ((cpuid.r) & (1U << bit)) != 0; \
- }
-
-/* cpuid(1): Processor Info and Feature Bits. */
-#define C(name, bit) X(name, f1c, bit)
- C(sse3, 0)
- C(pclmuldq, 1)
- C(dtes64, 2)
- C(monitor, 3)
- C(dscpl, 4)
- C(vmx, 5)
- C(smx, 6)
- C(eist, 7)
- C(tm2, 8)
- C(ssse3, 9)
- C(cnxtid, 10)
- C(fma, 12)
- C(cx16, 13)
- C(xtpr, 14)
- C(pdcm, 15)
- C(pcid, 17)
- C(dca, 18)
- C(sse41, 19)
- C(sse42, 20)
- C(x2apic, 21)
- C(movbe, 22)
- C(popcnt, 23)
- C(tscdeadline, 24)
- C(aes, 25)
- C(xsave, 26)
- C(osxsave, 27)
- C(avx, 28)
- C(f16c, 29)
- C(rdrand, 30)
-#undef C
-#define D(name, bit) X(name, f1d, bit)
- D(fpu, 0)
- D(vme, 1)
- D(de, 2)
- D(pse, 3)
- D(tsc, 4)
- D(msr, 5)
- D(pae, 6)
- D(mce, 7)
- D(cx8, 8)
- D(apic, 9)
- D(sep, 11)
- D(mtrr, 12)
- D(pge, 13)
- D(mca, 14)
- D(cmov, 15)
- D(pat, 16)
- D(pse36, 17)
- D(psn, 18)
- D(clfsh, 19)
- D(ds, 21)
- D(acpi, 22)
- D(mmx, 23)
- D(fxsr, 24)
- D(sse, 25)
- D(sse2, 26)
- D(ss, 27)
- D(htt, 28)
- D(tm, 29)
- D(pbe, 31)
-#undef D
-
-/* cpuid(7): Extended Features. */
-#define B(name, bit) X(name, f7b, bit)
- B(bmi1, 3)
- B(hle, 4)
- B(avx2, 5)
- B(smep, 7)
- B(bmi2, 8)
- B(erms, 9)
- B(invpcid, 10)
- B(rtm, 11)
- B(mpx, 14)
- B(avx512f, 16)
- B(avx512dq, 17)
- B(rdseed, 18)
- B(adx, 19)
- B(smap, 20)
- B(avx512ifma, 21)
- B(pcommit, 22)
- B(clflushopt, 23)
- B(clwb, 24)
- B(avx512pf, 26)
- B(avx512er, 27)
- B(avx512cd, 28)
- B(sha, 29)
- B(avx512bw, 30)
- B(avx512vl, 31)
-#undef B
-#define C(name, bit) X(name, f7c, bit)
- C(prefetchwt1, 0)
- C(avx512vbmi, 1)
-#undef C
-
-#undef X
-
-#endif /* ZSTD_COMMON_CPU_H */
diff --git a/vendor/github.com/DataDog/zstd/debug.c b/vendor/github.com/DataDog/zstd/debug.c
deleted file mode 100644
index 3ebdd1c..0000000
--- a/vendor/github.com/DataDog/zstd/debug.c
+++ /dev/null
@@ -1,44 +0,0 @@
-/* ******************************************************************
- debug
- Part of FSE library
- Copyright (C) 2013-present, Yann Collet.
-
- BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
- * Redistributions of source code must retain the above copyright
- notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
- copyright notice, this list of conditions and the following disclaimer
- in the documentation and/or other materials provided with the
- distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
- "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
- LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
- A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
- OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
- OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
- You can contact the author at :
- - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-
-
-/*
- * This module only hosts one global variable
- * which can be used to dynamically influence the verbosity of traces,
- * such as DEBUGLOG and RAWLOG
- */
-
-#include "debug.h"
-
-int g_debuglevel = DEBUGLEVEL;
diff --git a/vendor/github.com/DataDog/zstd/debug.h b/vendor/github.com/DataDog/zstd/debug.h
deleted file mode 100644
index b4fc89d..0000000
--- a/vendor/github.com/DataDog/zstd/debug.h
+++ /dev/null
@@ -1,134 +0,0 @@
-/* ******************************************************************
- debug
- Part of FSE library
- Copyright (C) 2013-present, Yann Collet.
-
- BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
- * Redistributions of source code must retain the above copyright
- notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
- copyright notice, this list of conditions and the following disclaimer
- in the documentation and/or other materials provided with the
- distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
- "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
- LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
- A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
- OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
- OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
- You can contact the author at :
- - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-
-
-/*
- * The purpose of this header is to enable debug functions.
- * They regroup assert(), DEBUGLOG() and RAWLOG() for run-time,
- * and DEBUG_STATIC_ASSERT() for compile-time.
- *
- * By default, DEBUGLEVEL==0, which means run-time debug is disabled.
- *
- * Level 1 enables assert() only.
- * Starting level 2, traces can be generated and pushed to stderr.
- * The higher the level, the more verbose the traces.
- *
- * It's possible to dynamically adjust level using variable g_debug_level,
- * which is only declared if DEBUGLEVEL>=2,
- * and is a global variable, not multi-thread protected (use with care)
- */
-
-#ifndef DEBUG_H_12987983217
-#define DEBUG_H_12987983217
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/* static assert is triggered at compile time, leaving no runtime artefact.
- * static assert only works with compile-time constants.
- * Also, this variant can only be used inside a function. */
-#define DEBUG_STATIC_ASSERT(c) (void)sizeof(char[(c) ? 1 : -1])
-
-
-/* DEBUGLEVEL is expected to be defined externally,
- * typically through compiler command line.
- * Value must be a number. */
-#ifndef DEBUGLEVEL
-# define DEBUGLEVEL 0
-#endif
-
-
-/* DEBUGFILE can be defined externally,
- * typically through compiler command line.
- * note : currently useless.
- * Value must be stderr or stdout */
-#ifndef DEBUGFILE
-# define DEBUGFILE stderr
-#endif
-
-
-/* recommended values for DEBUGLEVEL :
- * 0 : release mode, no debug, all run-time checks disabled
- * 1 : enables assert() only, no display
- * 2 : reserved, for currently active debug path
- * 3 : events once per object lifetime (CCtx, CDict, etc.)
- * 4 : events once per frame
- * 5 : events once per block
- * 6 : events once per sequence (verbose)
- * 7+: events at every position (*very* verbose)
- *
- * It's generally inconvenient to output traces > 5.
- * In which case, it's possible to selectively trigger high verbosity levels
- * by modifying g_debug_level.
- */
-
-#if (DEBUGLEVEL>=1)
-# include <assert.h>
-#else
-# ifndef assert /* assert may be already defined, due to prior #include <assert.h> */
-# define assert(condition) ((void)0) /* disable assert (default) */
-# endif
-#endif
-
-#if (DEBUGLEVEL>=2)
-# include <stdio.h>
-extern int g_debuglevel; /* the variable is only declared,
- it actually lives in debug.c,
- and is shared by the whole process.
- It's not thread-safe.
- It's useful when enabling very verbose levels
- on selective conditions (such as position in src) */
-
-# define RAWLOG(l, ...) { \
- if (l<=g_debuglevel) { \
- fprintf(stderr, __VA_ARGS__); \
- } }
-# define DEBUGLOG(l, ...) { \
- if (l<=g_debuglevel) { \
- fprintf(stderr, __FILE__ ": " __VA_ARGS__); \
- fprintf(stderr, " \n"); \
- } }
-#else
-# define RAWLOG(l, ...) {} /* disabled */
-# define DEBUGLOG(l, ...) {} /* disabled */
-#endif
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* DEBUG_H_12987983217 */
diff --git a/vendor/github.com/DataDog/zstd/divsufsort.c b/vendor/github.com/DataDog/zstd/divsufsort.c
deleted file mode 100644
index ead9220..0000000
--- a/vendor/github.com/DataDog/zstd/divsufsort.c
+++ /dev/null
@@ -1,1913 +0,0 @@
-/*
- * divsufsort.c for libdivsufsort-lite
- * Copyright (c) 2003-2008 Yuta Mori All Rights Reserved.
- *
- * Permission is hereby granted, free of charge, to any person
- * obtaining a copy of this software and associated documentation
- * files (the "Software"), to deal in the Software without
- * restriction, including without limitation the rights to use,
- * copy, modify, merge, publish, distribute, sublicense, and/or sell
- * copies of the Software, and to permit persons to whom the
- * Software is furnished to do so, subject to the following
- * conditions:
- *
- * The above copyright notice and this permission notice shall be
- * included in all copies or substantial portions of the Software.
- *
- * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
- * EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
- * OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
- * NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
- * HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
- * WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
- * FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
- * OTHER DEALINGS IN THE SOFTWARE.
- */
-
-/*- Compiler specifics -*/
-#ifdef __clang__
-#pragma clang diagnostic ignored "-Wshorten-64-to-32"
-#endif
-
-#if defined(_MSC_VER)
-# pragma warning(disable : 4244)
-# pragma warning(disable : 4127) /* C4127 : Condition expression is constant */
-#endif
-
-
-/*- Dependencies -*/
-#include <assert.h>
-#include <stdio.h>
-#include <stdlib.h>
-
-#include "divsufsort.h"
-
-/*- Constants -*/
-#if defined(INLINE)
-# undef INLINE
-#endif
-#if !defined(INLINE)
-# define INLINE __inline
-#endif
-#if defined(ALPHABET_SIZE) && (ALPHABET_SIZE < 1)
-# undef ALPHABET_SIZE
-#endif
-#if !defined(ALPHABET_SIZE)
-# define ALPHABET_SIZE (256)
-#endif
-#define BUCKET_A_SIZE (ALPHABET_SIZE)
-#define BUCKET_B_SIZE (ALPHABET_SIZE * ALPHABET_SIZE)
-#if defined(SS_INSERTIONSORT_THRESHOLD)
-# if SS_INSERTIONSORT_THRESHOLD < 1
-# undef SS_INSERTIONSORT_THRESHOLD
-# define SS_INSERTIONSORT_THRESHOLD (1)
-# endif
-#else
-# define SS_INSERTIONSORT_THRESHOLD (8)
-#endif
-#if defined(SS_BLOCKSIZE)
-# if SS_BLOCKSIZE < 0
-# undef SS_BLOCKSIZE
-# define SS_BLOCKSIZE (0)
-# elif 32768 <= SS_BLOCKSIZE
-# undef SS_BLOCKSIZE
-# define SS_BLOCKSIZE (32767)
-# endif
-#else
-# define SS_BLOCKSIZE (1024)
-#endif
-/* minstacksize = log(SS_BLOCKSIZE) / log(3) * 2 */
-#if SS_BLOCKSIZE == 0
-# define SS_MISORT_STACKSIZE (96)
-#elif SS_BLOCKSIZE <= 4096
-# define SS_MISORT_STACKSIZE (16)
-#else
-# define SS_MISORT_STACKSIZE (24)
-#endif
-#define SS_SMERGE_STACKSIZE (32)
-#define TR_INSERTIONSORT_THRESHOLD (8)
-#define TR_STACKSIZE (64)
-
-
-/*- Macros -*/
-#ifndef SWAP
-# define SWAP(_a, _b) do { t = (_a); (_a) = (_b); (_b) = t; } while(0)
-#endif /* SWAP */
-#ifndef MIN
-# define MIN(_a, _b) (((_a) < (_b)) ? (_a) : (_b))
-#endif /* MIN */
-#ifndef MAX
-# define MAX(_a, _b) (((_a) > (_b)) ? (_a) : (_b))
-#endif /* MAX */
-#define STACK_PUSH(_a, _b, _c, _d)\
- do {\
- assert(ssize < STACK_SIZE);\
- stack[ssize].a = (_a), stack[ssize].b = (_b),\
- stack[ssize].c = (_c), stack[ssize++].d = (_d);\
- } while(0)
-#define STACK_PUSH5(_a, _b, _c, _d, _e)\
- do {\
- assert(ssize < STACK_SIZE);\
- stack[ssize].a = (_a), stack[ssize].b = (_b),\
- stack[ssize].c = (_c), stack[ssize].d = (_d), stack[ssize++].e = (_e);\
- } while(0)
-#define STACK_POP(_a, _b, _c, _d)\
- do {\
- assert(0 <= ssize);\
- if(ssize == 0) { return; }\
- (_a) = stack[--ssize].a, (_b) = stack[ssize].b,\
- (_c) = stack[ssize].c, (_d) = stack[ssize].d;\
- } while(0)
-#define STACK_POP5(_a, _b, _c, _d, _e)\
- do {\
- assert(0 <= ssize);\
- if(ssize == 0) { return; }\
- (_a) = stack[--ssize].a, (_b) = stack[ssize].b,\
- (_c) = stack[ssize].c, (_d) = stack[ssize].d, (_e) = stack[ssize].e;\
- } while(0)
-#define BUCKET_A(_c0) bucket_A[(_c0)]
-#if ALPHABET_SIZE == 256
-#define BUCKET_B(_c0, _c1) (bucket_B[((_c1) << 8) | (_c0)])
-#define BUCKET_BSTAR(_c0, _c1) (bucket_B[((_c0) << 8) | (_c1)])
-#else
-#define BUCKET_B(_c0, _c1) (bucket_B[(_c1) * ALPHABET_SIZE + (_c0)])
-#define BUCKET_BSTAR(_c0, _c1) (bucket_B[(_c0) * ALPHABET_SIZE + (_c1)])
-#endif
-
-
-/*- Private Functions -*/
-
-static const int lg_table[256]= {
- -1,0,1,1,2,2,2,2,3,3,3,3,3,3,3,3,4,4,4,4,4,4,4,4,4,4,4,4,4,4,4,4,
- 5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,
- 6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,
- 6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,
- 7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,
- 7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,
- 7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,
- 7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7
-};
-
-#if (SS_BLOCKSIZE == 0) || (SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE)
-
-static INLINE
-int
-ss_ilg(int n) {
-#if SS_BLOCKSIZE == 0
- return (n & 0xffff0000) ?
- ((n & 0xff000000) ?
- 24 + lg_table[(n >> 24) & 0xff] :
- 16 + lg_table[(n >> 16) & 0xff]) :
- ((n & 0x0000ff00) ?
- 8 + lg_table[(n >> 8) & 0xff] :
- 0 + lg_table[(n >> 0) & 0xff]);
-#elif SS_BLOCKSIZE < 256
- return lg_table[n];
-#else
- return (n & 0xff00) ?
- 8 + lg_table[(n >> 8) & 0xff] :
- 0 + lg_table[(n >> 0) & 0xff];
-#endif
-}
-
-#endif /* (SS_BLOCKSIZE == 0) || (SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE) */
-
-#if SS_BLOCKSIZE != 0
-
-static const int sqq_table[256] = {
- 0, 16, 22, 27, 32, 35, 39, 42, 45, 48, 50, 53, 55, 57, 59, 61,
- 64, 65, 67, 69, 71, 73, 75, 76, 78, 80, 81, 83, 84, 86, 87, 89,
- 90, 91, 93, 94, 96, 97, 98, 99, 101, 102, 103, 104, 106, 107, 108, 109,
-110, 112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126,
-128, 128, 129, 130, 131, 132, 133, 134, 135, 136, 137, 138, 139, 140, 141, 142,
-143, 144, 144, 145, 146, 147, 148, 149, 150, 150, 151, 152, 153, 154, 155, 155,
-156, 157, 158, 159, 160, 160, 161, 162, 163, 163, 164, 165, 166, 167, 167, 168,
-169, 170, 170, 171, 172, 173, 173, 174, 175, 176, 176, 177, 178, 178, 179, 180,
-181, 181, 182, 183, 183, 184, 185, 185, 186, 187, 187, 188, 189, 189, 190, 191,
-192, 192, 193, 193, 194, 195, 195, 196, 197, 197, 198, 199, 199, 200, 201, 201,
-202, 203, 203, 204, 204, 205, 206, 206, 207, 208, 208, 209, 209, 210, 211, 211,
-212, 212, 213, 214, 214, 215, 215, 216, 217, 217, 218, 218, 219, 219, 220, 221,
-221, 222, 222, 223, 224, 224, 225, 225, 226, 226, 227, 227, 228, 229, 229, 230,
-230, 231, 231, 232, 232, 233, 234, 234, 235, 235, 236, 236, 237, 237, 238, 238,
-239, 240, 240, 241, 241, 242, 242, 243, 243, 244, 244, 245, 245, 246, 246, 247,
-247, 248, 248, 249, 249, 250, 250, 251, 251, 252, 252, 253, 253, 254, 254, 255
-};
-
-static INLINE
-int
-ss_isqrt(int x) {
- int y, e;
-
- if(x >= (SS_BLOCKSIZE * SS_BLOCKSIZE)) { return SS_BLOCKSIZE; }
- e = (x & 0xffff0000) ?
- ((x & 0xff000000) ?
- 24 + lg_table[(x >> 24) & 0xff] :
- 16 + lg_table[(x >> 16) & 0xff]) :
- ((x & 0x0000ff00) ?
- 8 + lg_table[(x >> 8) & 0xff] :
- 0 + lg_table[(x >> 0) & 0xff]);
-
- if(e >= 16) {
- y = sqq_table[x >> ((e - 6) - (e & 1))] << ((e >> 1) - 7);
- if(e >= 24) { y = (y + 1 + x / y) >> 1; }
- y = (y + 1 + x / y) >> 1;
- } else if(e >= 8) {
- y = (sqq_table[x >> ((e - 6) - (e & 1))] >> (7 - (e >> 1))) + 1;
- } else {
- return sqq_table[x] >> 4;
- }
-
- return (x < (y * y)) ? y - 1 : y;
-}
-
-#endif /* SS_BLOCKSIZE != 0 */
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Compares two suffixes. */
-static INLINE
-int
-ss_compare(const unsigned char *T,
- const int *p1, const int *p2,
- int depth) {
- const unsigned char *U1, *U2, *U1n, *U2n;
-
- for(U1 = T + depth + *p1,
- U2 = T + depth + *p2,
- U1n = T + *(p1 + 1) + 2,
- U2n = T + *(p2 + 1) + 2;
- (U1 < U1n) && (U2 < U2n) && (*U1 == *U2);
- ++U1, ++U2) {
- }
-
- return U1 < U1n ?
- (U2 < U2n ? *U1 - *U2 : 1) :
- (U2 < U2n ? -1 : 0);
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-#if (SS_BLOCKSIZE != 1) && (SS_INSERTIONSORT_THRESHOLD != 1)
-
-/* Insertionsort for small size groups */
-static
-void
-ss_insertionsort(const unsigned char *T, const int *PA,
- int *first, int *last, int depth) {
- int *i, *j;
- int t;
- int r;
-
- for(i = last - 2; first <= i; --i) {
- for(t = *i, j = i + 1; 0 < (r = ss_compare(T, PA + t, PA + *j, depth));) {
- do { *(j - 1) = *j; } while((++j < last) && (*j < 0));
- if(last <= j) { break; }
- }
- if(r == 0) { *j = ~*j; }
- *(j - 1) = t;
- }
-}
-
-#endif /* (SS_BLOCKSIZE != 1) && (SS_INSERTIONSORT_THRESHOLD != 1) */
-
-
-/*---------------------------------------------------------------------------*/
-
-#if (SS_BLOCKSIZE == 0) || (SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE)
-
-static INLINE
-void
-ss_fixdown(const unsigned char *Td, const int *PA,
- int *SA, int i, int size) {
- int j, k;
- int v;
- int c, d, e;
-
- for(v = SA[i], c = Td[PA[v]]; (j = 2 * i + 1) < size; SA[i] = SA[k], i = k) {
- d = Td[PA[SA[k = j++]]];
- if(d < (e = Td[PA[SA[j]]])) { k = j; d = e; }
- if(d <= c) { break; }
- }
- SA[i] = v;
-}
-
-/* Simple top-down heapsort. */
-static
-void
-ss_heapsort(const unsigned char *Td, const int *PA, int *SA, int size) {
- int i, m;
- int t;
-
- m = size;
- if((size % 2) == 0) {
- m--;
- if(Td[PA[SA[m / 2]]] < Td[PA[SA[m]]]) { SWAP(SA[m], SA[m / 2]); }
- }
-
- for(i = m / 2 - 1; 0 <= i; --i) { ss_fixdown(Td, PA, SA, i, m); }
- if((size % 2) == 0) { SWAP(SA[0], SA[m]); ss_fixdown(Td, PA, SA, 0, m); }
- for(i = m - 1; 0 < i; --i) {
- t = SA[0], SA[0] = SA[i];
- ss_fixdown(Td, PA, SA, 0, i);
- SA[i] = t;
- }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Returns the median of three elements. */
-static INLINE
-int *
-ss_median3(const unsigned char *Td, const int *PA,
- int *v1, int *v2, int *v3) {
- int *t;
- if(Td[PA[*v1]] > Td[PA[*v2]]) { SWAP(v1, v2); }
- if(Td[PA[*v2]] > Td[PA[*v3]]) {
- if(Td[PA[*v1]] > Td[PA[*v3]]) { return v1; }
- else { return v3; }
- }
- return v2;
-}
-
-/* Returns the median of five elements. */
-static INLINE
-int *
-ss_median5(const unsigned char *Td, const int *PA,
- int *v1, int *v2, int *v3, int *v4, int *v5) {
- int *t;
- if(Td[PA[*v2]] > Td[PA[*v3]]) { SWAP(v2, v3); }
- if(Td[PA[*v4]] > Td[PA[*v5]]) { SWAP(v4, v5); }
- if(Td[PA[*v2]] > Td[PA[*v4]]) { SWAP(v2, v4); SWAP(v3, v5); }
- if(Td[PA[*v1]] > Td[PA[*v3]]) { SWAP(v1, v3); }
- if(Td[PA[*v1]] > Td[PA[*v4]]) { SWAP(v1, v4); SWAP(v3, v5); }
- if(Td[PA[*v3]] > Td[PA[*v4]]) { return v4; }
- return v3;
-}
-
-/* Returns the pivot element. */
-static INLINE
-int *
-ss_pivot(const unsigned char *Td, const int *PA, int *first, int *last) {
- int *middle;
- int t;
-
- t = last - first;
- middle = first + t / 2;
-
- if(t <= 512) {
- if(t <= 32) {
- return ss_median3(Td, PA, first, middle, last - 1);
- } else {
- t >>= 2;
- return ss_median5(Td, PA, first, first + t, middle, last - 1 - t, last - 1);
- }
- }
- t >>= 3;
- first = ss_median3(Td, PA, first, first + t, first + (t << 1));
- middle = ss_median3(Td, PA, middle - t, middle, middle + t);
- last = ss_median3(Td, PA, last - 1 - (t << 1), last - 1 - t, last - 1);
- return ss_median3(Td, PA, first, middle, last);
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Binary partition for substrings. */
-static INLINE
-int *
-ss_partition(const int *PA,
- int *first, int *last, int depth) {
- int *a, *b;
- int t;
- for(a = first - 1, b = last;;) {
- for(; (++a < b) && ((PA[*a] + depth) >= (PA[*a + 1] + 1));) { *a = ~*a; }
- for(; (a < --b) && ((PA[*b] + depth) < (PA[*b + 1] + 1));) { }
- if(b <= a) { break; }
- t = ~*b;
- *b = *a;
- *a = t;
- }
- if(first < a) { *first = ~*first; }
- return a;
-}
-
-/* Multikey introsort for medium size groups. */
-static
-void
-ss_mintrosort(const unsigned char *T, const int *PA,
- int *first, int *last,
- int depth) {
-#define STACK_SIZE SS_MISORT_STACKSIZE
- struct { int *a, *b, c; int d; } stack[STACK_SIZE];
- const unsigned char *Td;
- int *a, *b, *c, *d, *e, *f;
- int s, t;
- int ssize;
- int limit;
- int v, x = 0;
-
- for(ssize = 0, limit = ss_ilg(last - first);;) {
-
- if((last - first) <= SS_INSERTIONSORT_THRESHOLD) {
-#if 1 < SS_INSERTIONSORT_THRESHOLD
- if(1 < (last - first)) { ss_insertionsort(T, PA, first, last, depth); }
-#endif
- STACK_POP(first, last, depth, limit);
- continue;
- }
-
- Td = T + depth;
- if(limit-- == 0) { ss_heapsort(Td, PA, first, last - first); }
- if(limit < 0) {
- for(a = first + 1, v = Td[PA[*first]]; a < last; ++a) {
- if((x = Td[PA[*a]]) != v) {
- if(1 < (a - first)) { break; }
- v = x;
- first = a;
- }
- }
- if(Td[PA[*first] - 1] < v) {
- first = ss_partition(PA, first, a, depth);
- }
- if((a - first) <= (last - a)) {
- if(1 < (a - first)) {
- STACK_PUSH(a, last, depth, -1);
- last = a, depth += 1, limit = ss_ilg(a - first);
- } else {
- first = a, limit = -1;
- }
- } else {
- if(1 < (last - a)) {
- STACK_PUSH(first, a, depth + 1, ss_ilg(a - first));
- first = a, limit = -1;
- } else {
- last = a, depth += 1, limit = ss_ilg(a - first);
- }
- }
- continue;
- }
-
- /* choose pivot */
- a = ss_pivot(Td, PA, first, last);
- v = Td[PA[*a]];
- SWAP(*first, *a);
-
- /* partition */
- for(b = first; (++b < last) && ((x = Td[PA[*b]]) == v);) { }
- if(((a = b) < last) && (x < v)) {
- for(; (++b < last) && ((x = Td[PA[*b]]) <= v);) {
- if(x == v) { SWAP(*b, *a); ++a; }
- }
- }
- for(c = last; (b < --c) && ((x = Td[PA[*c]]) == v);) { }
- if((b < (d = c)) && (x > v)) {
- for(; (b < --c) && ((x = Td[PA[*c]]) >= v);) {
- if(x == v) { SWAP(*c, *d); --d; }
- }
- }
- for(; b < c;) {
- SWAP(*b, *c);
- for(; (++b < c) && ((x = Td[PA[*b]]) <= v);) {
- if(x == v) { SWAP(*b, *a); ++a; }
- }
- for(; (b < --c) && ((x = Td[PA[*c]]) >= v);) {
- if(x == v) { SWAP(*c, *d); --d; }
- }
- }
-
- if(a <= d) {
- c = b - 1;
-
- if((s = a - first) > (t = b - a)) { s = t; }
- for(e = first, f = b - s; 0 < s; --s, ++e, ++f) { SWAP(*e, *f); }
- if((s = d - c) > (t = last - d - 1)) { s = t; }
- for(e = b, f = last - s; 0 < s; --s, ++e, ++f) { SWAP(*e, *f); }
-
- a = first + (b - a), c = last - (d - c);
- b = (v <= Td[PA[*a] - 1]) ? a : ss_partition(PA, a, c, depth);
-
- if((a - first) <= (last - c)) {
- if((last - c) <= (c - b)) {
- STACK_PUSH(b, c, depth + 1, ss_ilg(c - b));
- STACK_PUSH(c, last, depth, limit);
- last = a;
- } else if((a - first) <= (c - b)) {
- STACK_PUSH(c, last, depth, limit);
- STACK_PUSH(b, c, depth + 1, ss_ilg(c - b));
- last = a;
- } else {
- STACK_PUSH(c, last, depth, limit);
- STACK_PUSH(first, a, depth, limit);
- first = b, last = c, depth += 1, limit = ss_ilg(c - b);
- }
- } else {
- if((a - first) <= (c - b)) {
- STACK_PUSH(b, c, depth + 1, ss_ilg(c - b));
- STACK_PUSH(first, a, depth, limit);
- first = c;
- } else if((last - c) <= (c - b)) {
- STACK_PUSH(first, a, depth, limit);
- STACK_PUSH(b, c, depth + 1, ss_ilg(c - b));
- first = c;
- } else {
- STACK_PUSH(first, a, depth, limit);
- STACK_PUSH(c, last, depth, limit);
- first = b, last = c, depth += 1, limit = ss_ilg(c - b);
- }
- }
- } else {
- limit += 1;
- if(Td[PA[*first] - 1] < v) {
- first = ss_partition(PA, first, last, depth);
- limit = ss_ilg(last - first);
- }
- depth += 1;
- }
- }
-#undef STACK_SIZE
-}
-
-#endif /* (SS_BLOCKSIZE == 0) || (SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE) */
-
-
-/*---------------------------------------------------------------------------*/
-
-#if SS_BLOCKSIZE != 0
-
-static INLINE
-void
-ss_blockswap(int *a, int *b, int n) {
- int t;
- for(; 0 < n; --n, ++a, ++b) {
- t = *a, *a = *b, *b = t;
- }
-}
-
-static INLINE
-void
-ss_rotate(int *first, int *middle, int *last) {
- int *a, *b, t;
- int l, r;
- l = middle - first, r = last - middle;
- for(; (0 < l) && (0 < r);) {
- if(l == r) { ss_blockswap(first, middle, l); break; }
- if(l < r) {
- a = last - 1, b = middle - 1;
- t = *a;
- do {
- *a-- = *b, *b-- = *a;
- if(b < first) {
- *a = t;
- last = a;
- if((r -= l + 1) <= l) { break; }
- a -= 1, b = middle - 1;
- t = *a;
- }
- } while(1);
- } else {
- a = first, b = middle;
- t = *a;
- do {
- *a++ = *b, *b++ = *a;
- if(last <= b) {
- *a = t;
- first = a + 1;
- if((l -= r + 1) <= r) { break; }
- a += 1, b = middle;
- t = *a;
- }
- } while(1);
- }
- }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-static
-void
-ss_inplacemerge(const unsigned char *T, const int *PA,
- int *first, int *middle, int *last,
- int depth) {
- const int *p;
- int *a, *b;
- int len, half;
- int q, r;
- int x;
-
- for(;;) {
- if(*(last - 1) < 0) { x = 1; p = PA + ~*(last - 1); }
- else { x = 0; p = PA + *(last - 1); }
- for(a = first, len = middle - first, half = len >> 1, r = -1;
- 0 < len;
- len = half, half >>= 1) {
- b = a + half;
- q = ss_compare(T, PA + ((0 <= *b) ? *b : ~*b), p, depth);
- if(q < 0) {
- a = b + 1;
- half -= (len & 1) ^ 1;
- } else {
- r = q;
- }
- }
- if(a < middle) {
- if(r == 0) { *a = ~*a; }
- ss_rotate(a, middle, last);
- last -= middle - a;
- middle = a;
- if(first == middle) { break; }
- }
- --last;
- if(x != 0) { while(*--last < 0) { } }
- if(middle == last) { break; }
- }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Merge-forward with internal buffer. */
-static
-void
-ss_mergeforward(const unsigned char *T, const int *PA,
- int *first, int *middle, int *last,
- int *buf, int depth) {
- int *a, *b, *c, *bufend;
- int t;
- int r;
-
- bufend = buf + (middle - first) - 1;
- ss_blockswap(buf, first, middle - first);
-
- for(t = *(a = first), b = buf, c = middle;;) {
- r = ss_compare(T, PA + *b, PA + *c, depth);
- if(r < 0) {
- do {
- *a++ = *b;
- if(bufend <= b) { *bufend = t; return; }
- *b++ = *a;
- } while(*b < 0);
- } else if(r > 0) {
- do {
- *a++ = *c, *c++ = *a;
- if(last <= c) {
- while(b < bufend) { *a++ = *b, *b++ = *a; }
- *a = *b, *b = t;
- return;
- }
- } while(*c < 0);
- } else {
- *c = ~*c;
- do {
- *a++ = *b;
- if(bufend <= b) { *bufend = t; return; }
- *b++ = *a;
- } while(*b < 0);
-
- do {
- *a++ = *c, *c++ = *a;
- if(last <= c) {
- while(b < bufend) { *a++ = *b, *b++ = *a; }
- *a = *b, *b = t;
- return;
- }
- } while(*c < 0);
- }
- }
-}
-
-/* Merge-backward with internal buffer. */
-static
-void
-ss_mergebackward(const unsigned char *T, const int *PA,
- int *first, int *middle, int *last,
- int *buf, int depth) {
- const int *p1, *p2;
- int *a, *b, *c, *bufend;
- int t;
- int r;
- int x;
-
- bufend = buf + (last - middle) - 1;
- ss_blockswap(buf, middle, last - middle);
-
- x = 0;
- if(*bufend < 0) { p1 = PA + ~*bufend; x |= 1; }
- else { p1 = PA + *bufend; }
- if(*(middle - 1) < 0) { p2 = PA + ~*(middle - 1); x |= 2; }
- else { p2 = PA + *(middle - 1); }
- for(t = *(a = last - 1), b = bufend, c = middle - 1;;) {
- r = ss_compare(T, p1, p2, depth);
- if(0 < r) {
- if(x & 1) { do { *a-- = *b, *b-- = *a; } while(*b < 0); x ^= 1; }
- *a-- = *b;
- if(b <= buf) { *buf = t; break; }
- *b-- = *a;
- if(*b < 0) { p1 = PA + ~*b; x |= 1; }
- else { p1 = PA + *b; }
- } else if(r < 0) {
- if(x & 2) { do { *a-- = *c, *c-- = *a; } while(*c < 0); x ^= 2; }
- *a-- = *c, *c-- = *a;
- if(c < first) {
- while(buf < b) { *a-- = *b, *b-- = *a; }
- *a = *b, *b = t;
- break;
- }
- if(*c < 0) { p2 = PA + ~*c; x |= 2; }
- else { p2 = PA + *c; }
- } else {
- if(x & 1) { do { *a-- = *b, *b-- = *a; } while(*b < 0); x ^= 1; }
- *a-- = ~*b;
- if(b <= buf) { *buf = t; break; }
- *b-- = *a;
- if(x & 2) { do { *a-- = *c, *c-- = *a; } while(*c < 0); x ^= 2; }
- *a-- = *c, *c-- = *a;
- if(c < first) {
- while(buf < b) { *a-- = *b, *b-- = *a; }
- *a = *b, *b = t;
- break;
- }
- if(*b < 0) { p1 = PA + ~*b; x |= 1; }
- else { p1 = PA + *b; }
- if(*c < 0) { p2 = PA + ~*c; x |= 2; }
- else { p2 = PA + *c; }
- }
- }
-}
-
-/* D&C based merge. */
-static
-void
-ss_swapmerge(const unsigned char *T, const int *PA,
- int *first, int *middle, int *last,
- int *buf, int bufsize, int depth) {
-#define STACK_SIZE SS_SMERGE_STACKSIZE
-#define GETIDX(a) ((0 <= (a)) ? (a) : (~(a)))
-#define MERGE_CHECK(a, b, c)\
- do {\
- if(((c) & 1) ||\
- (((c) & 2) && (ss_compare(T, PA + GETIDX(*((a) - 1)), PA + *(a), depth) == 0))) {\
- *(a) = ~*(a);\
- }\
- if(((c) & 4) && ((ss_compare(T, PA + GETIDX(*((b) - 1)), PA + *(b), depth) == 0))) {\
- *(b) = ~*(b);\
- }\
- } while(0)
- struct { int *a, *b, *c; int d; } stack[STACK_SIZE];
- int *l, *r, *lm, *rm;
- int m, len, half;
- int ssize;
- int check, next;
-
- for(check = 0, ssize = 0;;) {
- if((last - middle) <= bufsize) {
- if((first < middle) && (middle < last)) {
- ss_mergebackward(T, PA, first, middle, last, buf, depth);
- }
- MERGE_CHECK(first, last, check);
- STACK_POP(first, middle, last, check);
- continue;
- }
-
- if((middle - first) <= bufsize) {
- if(first < middle) {
- ss_mergeforward(T, PA, first, middle, last, buf, depth);
- }
- MERGE_CHECK(first, last, check);
- STACK_POP(first, middle, last, check);
- continue;
- }
-
- for(m = 0, len = MIN(middle - first, last - middle), half = len >> 1;
- 0 < len;
- len = half, half >>= 1) {
- if(ss_compare(T, PA + GETIDX(*(middle + m + half)),
- PA + GETIDX(*(middle - m - half - 1)), depth) < 0) {
- m += half + 1;
- half -= (len & 1) ^ 1;
- }
- }
-
- if(0 < m) {
- lm = middle - m, rm = middle + m;
- ss_blockswap(lm, middle, m);
- l = r = middle, next = 0;
- if(rm < last) {
- if(*rm < 0) {
- *rm = ~*rm;
- if(first < lm) { for(; *--l < 0;) { } next |= 4; }
- next |= 1;
- } else if(first < lm) {
- for(; *r < 0; ++r) { }
- next |= 2;
- }
- }
-
- if((l - first) <= (last - r)) {
- STACK_PUSH(r, rm, last, (next & 3) | (check & 4));
- middle = lm, last = l, check = (check & 3) | (next & 4);
- } else {
- if((next & 2) && (r == middle)) { next ^= 6; }
- STACK_PUSH(first, lm, l, (check & 3) | (next & 4));
- first = r, middle = rm, check = (next & 3) | (check & 4);
- }
- } else {
- if(ss_compare(T, PA + GETIDX(*(middle - 1)), PA + *middle, depth) == 0) {
- *middle = ~*middle;
- }
- MERGE_CHECK(first, last, check);
- STACK_POP(first, middle, last, check);
- }
- }
-#undef STACK_SIZE
-}
-
-#endif /* SS_BLOCKSIZE != 0 */
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Substring sort */
-static
-void
-sssort(const unsigned char *T, const int *PA,
- int *first, int *last,
- int *buf, int bufsize,
- int depth, int n, int lastsuffix) {
- int *a;
-#if SS_BLOCKSIZE != 0
- int *b, *middle, *curbuf;
- int j, k, curbufsize, limit;
-#endif
- int i;
-
- if(lastsuffix != 0) { ++first; }
-
-#if SS_BLOCKSIZE == 0
- ss_mintrosort(T, PA, first, last, depth);
-#else
- if((bufsize < SS_BLOCKSIZE) &&
- (bufsize < (last - first)) &&
- (bufsize < (limit = ss_isqrt(last - first)))) {
- if(SS_BLOCKSIZE < limit) { limit = SS_BLOCKSIZE; }
- buf = middle = last - limit, bufsize = limit;
- } else {
- middle = last, limit = 0;
- }
- for(a = first, i = 0; SS_BLOCKSIZE < (middle - a); a += SS_BLOCKSIZE, ++i) {
-#if SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE
- ss_mintrosort(T, PA, a, a + SS_BLOCKSIZE, depth);
-#elif 1 < SS_BLOCKSIZE
- ss_insertionsort(T, PA, a, a + SS_BLOCKSIZE, depth);
-#endif
- curbufsize = last - (a + SS_BLOCKSIZE);
- curbuf = a + SS_BLOCKSIZE;
- if(curbufsize <= bufsize) { curbufsize = bufsize, curbuf = buf; }
- for(b = a, k = SS_BLOCKSIZE, j = i; j & 1; b -= k, k <<= 1, j >>= 1) {
- ss_swapmerge(T, PA, b - k, b, b + k, curbuf, curbufsize, depth);
- }
- }
-#if SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE
- ss_mintrosort(T, PA, a, middle, depth);
-#elif 1 < SS_BLOCKSIZE
- ss_insertionsort(T, PA, a, middle, depth);
-#endif
- for(k = SS_BLOCKSIZE; i != 0; k <<= 1, i >>= 1) {
- if(i & 1) {
- ss_swapmerge(T, PA, a - k, a, middle, buf, bufsize, depth);
- a -= k;
- }
- }
- if(limit != 0) {
-#if SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE
- ss_mintrosort(T, PA, middle, last, depth);
-#elif 1 < SS_BLOCKSIZE
- ss_insertionsort(T, PA, middle, last, depth);
-#endif
- ss_inplacemerge(T, PA, first, middle, last, depth);
- }
-#endif
-
- if(lastsuffix != 0) {
- /* Insert last type B* suffix. */
- int PAi[2]; PAi[0] = PA[*(first - 1)], PAi[1] = n - 2;
- for(a = first, i = *(first - 1);
- (a < last) && ((*a < 0) || (0 < ss_compare(T, &(PAi[0]), PA + *a, depth)));
- ++a) {
- *(a - 1) = *a;
- }
- *(a - 1) = i;
- }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-static INLINE
-int
-tr_ilg(int n) {
- return (n & 0xffff0000) ?
- ((n & 0xff000000) ?
- 24 + lg_table[(n >> 24) & 0xff] :
- 16 + lg_table[(n >> 16) & 0xff]) :
- ((n & 0x0000ff00) ?
- 8 + lg_table[(n >> 8) & 0xff] :
- 0 + lg_table[(n >> 0) & 0xff]);
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Simple insertionsort for small size groups. */
-static
-void
-tr_insertionsort(const int *ISAd, int *first, int *last) {
- int *a, *b;
- int t, r;
-
- for(a = first + 1; a < last; ++a) {
- for(t = *a, b = a - 1; 0 > (r = ISAd[t] - ISAd[*b]);) {
- do { *(b + 1) = *b; } while((first <= --b) && (*b < 0));
- if(b < first) { break; }
- }
- if(r == 0) { *b = ~*b; }
- *(b + 1) = t;
- }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-static INLINE
-void
-tr_fixdown(const int *ISAd, int *SA, int i, int size) {
- int j, k;
- int v;
- int c, d, e;
-
- for(v = SA[i], c = ISAd[v]; (j = 2 * i + 1) < size; SA[i] = SA[k], i = k) {
- d = ISAd[SA[k = j++]];
- if(d < (e = ISAd[SA[j]])) { k = j; d = e; }
- if(d <= c) { break; }
- }
- SA[i] = v;
-}
-
-/* Simple top-down heapsort. */
-static
-void
-tr_heapsort(const int *ISAd, int *SA, int size) {
- int i, m;
- int t;
-
- m = size;
- if((size % 2) == 0) {
- m--;
- if(ISAd[SA[m / 2]] < ISAd[SA[m]]) { SWAP(SA[m], SA[m / 2]); }
- }
-
- for(i = m / 2 - 1; 0 <= i; --i) { tr_fixdown(ISAd, SA, i, m); }
- if((size % 2) == 0) { SWAP(SA[0], SA[m]); tr_fixdown(ISAd, SA, 0, m); }
- for(i = m - 1; 0 < i; --i) {
- t = SA[0], SA[0] = SA[i];
- tr_fixdown(ISAd, SA, 0, i);
- SA[i] = t;
- }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Returns the median of three elements. */
-static INLINE
-int *
-tr_median3(const int *ISAd, int *v1, int *v2, int *v3) {
- int *t;
- if(ISAd[*v1] > ISAd[*v2]) { SWAP(v1, v2); }
- if(ISAd[*v2] > ISAd[*v3]) {
- if(ISAd[*v1] > ISAd[*v3]) { return v1; }
- else { return v3; }
- }
- return v2;
-}
-
-/* Returns the median of five elements. */
-static INLINE
-int *
-tr_median5(const int *ISAd,
- int *v1, int *v2, int *v3, int *v4, int *v5) {
- int *t;
- if(ISAd[*v2] > ISAd[*v3]) { SWAP(v2, v3); }
- if(ISAd[*v4] > ISAd[*v5]) { SWAP(v4, v5); }
- if(ISAd[*v2] > ISAd[*v4]) { SWAP(v2, v4); SWAP(v3, v5); }
- if(ISAd[*v1] > ISAd[*v3]) { SWAP(v1, v3); }
- if(ISAd[*v1] > ISAd[*v4]) { SWAP(v1, v4); SWAP(v3, v5); }
- if(ISAd[*v3] > ISAd[*v4]) { return v4; }
- return v3;
-}
-
-/* Returns the pivot element. */
-static INLINE
-int *
-tr_pivot(const int *ISAd, int *first, int *last) {
- int *middle;
- int t;
-
- t = last - first;
- middle = first + t / 2;
-
- if(t <= 512) {
- if(t <= 32) {
- return tr_median3(ISAd, first, middle, last - 1);
- } else {
- t >>= 2;
- return tr_median5(ISAd, first, first + t, middle, last - 1 - t, last - 1);
- }
- }
- t >>= 3;
- first = tr_median3(ISAd, first, first + t, first + (t << 1));
- middle = tr_median3(ISAd, middle - t, middle, middle + t);
- last = tr_median3(ISAd, last - 1 - (t << 1), last - 1 - t, last - 1);
- return tr_median3(ISAd, first, middle, last);
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-typedef struct _trbudget_t trbudget_t;
-struct _trbudget_t {
- int chance;
- int remain;
- int incval;
- int count;
-};
-
-static INLINE
-void
-trbudget_init(trbudget_t *budget, int chance, int incval) {
- budget->chance = chance;
- budget->remain = budget->incval = incval;
-}
-
-static INLINE
-int
-trbudget_check(trbudget_t *budget, int size) {
- if(size <= budget->remain) { budget->remain -= size; return 1; }
- if(budget->chance == 0) { budget->count += size; return 0; }
- budget->remain += budget->incval - size;
- budget->chance -= 1;
- return 1;
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-static INLINE
-void
-tr_partition(const int *ISAd,
- int *first, int *middle, int *last,
- int **pa, int **pb, int v) {
- int *a, *b, *c, *d, *e, *f;
- int t, s;
- int x = 0;
-
- for(b = middle - 1; (++b < last) && ((x = ISAd[*b]) == v);) { }
- if(((a = b) < last) && (x < v)) {
- for(; (++b < last) && ((x = ISAd[*b]) <= v);) {
- if(x == v) { SWAP(*b, *a); ++a; }
- }
- }
- for(c = last; (b < --c) && ((x = ISAd[*c]) == v);) { }
- if((b < (d = c)) && (x > v)) {
- for(; (b < --c) && ((x = ISAd[*c]) >= v);) {
- if(x == v) { SWAP(*c, *d); --d; }
- }
- }
- for(; b < c;) {
- SWAP(*b, *c);
- for(; (++b < c) && ((x = ISAd[*b]) <= v);) {
- if(x == v) { SWAP(*b, *a); ++a; }
- }
- for(; (b < --c) && ((x = ISAd[*c]) >= v);) {
- if(x == v) { SWAP(*c, *d); --d; }
- }
- }
-
- if(a <= d) {
- c = b - 1;
- if((s = a - first) > (t = b - a)) { s = t; }
- for(e = first, f = b - s; 0 < s; --s, ++e, ++f) { SWAP(*e, *f); }
- if((s = d - c) > (t = last - d - 1)) { s = t; }
- for(e = b, f = last - s; 0 < s; --s, ++e, ++f) { SWAP(*e, *f); }
- first += (b - a), last -= (d - c);
- }
- *pa = first, *pb = last;
-}
-
-static
-void
-tr_copy(int *ISA, const int *SA,
- int *first, int *a, int *b, int *last,
- int depth) {
- /* sort suffixes of middle partition
- by using sorted order of suffixes of left and right partition. */
- int *c, *d, *e;
- int s, v;
-
- v = b - SA - 1;
- for(c = first, d = a - 1; c <= d; ++c) {
- if((0 <= (s = *c - depth)) && (ISA[s] == v)) {
- *++d = s;
- ISA[s] = d - SA;
- }
- }
- for(c = last - 1, e = d + 1, d = b; e < d; --c) {
- if((0 <= (s = *c - depth)) && (ISA[s] == v)) {
- *--d = s;
- ISA[s] = d - SA;
- }
- }
-}
-
-static
-void
-tr_partialcopy(int *ISA, const int *SA,
- int *first, int *a, int *b, int *last,
- int depth) {
- int *c, *d, *e;
- int s, v;
- int rank, lastrank, newrank = -1;
-
- v = b - SA - 1;
- lastrank = -1;
- for(c = first, d = a - 1; c <= d; ++c) {
- if((0 <= (s = *c - depth)) && (ISA[s] == v)) {
- *++d = s;
- rank = ISA[s + depth];
- if(lastrank != rank) { lastrank = rank; newrank = d - SA; }
- ISA[s] = newrank;
- }
- }
-
- lastrank = -1;
- for(e = d; first <= e; --e) {
- rank = ISA[*e];
- if(lastrank != rank) { lastrank = rank; newrank = e - SA; }
- if(newrank != rank) { ISA[*e] = newrank; }
- }
-
- lastrank = -1;
- for(c = last - 1, e = d + 1, d = b; e < d; --c) {
- if((0 <= (s = *c - depth)) && (ISA[s] == v)) {
- *--d = s;
- rank = ISA[s + depth];
- if(lastrank != rank) { lastrank = rank; newrank = d - SA; }
- ISA[s] = newrank;
- }
- }
-}
-
-static
-void
-tr_introsort(int *ISA, const int *ISAd,
- int *SA, int *first, int *last,
- trbudget_t *budget) {
-#define STACK_SIZE TR_STACKSIZE
- struct { const int *a; int *b, *c; int d, e; }stack[STACK_SIZE];
- int *a, *b, *c;
- int t;
- int v, x = 0;
- int incr = ISAd - ISA;
- int limit, next;
- int ssize, trlink = -1;
-
- for(ssize = 0, limit = tr_ilg(last - first);;) {
-
- if(limit < 0) {
- if(limit == -1) {
- /* tandem repeat partition */
- tr_partition(ISAd - incr, first, first, last, &a, &b, last - SA - 1);
-
- /* update ranks */
- if(a < last) {
- for(c = first, v = a - SA - 1; c < a; ++c) { ISA[*c] = v; }
- }
- if(b < last) {
- for(c = a, v = b - SA - 1; c < b; ++c) { ISA[*c] = v; }
- }
-
- /* push */
- if(1 < (b - a)) {
- STACK_PUSH5(NULL, a, b, 0, 0);
- STACK_PUSH5(ISAd - incr, first, last, -2, trlink);
- trlink = ssize - 2;
- }
- if((a - first) <= (last - b)) {
- if(1 < (a - first)) {
- STACK_PUSH5(ISAd, b, last, tr_ilg(last - b), trlink);
- last = a, limit = tr_ilg(a - first);
- } else if(1 < (last - b)) {
- first = b, limit = tr_ilg(last - b);
- } else {
- STACK_POP5(ISAd, first, last, limit, trlink);
- }
- } else {
- if(1 < (last - b)) {
- STACK_PUSH5(ISAd, first, a, tr_ilg(a - first), trlink);
- first = b, limit = tr_ilg(last - b);
- } else if(1 < (a - first)) {
- last = a, limit = tr_ilg(a - first);
- } else {
- STACK_POP5(ISAd, first, last, limit, trlink);
- }
- }
- } else if(limit == -2) {
- /* tandem repeat copy */
- a = stack[--ssize].b, b = stack[ssize].c;
- if(stack[ssize].d == 0) {
- tr_copy(ISA, SA, first, a, b, last, ISAd - ISA);
- } else {
- if(0 <= trlink) { stack[trlink].d = -1; }
- tr_partialcopy(ISA, SA, first, a, b, last, ISAd - ISA);
- }
- STACK_POP5(ISAd, first, last, limit, trlink);
- } else {
- /* sorted partition */
- if(0 <= *first) {
- a = first;
- do { ISA[*a] = a - SA; } while((++a < last) && (0 <= *a));
- first = a;
- }
- if(first < last) {
- a = first; do { *a = ~*a; } while(*++a < 0);
- next = (ISA[*a] != ISAd[*a]) ? tr_ilg(a - first + 1) : -1;
- if(++a < last) { for(b = first, v = a - SA - 1; b < a; ++b) { ISA[*b] = v; } }
-
- /* push */
- if(trbudget_check(budget, a - first)) {
- if((a - first) <= (last - a)) {
- STACK_PUSH5(ISAd, a, last, -3, trlink);
- ISAd += incr, last = a, limit = next;
- } else {
- if(1 < (last - a)) {
- STACK_PUSH5(ISAd + incr, first, a, next, trlink);
- first = a, limit = -3;
- } else {
- ISAd += incr, last = a, limit = next;
- }
- }
- } else {
- if(0 <= trlink) { stack[trlink].d = -1; }
- if(1 < (last - a)) {
- first = a, limit = -3;
- } else {
- STACK_POP5(ISAd, first, last, limit, trlink);
- }
- }
- } else {
- STACK_POP5(ISAd, first, last, limit, trlink);
- }
- }
- continue;
- }
-
- if((last - first) <= TR_INSERTIONSORT_THRESHOLD) {
- tr_insertionsort(ISAd, first, last);
- limit = -3;
- continue;
- }
-
- if(limit-- == 0) {
- tr_heapsort(ISAd, first, last - first);
- for(a = last - 1; first < a; a = b) {
- for(x = ISAd[*a], b = a - 1; (first <= b) && (ISAd[*b] == x); --b) { *b = ~*b; }
- }
- limit = -3;
- continue;
- }
-
- /* choose pivot */
- a = tr_pivot(ISAd, first, last);
- SWAP(*first, *a);
- v = ISAd[*first];
-
- /* partition */
- tr_partition(ISAd, first, first + 1, last, &a, &b, v);
- if((last - first) != (b - a)) {
- next = (ISA[*a] != v) ? tr_ilg(b - a) : -1;
-
- /* update ranks */
- for(c = first, v = a - SA - 1; c < a; ++c) { ISA[*c] = v; }
- if(b < last) { for(c = a, v = b - SA - 1; c < b; ++c) { ISA[*c] = v; } }
-
- /* push */
- if((1 < (b - a)) && (trbudget_check(budget, b - a))) {
- if((a - first) <= (last - b)) {
- if((last - b) <= (b - a)) {
- if(1 < (a - first)) {
- STACK_PUSH5(ISAd + incr, a, b, next, trlink);
- STACK_PUSH5(ISAd, b, last, limit, trlink);
- last = a;
- } else if(1 < (last - b)) {
- STACK_PUSH5(ISAd + incr, a, b, next, trlink);
- first = b;
- } else {
- ISAd += incr, first = a, last = b, limit = next;
- }
- } else if((a - first) <= (b - a)) {
- if(1 < (a - first)) {
- STACK_PUSH5(ISAd, b, last, limit, trlink);
- STACK_PUSH5(ISAd + incr, a, b, next, trlink);
- last = a;
- } else {
- STACK_PUSH5(ISAd, b, last, limit, trlink);
- ISAd += incr, first = a, last = b, limit = next;
- }
- } else {
- STACK_PUSH5(ISAd, b, last, limit, trlink);
- STACK_PUSH5(ISAd, first, a, limit, trlink);
- ISAd += incr, first = a, last = b, limit = next;
- }
- } else {
- if((a - first) <= (b - a)) {
- if(1 < (last - b)) {
- STACK_PUSH5(ISAd + incr, a, b, next, trlink);
- STACK_PUSH5(ISAd, first, a, limit, trlink);
- first = b;
- } else if(1 < (a - first)) {
- STACK_PUSH5(ISAd + incr, a, b, next, trlink);
- last = a;
- } else {
- ISAd += incr, first = a, last = b, limit = next;
- }
- } else if((last - b) <= (b - a)) {
- if(1 < (last - b)) {
- STACK_PUSH5(ISAd, first, a, limit, trlink);
- STACK_PUSH5(ISAd + incr, a, b, next, trlink);
- first = b;
- } else {
- STACK_PUSH5(ISAd, first, a, limit, trlink);
- ISAd += incr, first = a, last = b, limit = next;
- }
- } else {
- STACK_PUSH5(ISAd, first, a, limit, trlink);
- STACK_PUSH5(ISAd, b, last, limit, trlink);
- ISAd += incr, first = a, last = b, limit = next;
- }
- }
- } else {
- if((1 < (b - a)) && (0 <= trlink)) { stack[trlink].d = -1; }
- if((a - first) <= (last - b)) {
- if(1 < (a - first)) {
- STACK_PUSH5(ISAd, b, last, limit, trlink);
- last = a;
- } else if(1 < (last - b)) {
- first = b;
- } else {
- STACK_POP5(ISAd, first, last, limit, trlink);
- }
- } else {
- if(1 < (last - b)) {
- STACK_PUSH5(ISAd, first, a, limit, trlink);
- first = b;
- } else if(1 < (a - first)) {
- last = a;
- } else {
- STACK_POP5(ISAd, first, last, limit, trlink);
- }
- }
- }
- } else {
- if(trbudget_check(budget, last - first)) {
- limit = tr_ilg(last - first), ISAd += incr;
- } else {
- if(0 <= trlink) { stack[trlink].d = -1; }
- STACK_POP5(ISAd, first, last, limit, trlink);
- }
- }
- }
-#undef STACK_SIZE
-}
-
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Tandem repeat sort */
-static
-void
-trsort(int *ISA, int *SA, int n, int depth) {
- int *ISAd;
- int *first, *last;
- trbudget_t budget;
- int t, skip, unsorted;
-
- trbudget_init(&budget, tr_ilg(n) * 2 / 3, n);
-/* trbudget_init(&budget, tr_ilg(n) * 3 / 4, n); */
- for(ISAd = ISA + depth; -n < *SA; ISAd += ISAd - ISA) {
- first = SA;
- skip = 0;
- unsorted = 0;
- do {
- if((t = *first) < 0) { first -= t; skip += t; }
- else {
- if(skip != 0) { *(first + skip) = skip; skip = 0; }
- last = SA + ISA[t] + 1;
- if(1 < (last - first)) {
- budget.count = 0;
- tr_introsort(ISA, ISAd, SA, first, last, &budget);
- if(budget.count != 0) { unsorted += budget.count; }
- else { skip = first - last; }
- } else if((last - first) == 1) {
- skip = -1;
- }
- first = last;
- }
- } while(first < (SA + n));
- if(skip != 0) { *(first + skip) = skip; }
- if(unsorted == 0) { break; }
- }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Sorts suffixes of type B*. */
-static
-int
-sort_typeBstar(const unsigned char *T, int *SA,
- int *bucket_A, int *bucket_B,
- int n, int openMP) {
- int *PAb, *ISAb, *buf;
-#ifdef LIBBSC_OPENMP
- int *curbuf;
- int l;
-#endif
- int i, j, k, t, m, bufsize;
- int c0, c1;
-#ifdef LIBBSC_OPENMP
- int d0, d1;
-#endif
- (void)openMP;
-
- /* Initialize bucket arrays. */
- for(i = 0; i < BUCKET_A_SIZE; ++i) { bucket_A[i] = 0; }
- for(i = 0; i < BUCKET_B_SIZE; ++i) { bucket_B[i] = 0; }
-
- /* Count the number of occurrences of the first one or two characters of each
- type A, B and B* suffix. Moreover, store the beginning position of all
- type B* suffixes into the array SA. */
- for(i = n - 1, m = n, c0 = T[n - 1]; 0 <= i;) {
- /* type A suffix. */
- do { ++BUCKET_A(c1 = c0); } while((0 <= --i) && ((c0 = T[i]) >= c1));
- if(0 <= i) {
- /* type B* suffix. */
- ++BUCKET_BSTAR(c0, c1);
- SA[--m] = i;
- /* type B suffix. */
- for(--i, c1 = c0; (0 <= i) && ((c0 = T[i]) <= c1); --i, c1 = c0) {
- ++BUCKET_B(c0, c1);
- }
- }
- }
- m = n - m;
-/*
-note:
- A type B* suffix is lexicographically smaller than a type B suffix that
- begins with the same first two characters.
-*/
-
- /* Calculate the index of start/end point of each bucket. */
- for(c0 = 0, i = 0, j = 0; c0 < ALPHABET_SIZE; ++c0) {
- t = i + BUCKET_A(c0);
- BUCKET_A(c0) = i + j; /* start point */
- i = t + BUCKET_B(c0, c0);
- for(c1 = c0 + 1; c1 < ALPHABET_SIZE; ++c1) {
- j += BUCKET_BSTAR(c0, c1);
- BUCKET_BSTAR(c0, c1) = j; /* end point */
- i += BUCKET_B(c0, c1);
- }
- }
-
- if(0 < m) {
- /* Sort the type B* suffixes by their first two characters. */
- PAb = SA + n - m; ISAb = SA + m;
- for(i = m - 2; 0 <= i; --i) {
- t = PAb[i], c0 = T[t], c1 = T[t + 1];
- SA[--BUCKET_BSTAR(c0, c1)] = i;
- }
- t = PAb[m - 1], c0 = T[t], c1 = T[t + 1];
- SA[--BUCKET_BSTAR(c0, c1)] = m - 1;
-
- /* Sort the type B* substrings using sssort. */
-#ifdef LIBBSC_OPENMP
- if (openMP)
- {
- buf = SA + m;
- c0 = ALPHABET_SIZE - 2, c1 = ALPHABET_SIZE - 1, j = m;
-#pragma omp parallel default(shared) private(bufsize, curbuf, k, l, d0, d1)
- {
- bufsize = (n - (2 * m)) / omp_get_num_threads();
- curbuf = buf + omp_get_thread_num() * bufsize;
- k = 0;
- for(;;) {
- #pragma omp critical(sssort_lock)
- {
- if(0 < (l = j)) {
- d0 = c0, d1 = c1;
- do {
- k = BUCKET_BSTAR(d0, d1);
- if(--d1 <= d0) {
- d1 = ALPHABET_SIZE - 1;
- if(--d0 < 0) { break; }
- }
- } while(((l - k) <= 1) && (0 < (l = k)));
- c0 = d0, c1 = d1, j = k;
- }
- }
- if(l == 0) { break; }
- sssort(T, PAb, SA + k, SA + l,
- curbuf, bufsize, 2, n, *(SA + k) == (m - 1));
- }
- }
- }
- else
- {
- buf = SA + m, bufsize = n - (2 * m);
- for(c0 = ALPHABET_SIZE - 2, j = m; 0 < j; --c0) {
- for(c1 = ALPHABET_SIZE - 1; c0 < c1; j = i, --c1) {
- i = BUCKET_BSTAR(c0, c1);
- if(1 < (j - i)) {
- sssort(T, PAb, SA + i, SA + j,
- buf, bufsize, 2, n, *(SA + i) == (m - 1));
- }
- }
- }
- }
-#else
- buf = SA + m, bufsize = n - (2 * m);
- for(c0 = ALPHABET_SIZE - 2, j = m; 0 < j; --c0) {
- for(c1 = ALPHABET_SIZE - 1; c0 < c1; j = i, --c1) {
- i = BUCKET_BSTAR(c0, c1);
- if(1 < (j - i)) {
- sssort(T, PAb, SA + i, SA + j,
- buf, bufsize, 2, n, *(SA + i) == (m - 1));
- }
- }
- }
-#endif
-
- /* Compute ranks of type B* substrings. */
- for(i = m - 1; 0 <= i; --i) {
- if(0 <= SA[i]) {
- j = i;
- do { ISAb[SA[i]] = i; } while((0 <= --i) && (0 <= SA[i]));
- SA[i + 1] = i - j;
- if(i <= 0) { break; }
- }
- j = i;
- do { ISAb[SA[i] = ~SA[i]] = j; } while(SA[--i] < 0);
- ISAb[SA[i]] = j;
- }
-
- /* Construct the inverse suffix array of type B* suffixes using trsort. */
- trsort(ISAb, SA, m, 1);
-
- /* Set the sorted order of tyoe B* suffixes. */
- for(i = n - 1, j = m, c0 = T[n - 1]; 0 <= i;) {
- for(--i, c1 = c0; (0 <= i) && ((c0 = T[i]) >= c1); --i, c1 = c0) { }
- if(0 <= i) {
- t = i;
- for(--i, c1 = c0; (0 <= i) && ((c0 = T[i]) <= c1); --i, c1 = c0) { }
- SA[ISAb[--j]] = ((t == 0) || (1 < (t - i))) ? t : ~t;
- }
- }
-
- /* Calculate the index of start/end point of each bucket. */
- BUCKET_B(ALPHABET_SIZE - 1, ALPHABET_SIZE - 1) = n; /* end point */
- for(c0 = ALPHABET_SIZE - 2, k = m - 1; 0 <= c0; --c0) {
- i = BUCKET_A(c0 + 1) - 1;
- for(c1 = ALPHABET_SIZE - 1; c0 < c1; --c1) {
- t = i - BUCKET_B(c0, c1);
- BUCKET_B(c0, c1) = i; /* end point */
-
- /* Move all type B* suffixes to the correct position. */
- for(i = t, j = BUCKET_BSTAR(c0, c1);
- j <= k;
- --i, --k) { SA[i] = SA[k]; }
- }
- BUCKET_BSTAR(c0, c0 + 1) = i - BUCKET_B(c0, c0) + 1; /* start point */
- BUCKET_B(c0, c0) = i; /* end point */
- }
- }
-
- return m;
-}
-
-/* Constructs the suffix array by using the sorted order of type B* suffixes. */
-static
-void
-construct_SA(const unsigned char *T, int *SA,
- int *bucket_A, int *bucket_B,
- int n, int m) {
- int *i, *j, *k;
- int s;
- int c0, c1, c2;
-
- if(0 < m) {
- /* Construct the sorted order of type B suffixes by using
- the sorted order of type B* suffixes. */
- for(c1 = ALPHABET_SIZE - 2; 0 <= c1; --c1) {
- /* Scan the suffix array from right to left. */
- for(i = SA + BUCKET_BSTAR(c1, c1 + 1),
- j = SA + BUCKET_A(c1 + 1) - 1, k = NULL, c2 = -1;
- i <= j;
- --j) {
- if(0 < (s = *j)) {
- assert(T[s] == c1);
- assert(((s + 1) < n) && (T[s] <= T[s + 1]));
- assert(T[s - 1] <= T[s]);
- *j = ~s;
- c0 = T[--s];
- if((0 < s) && (T[s - 1] > c0)) { s = ~s; }
- if(c0 != c2) {
- if(0 <= c2) { BUCKET_B(c2, c1) = k - SA; }
- k = SA + BUCKET_B(c2 = c0, c1);
- }
- assert(k < j); assert(k != NULL);
- *k-- = s;
- } else {
- assert(((s == 0) && (T[s] == c1)) || (s < 0));
- *j = ~s;
- }
- }
- }
- }
-
- /* Construct the suffix array by using
- the sorted order of type B suffixes. */
- k = SA + BUCKET_A(c2 = T[n - 1]);
- *k++ = (T[n - 2] < c2) ? ~(n - 1) : (n - 1);
- /* Scan the suffix array from left to right. */
- for(i = SA, j = SA + n; i < j; ++i) {
- if(0 < (s = *i)) {
- assert(T[s - 1] >= T[s]);
- c0 = T[--s];
- if((s == 0) || (T[s - 1] < c0)) { s = ~s; }
- if(c0 != c2) {
- BUCKET_A(c2) = k - SA;
- k = SA + BUCKET_A(c2 = c0);
- }
- assert(i < k);
- *k++ = s;
- } else {
- assert(s < 0);
- *i = ~s;
- }
- }
-}
-
-/* Constructs the burrows-wheeler transformed string directly
- by using the sorted order of type B* suffixes. */
-static
-int
-construct_BWT(const unsigned char *T, int *SA,
- int *bucket_A, int *bucket_B,
- int n, int m) {
- int *i, *j, *k, *orig;
- int s;
- int c0, c1, c2;
-
- if(0 < m) {
- /* Construct the sorted order of type B suffixes by using
- the sorted order of type B* suffixes. */
- for(c1 = ALPHABET_SIZE - 2; 0 <= c1; --c1) {
- /* Scan the suffix array from right to left. */
- for(i = SA + BUCKET_BSTAR(c1, c1 + 1),
- j = SA + BUCKET_A(c1 + 1) - 1, k = NULL, c2 = -1;
- i <= j;
- --j) {
- if(0 < (s = *j)) {
- assert(T[s] == c1);
- assert(((s + 1) < n) && (T[s] <= T[s + 1]));
- assert(T[s - 1] <= T[s]);
- c0 = T[--s];
- *j = ~((int)c0);
- if((0 < s) && (T[s - 1] > c0)) { s = ~s; }
- if(c0 != c2) {
- if(0 <= c2) { BUCKET_B(c2, c1) = k - SA; }
- k = SA + BUCKET_B(c2 = c0, c1);
- }
- assert(k < j); assert(k != NULL);
- *k-- = s;
- } else if(s != 0) {
- *j = ~s;
-#ifndef NDEBUG
- } else {
- assert(T[s] == c1);
-#endif
- }
- }
- }
- }
-
- /* Construct the BWTed string by using
- the sorted order of type B suffixes. */
- k = SA + BUCKET_A(c2 = T[n - 1]);
- *k++ = (T[n - 2] < c2) ? ~((int)T[n - 2]) : (n - 1);
- /* Scan the suffix array from left to right. */
- for(i = SA, j = SA + n, orig = SA; i < j; ++i) {
- if(0 < (s = *i)) {
- assert(T[s - 1] >= T[s]);
- c0 = T[--s];
- *i = c0;
- if((0 < s) && (T[s - 1] < c0)) { s = ~((int)T[s - 1]); }
- if(c0 != c2) {
- BUCKET_A(c2) = k - SA;
- k = SA + BUCKET_A(c2 = c0);
- }
- assert(i < k);
- *k++ = s;
- } else if(s != 0) {
- *i = ~s;
- } else {
- orig = i;
- }
- }
-
- return orig - SA;
-}
-
-/* Constructs the burrows-wheeler transformed string directly
- by using the sorted order of type B* suffixes. */
-static
-int
-construct_BWT_indexes(const unsigned char *T, int *SA,
- int *bucket_A, int *bucket_B,
- int n, int m,
- unsigned char * num_indexes, int * indexes) {
- int *i, *j, *k, *orig;
- int s;
- int c0, c1, c2;
-
- int mod = n / 8;
- {
- mod |= mod >> 1; mod |= mod >> 2;
- mod |= mod >> 4; mod |= mod >> 8;
- mod |= mod >> 16; mod >>= 1;
-
- *num_indexes = (unsigned char)((n - 1) / (mod + 1));
- }
-
- if(0 < m) {
- /* Construct the sorted order of type B suffixes by using
- the sorted order of type B* suffixes. */
- for(c1 = ALPHABET_SIZE - 2; 0 <= c1; --c1) {
- /* Scan the suffix array from right to left. */
- for(i = SA + BUCKET_BSTAR(c1, c1 + 1),
- j = SA + BUCKET_A(c1 + 1) - 1, k = NULL, c2 = -1;
- i <= j;
- --j) {
- if(0 < (s = *j)) {
- assert(T[s] == c1);
- assert(((s + 1) < n) && (T[s] <= T[s + 1]));
- assert(T[s - 1] <= T[s]);
-
- if ((s & mod) == 0) indexes[s / (mod + 1) - 1] = j - SA;
-
- c0 = T[--s];
- *j = ~((int)c0);
- if((0 < s) && (T[s - 1] > c0)) { s = ~s; }
- if(c0 != c2) {
- if(0 <= c2) { BUCKET_B(c2, c1) = k - SA; }
- k = SA + BUCKET_B(c2 = c0, c1);
- }
- assert(k < j); assert(k != NULL);
- *k-- = s;
- } else if(s != 0) {
- *j = ~s;
-#ifndef NDEBUG
- } else {
- assert(T[s] == c1);
-#endif
- }
- }
- }
- }
-
- /* Construct the BWTed string by using
- the sorted order of type B suffixes. */
- k = SA + BUCKET_A(c2 = T[n - 1]);
- if (T[n - 2] < c2) {
- if (((n - 1) & mod) == 0) indexes[(n - 1) / (mod + 1) - 1] = k - SA;
- *k++ = ~((int)T[n - 2]);
- }
- else {
- *k++ = n - 1;
- }
-
- /* Scan the suffix array from left to right. */
- for(i = SA, j = SA + n, orig = SA; i < j; ++i) {
- if(0 < (s = *i)) {
- assert(T[s - 1] >= T[s]);
-
- if ((s & mod) == 0) indexes[s / (mod + 1) - 1] = i - SA;
-
- c0 = T[--s];
- *i = c0;
- if(c0 != c2) {
- BUCKET_A(c2) = k - SA;
- k = SA + BUCKET_A(c2 = c0);
- }
- assert(i < k);
- if((0 < s) && (T[s - 1] < c0)) {
- if ((s & mod) == 0) indexes[s / (mod + 1) - 1] = k - SA;
- *k++ = ~((int)T[s - 1]);
- } else
- *k++ = s;
- } else if(s != 0) {
- *i = ~s;
- } else {
- orig = i;
- }
- }
-
- return orig - SA;
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/*- Function -*/
-
-int
-divsufsort(const unsigned char *T, int *SA, int n, int openMP) {
- int *bucket_A, *bucket_B;
- int m;
- int err = 0;
-
- /* Check arguments. */
- if((T == NULL) || (SA == NULL) || (n < 0)) { return -1; }
- else if(n == 0) { return 0; }
- else if(n == 1) { SA[0] = 0; return 0; }
- else if(n == 2) { m = (T[0] < T[1]); SA[m ^ 1] = 0, SA[m] = 1; return 0; }
-
- bucket_A = (int *)malloc(BUCKET_A_SIZE * sizeof(int));
- bucket_B = (int *)malloc(BUCKET_B_SIZE * sizeof(int));
-
- /* Suffixsort. */
- if((bucket_A != NULL) && (bucket_B != NULL)) {
- m = sort_typeBstar(T, SA, bucket_A, bucket_B, n, openMP);
- construct_SA(T, SA, bucket_A, bucket_B, n, m);
- } else {
- err = -2;
- }
-
- free(bucket_B);
- free(bucket_A);
-
- return err;
-}
-
-int
-divbwt(const unsigned char *T, unsigned char *U, int *A, int n, unsigned char * num_indexes, int * indexes, int openMP) {
- int *B;
- int *bucket_A, *bucket_B;
- int m, pidx, i;
-
- /* Check arguments. */
- if((T == NULL) || (U == NULL) || (n < 0)) { return -1; }
- else if(n <= 1) { if(n == 1) { U[0] = T[0]; } return n; }
-
- if((B = A) == NULL) { B = (int *)malloc((size_t)(n + 1) * sizeof(int)); }
- bucket_A = (int *)malloc(BUCKET_A_SIZE * sizeof(int));
- bucket_B = (int *)malloc(BUCKET_B_SIZE * sizeof(int));
-
- /* Burrows-Wheeler Transform. */
- if((B != NULL) && (bucket_A != NULL) && (bucket_B != NULL)) {
- m = sort_typeBstar(T, B, bucket_A, bucket_B, n, openMP);
-
- if (num_indexes == NULL || indexes == NULL) {
- pidx = construct_BWT(T, B, bucket_A, bucket_B, n, m);
- } else {
- pidx = construct_BWT_indexes(T, B, bucket_A, bucket_B, n, m, num_indexes, indexes);
- }
-
- /* Copy to output string. */
- U[0] = T[n - 1];
- for(i = 0; i < pidx; ++i) { U[i + 1] = (unsigned char)B[i]; }
- for(i += 1; i < n; ++i) { U[i] = (unsigned char)B[i]; }
- pidx += 1;
- } else {
- pidx = -2;
- }
-
- free(bucket_B);
- free(bucket_A);
- if(A == NULL) { free(B); }
-
- return pidx;
-}
diff --git a/vendor/github.com/DataDog/zstd/divsufsort.h b/vendor/github.com/DataDog/zstd/divsufsort.h
deleted file mode 100644
index 5440994..0000000
--- a/vendor/github.com/DataDog/zstd/divsufsort.h
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * divsufsort.h for libdivsufsort-lite
- * Copyright (c) 2003-2008 Yuta Mori All Rights Reserved.
- *
- * Permission is hereby granted, free of charge, to any person
- * obtaining a copy of this software and associated documentation
- * files (the "Software"), to deal in the Software without
- * restriction, including without limitation the rights to use,
- * copy, modify, merge, publish, distribute, sublicense, and/or sell
- * copies of the Software, and to permit persons to whom the
- * Software is furnished to do so, subject to the following
- * conditions:
- *
- * The above copyright notice and this permission notice shall be
- * included in all copies or substantial portions of the Software.
- *
- * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
- * EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
- * OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
- * NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
- * HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
- * WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
- * FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
- * OTHER DEALINGS IN THE SOFTWARE.
- */
-
-#ifndef _DIVSUFSORT_H
-#define _DIVSUFSORT_H 1
-
-#ifdef __cplusplus
-extern "C" {
-#endif /* __cplusplus */
-
-
-/*- Prototypes -*/
-
-/**
- * Constructs the suffix array of a given string.
- * @param T [0..n-1] The input string.
- * @param SA [0..n-1] The output array of suffixes.
- * @param n The length of the given string.
- * @param openMP enables OpenMP optimization.
- * @return 0 if no error occurred, -1 or -2 otherwise.
- */
-int
-divsufsort(const unsigned char *T, int *SA, int n, int openMP);
-
-/**
- * Constructs the burrows-wheeler transformed string of a given string.
- * @param T [0..n-1] The input string.
- * @param U [0..n-1] The output string. (can be T)
- * @param A [0..n-1] The temporary array. (can be NULL)
- * @param n The length of the given string.
- * @param num_indexes The length of secondary indexes array. (can be NULL)
- * @param indexes The secondary indexes array. (can be NULL)
- * @param openMP enables OpenMP optimization.
- * @return The primary index if no error occurred, -1 or -2 otherwise.
- */
-int
-divbwt(const unsigned char *T, unsigned char *U, int *A, int n, unsigned char * num_indexes, int * indexes, int openMP);
-
-
-#ifdef __cplusplus
-} /* extern "C" */
-#endif /* __cplusplus */
-
-#endif /* _DIVSUFSORT_H */
diff --git a/vendor/github.com/DataDog/zstd/entropy_common.c b/vendor/github.com/DataDog/zstd/entropy_common.c
deleted file mode 100644
index b12944e..0000000
--- a/vendor/github.com/DataDog/zstd/entropy_common.c
+++ /dev/null
@@ -1,236 +0,0 @@
-/*
- Common functions of New Generation Entropy library
- Copyright (C) 2016, Yann Collet.
-
- BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
- * Redistributions of source code must retain the above copyright
- notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
- copyright notice, this list of conditions and the following disclaimer
- in the documentation and/or other materials provided with the
- distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
- "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
- LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
- A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
- OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
- OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
- You can contact the author at :
- - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy
- - Public forum : https://groups.google.com/forum/#!forum/lz4c
-*************************************************************************** */
-
-/* *************************************
-* Dependencies
-***************************************/
-#include "mem.h"
-#include "error_private.h" /* ERR_*, ERROR */
-#define FSE_STATIC_LINKING_ONLY /* FSE_MIN_TABLELOG */
-#include "fse.h"
-#define HUF_STATIC_LINKING_ONLY /* HUF_TABLELOG_ABSOLUTEMAX */
-#include "huf.h"
-
-
-/*=== Version ===*/
-unsigned FSE_versionNumber(void) { return FSE_VERSION_NUMBER; }
-
-
-/*=== Error Management ===*/
-unsigned FSE_isError(size_t code) { return ERR_isError(code); }
-const char* FSE_getErrorName(size_t code) { return ERR_getErrorName(code); }
-
-unsigned HUF_isError(size_t code) { return ERR_isError(code); }
-const char* HUF_getErrorName(size_t code) { return ERR_getErrorName(code); }
-
-
-/*-**************************************************************
-* FSE NCount encoding-decoding
-****************************************************************/
-size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr,
- const void* headerBuffer, size_t hbSize)
-{
- const BYTE* const istart = (const BYTE*) headerBuffer;
- const BYTE* const iend = istart + hbSize;
- const BYTE* ip = istart;
- int nbBits;
- int remaining;
- int threshold;
- U32 bitStream;
- int bitCount;
- unsigned charnum = 0;
- int previous0 = 0;
-
- if (hbSize < 4) {
- /* This function only works when hbSize >= 4 */
- char buffer[4];
- memset(buffer, 0, sizeof(buffer));
- memcpy(buffer, headerBuffer, hbSize);
- { size_t const countSize = FSE_readNCount(normalizedCounter, maxSVPtr, tableLogPtr,
- buffer, sizeof(buffer));
- if (FSE_isError(countSize)) return countSize;
- if (countSize > hbSize) return ERROR(corruption_detected);
- return countSize;
- } }
- assert(hbSize >= 4);
-
- /* init */
- memset(normalizedCounter, 0, (*maxSVPtr+1) * sizeof(normalizedCounter[0])); /* all symbols not present in NCount have a frequency of 0 */
- bitStream = MEM_readLE32(ip);
- nbBits = (bitStream & 0xF) + FSE_MIN_TABLELOG; /* extract tableLog */
- if (nbBits > FSE_TABLELOG_ABSOLUTE_MAX) return ERROR(tableLog_tooLarge);
- bitStream >>= 4;
- bitCount = 4;
- *tableLogPtr = nbBits;
- remaining = (1<<nbBits)+1;
- threshold = 1<<nbBits;
- nbBits++;
-
- while ((remaining>1) & (charnum<=*maxSVPtr)) {
- if (previous0) {
- unsigned n0 = charnum;
- while ((bitStream & 0xFFFF) == 0xFFFF) {
- n0 += 24;
- if (ip < iend-5) {
- ip += 2;
- bitStream = MEM_readLE32(ip) >> bitCount;
- } else {
- bitStream >>= 16;
- bitCount += 16;
- } }
- while ((bitStream & 3) == 3) {
- n0 += 3;
- bitStream >>= 2;
- bitCount += 2;
- }
- n0 += bitStream & 3;
- bitCount += 2;
- if (n0 > *maxSVPtr) return ERROR(maxSymbolValue_tooSmall);
- while (charnum < n0) normalizedCounter[charnum++] = 0;
- if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) {
- assert((bitCount >> 3) <= 3); /* For first condition to work */
- ip += bitCount>>3;
- bitCount &= 7;
- bitStream = MEM_readLE32(ip) >> bitCount;
- } else {
- bitStream >>= 2;
- } }
- { int const max = (2*threshold-1) - remaining;
- int count;
-
- if ((bitStream & (threshold-1)) < (U32)max) {
- count = bitStream & (threshold-1);
- bitCount += nbBits-1;
- } else {
- count = bitStream & (2*threshold-1);
- if (count >= threshold) count -= max;
- bitCount += nbBits;
- }
-
- count--; /* extra accuracy */
- remaining -= count < 0 ? -count : count; /* -1 means +1 */
- normalizedCounter[charnum++] = (short)count;
- previous0 = !count;
- while (remaining < threshold) {
- nbBits--;
- threshold >>= 1;
- }
-
- if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) {
- ip += bitCount>>3;
- bitCount &= 7;
- } else {
- bitCount -= (int)(8 * (iend - 4 - ip));
- ip = iend - 4;
- }
- bitStream = MEM_readLE32(ip) >> (bitCount & 31);
- } } /* while ((remaining>1) & (charnum<=*maxSVPtr)) */
- if (remaining != 1) return ERROR(corruption_detected);
- if (bitCount > 32) return ERROR(corruption_detected);
- *maxSVPtr = charnum-1;
-
- ip += (bitCount+7)>>3;
- return ip-istart;
-}
-
-
-/*! HUF_readStats() :
- Read compact Huffman tree, saved by HUF_writeCTable().
- `huffWeight` is destination buffer.
- `rankStats` is assumed to be a table of at least HUF_TABLELOG_MAX U32.
- @return : size read from `src` , or an error Code .
- Note : Needed by HUF_readCTable() and HUF_readDTableX?() .
-*/
-size_t HUF_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats,
- U32* nbSymbolsPtr, U32* tableLogPtr,
- const void* src, size_t srcSize)
-{
- U32 weightTotal;
- const BYTE* ip = (const BYTE*) src;
- size_t iSize;
- size_t oSize;
-
- if (!srcSize) return ERROR(srcSize_wrong);
- iSize = ip[0];
- /* memset(huffWeight, 0, hwSize); *//* is not necessary, even though some analyzer complain ... */
-
- if (iSize >= 128) { /* special header */
- oSize = iSize - 127;
- iSize = ((oSize+1)/2);
- if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
- if (oSize >= hwSize) return ERROR(corruption_detected);
- ip += 1;
- { U32 n;
- for (n=0; n<oSize; n+=2) {
- huffWeight[n] = ip[n/2] >> 4;
- huffWeight[n+1] = ip[n/2] & 15;
- } } }
- else { /* header compressed with FSE (normal case) */
- FSE_DTable fseWorkspace[FSE_DTABLE_SIZE_U32(6)]; /* 6 is max possible tableLog for HUF header (maybe even 5, to be tested) */
- if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
- oSize = FSE_decompress_wksp(huffWeight, hwSize-1, ip+1, iSize, fseWorkspace, 6); /* max (hwSize-1) values decoded, as last one is implied */
- if (FSE_isError(oSize)) return oSize;
- }
-
- /* collect weight stats */
- memset(rankStats, 0, (HUF_TABLELOG_MAX + 1) * sizeof(U32));
- weightTotal = 0;
- { U32 n; for (n=0; n<oSize; n++) {
- if (huffWeight[n] >= HUF_TABLELOG_MAX) return ERROR(corruption_detected);
- rankStats[huffWeight[n]]++;
- weightTotal += (1 << huffWeight[n]) >> 1;
- } }
- if (weightTotal == 0) return ERROR(corruption_detected);
-
- /* get last non-null symbol weight (implied, total must be 2^n) */
- { U32 const tableLog = BIT_highbit32(weightTotal) + 1;
- if (tableLog > HUF_TABLELOG_MAX) return ERROR(corruption_detected);
- *tableLogPtr = tableLog;
- /* determine last weight */
- { U32 const total = 1 << tableLog;
- U32 const rest = total - weightTotal;
- U32 const verif = 1 << BIT_highbit32(rest);
- U32 const lastWeight = BIT_highbit32(rest) + 1;
- if (verif != rest) return ERROR(corruption_detected); /* last value must be a clean power of 2 */
- huffWeight[oSize] = (BYTE)lastWeight;
- rankStats[lastWeight]++;
- } }
-
- /* check tree construction validity */
- if ((rankStats[1] < 2) || (rankStats[1] & 1)) return ERROR(corruption_detected); /* by construction : at least 2 elts of rank 1, must be even */
-
- /* results */
- *nbSymbolsPtr = (U32)(oSize+1);
- return iSize+1;
-}
diff --git a/vendor/github.com/DataDog/zstd/error_private.c b/vendor/github.com/DataDog/zstd/error_private.c
deleted file mode 100644
index 7c1bb67..0000000
--- a/vendor/github.com/DataDog/zstd/error_private.c
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-/* The purpose of this file is to have a single list of error strings embedded in binary */
-
-#include "error_private.h"
-
-const char* ERR_getErrorString(ERR_enum code)
-{
-#ifdef ZSTD_STRIP_ERROR_STRINGS
- (void)code;
- return "Error strings stripped";
-#else
- static const char* const notErrorCode = "Unspecified error code";
- switch( code )
- {
- case PREFIX(no_error): return "No error detected";
- case PREFIX(GENERIC): return "Error (generic)";
- case PREFIX(prefix_unknown): return "Unknown frame descriptor";
- case PREFIX(version_unsupported): return "Version not supported";
- case PREFIX(frameParameter_unsupported): return "Unsupported frame parameter";
- case PREFIX(frameParameter_windowTooLarge): return "Frame requires too much memory for decoding";
- case PREFIX(corruption_detected): return "Corrupted block detected";
- case PREFIX(checksum_wrong): return "Restored data doesn't match checksum";
- case PREFIX(parameter_unsupported): return "Unsupported parameter";
- case PREFIX(parameter_outOfBound): return "Parameter is out of bound";
- case PREFIX(init_missing): return "Context should be init first";
- case PREFIX(memory_allocation): return "Allocation error : not enough memory";
- case PREFIX(workSpace_tooSmall): return "workSpace buffer is not large enough";
- case PREFIX(stage_wrong): return "Operation not authorized at current processing stage";
- case PREFIX(tableLog_tooLarge): return "tableLog requires too much memory : unsupported";
- case PREFIX(maxSymbolValue_tooLarge): return "Unsupported max Symbol Value : too large";
- case PREFIX(maxSymbolValue_tooSmall): return "Specified maxSymbolValue is too small";
- case PREFIX(dictionary_corrupted): return "Dictionary is corrupted";
- case PREFIX(dictionary_wrong): return "Dictionary mismatch";
- case PREFIX(dictionaryCreation_failed): return "Cannot create Dictionary from provided samples";
- case PREFIX(dstSize_tooSmall): return "Destination buffer is too small";
- case PREFIX(srcSize_wrong): return "Src size is incorrect";
- case PREFIX(dstBuffer_null): return "Operation on NULL destination buffer";
- /* following error codes are not stable and may be removed or changed in a future version */
- case PREFIX(frameIndex_tooLarge): return "Frame index is too large";
- case PREFIX(seekableIO): return "An I/O error occurred when reading/seeking";
- case PREFIX(maxCode):
- default: return notErrorCode;
- }
-#endif
-}
diff --git a/vendor/github.com/DataDog/zstd/error_private.h b/vendor/github.com/DataDog/zstd/error_private.h
deleted file mode 100644
index 0d2fa7e..0000000
--- a/vendor/github.com/DataDog/zstd/error_private.h
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-/* Note : this module is expected to remain private, do not expose it */
-
-#ifndef ERROR_H_MODULE
-#define ERROR_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/* ****************************************
-* Dependencies
-******************************************/
-#include <stddef.h> /* size_t */
-#include "zstd_errors.h" /* enum list */
-
-
-/* ****************************************
-* Compiler-specific
-******************************************/
-#if defined(__GNUC__)
-# define ERR_STATIC static __attribute__((unused))
-#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-# define ERR_STATIC static inline
-#elif defined(_MSC_VER)
-# define ERR_STATIC static __inline
-#else
-# define ERR_STATIC static /* this version may generate warnings for unused static functions; disable the relevant warning */
-#endif
-
-
-/*-****************************************
-* Customization (error_public.h)
-******************************************/
-typedef ZSTD_ErrorCode ERR_enum;
-#define PREFIX(name) ZSTD_error_##name
-
-
-/*-****************************************
-* Error codes handling
-******************************************/
-#undef ERROR /* reported already defined on VS 2015 (Rich Geldreich) */
-#define ERROR(name) ZSTD_ERROR(name)
-#define ZSTD_ERROR(name) ((size_t)-PREFIX(name))
-
-ERR_STATIC unsigned ERR_isError(size_t code) { return (code > ERROR(maxCode)); }
-
-ERR_STATIC ERR_enum ERR_getErrorCode(size_t code) { if (!ERR_isError(code)) return (ERR_enum)0; return (ERR_enum) (0-code); }
-
-
-/*-****************************************
-* Error Strings
-******************************************/
-
-const char* ERR_getErrorString(ERR_enum code); /* error_private.c */
-
-ERR_STATIC const char* ERR_getErrorName(size_t code)
-{
- return ERR_getErrorString(ERR_getErrorCode(code));
-}
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ERROR_H_MODULE */
diff --git a/vendor/github.com/DataDog/zstd/errors.go b/vendor/github.com/DataDog/zstd/errors.go
deleted file mode 100644
index 38db0d5..0000000
--- a/vendor/github.com/DataDog/zstd/errors.go
+++ /dev/null
@@ -1,35 +0,0 @@
-package zstd
-
-/*
-#define ZSTD_STATIC_LINKING_ONLY
-#include "zstd.h"
-*/
-import "C"
-
-// ErrorCode is an error returned by the zstd library.
-type ErrorCode int
-
-// Error returns the error string given by zstd
-func (e ErrorCode) Error() string {
- return C.GoString(C.ZSTD_getErrorName(C.size_t(e)))
-}
-
-func cIsError(code int) bool {
- return int(C.ZSTD_isError(C.size_t(code))) != 0
-}
-
-// getError returns an error for the return code, or nil if it's not an error
-func getError(code int) error {
- if code < 0 && cIsError(code) {
- return ErrorCode(code)
- }
- return nil
-}
-
-// IsDstSizeTooSmallError returns whether the error correspond to zstd standard sDstSizeTooSmall error
-func IsDstSizeTooSmallError(e error) bool {
- if e != nil && e.Error() == "Destination buffer is too small" {
- return true
- }
- return false
-}
diff --git a/vendor/github.com/DataDog/zstd/fastcover.c b/vendor/github.com/DataDog/zstd/fastcover.c
deleted file mode 100644
index 941bb5a..0000000
--- a/vendor/github.com/DataDog/zstd/fastcover.c
+++ /dev/null
@@ -1,747 +0,0 @@
-/*-*************************************
-* Dependencies
-***************************************/
-#include <stdio.h> /* fprintf */
-#include <stdlib.h> /* malloc, free, qsort */
-#include <string.h> /* memset */
-#include <time.h> /* clock */
-
-#include "mem.h" /* read */
-#include "pool.h"
-#include "threading.h"
-#include "cover.h"
-#include "zstd_internal.h" /* includes zstd.h */
-#ifndef ZDICT_STATIC_LINKING_ONLY
-#define ZDICT_STATIC_LINKING_ONLY
-#endif
-#include "zdict.h"
-
-
-/*-*************************************
-* Constants
-***************************************/
-#define FASTCOVER_MAX_SAMPLES_SIZE (sizeof(size_t) == 8 ? ((unsigned)-1) : ((unsigned)1 GB))
-#define FASTCOVER_MAX_F 31
-#define FASTCOVER_MAX_ACCEL 10
-#define DEFAULT_SPLITPOINT 0.75
-#define DEFAULT_F 20
-#define DEFAULT_ACCEL 1
-
-
-/*-*************************************
-* Console display
-***************************************/
-static int g_displayLevel = 2;
-#define DISPLAY(...) \
- { \
- fprintf(stderr, __VA_ARGS__); \
- fflush(stderr); \
- }
-#define LOCALDISPLAYLEVEL(displayLevel, l, ...) \
- if (displayLevel >= l) { \
- DISPLAY(__VA_ARGS__); \
- } /* 0 : no display; 1: errors; 2: default; 3: details; 4: debug */
-#define DISPLAYLEVEL(l, ...) LOCALDISPLAYLEVEL(g_displayLevel, l, __VA_ARGS__)
-
-#define LOCALDISPLAYUPDATE(displayLevel, l, ...) \
- if (displayLevel >= l) { \
- if ((clock() - g_time > refreshRate) || (displayLevel >= 4)) { \
- g_time = clock(); \
- DISPLAY(__VA_ARGS__); \
- } \
- }
-#define DISPLAYUPDATE(l, ...) LOCALDISPLAYUPDATE(g_displayLevel, l, __VA_ARGS__)
-static const clock_t refreshRate = CLOCKS_PER_SEC * 15 / 100;
-static clock_t g_time = 0;
-
-
-/*-*************************************
-* Hash Functions
-***************************************/
-static const U64 prime6bytes = 227718039650203ULL;
-static size_t ZSTD_hash6(U64 u, U32 h) { return (size_t)(((u << (64-48)) * prime6bytes) >> (64-h)) ; }
-static size_t ZSTD_hash6Ptr(const void* p, U32 h) { return ZSTD_hash6(MEM_readLE64(p), h); }
-
-static const U64 prime8bytes = 0xCF1BBCDCB7A56463ULL;
-static size_t ZSTD_hash8(U64 u, U32 h) { return (size_t)(((u) * prime8bytes) >> (64-h)) ; }
-static size_t ZSTD_hash8Ptr(const void* p, U32 h) { return ZSTD_hash8(MEM_readLE64(p), h); }
-
-
-/**
- * Hash the d-byte value pointed to by p and mod 2^f
- */
-static size_t FASTCOVER_hashPtrToIndex(const void* p, U32 h, unsigned d) {
- if (d == 6) {
- return ZSTD_hash6Ptr(p, h) & ((1 << h) - 1);
- }
- return ZSTD_hash8Ptr(p, h) & ((1 << h) - 1);
-}
-
-
-/*-*************************************
-* Acceleration
-***************************************/
-typedef struct {
- unsigned finalize; /* Percentage of training samples used for ZDICT_finalizeDictionary */
- unsigned skip; /* Number of dmer skipped between each dmer counted in computeFrequency */
-} FASTCOVER_accel_t;
-
-
-static const FASTCOVER_accel_t FASTCOVER_defaultAccelParameters[FASTCOVER_MAX_ACCEL+1] = {
- { 100, 0 }, /* accel = 0, should not happen because accel = 0 defaults to accel = 1 */
- { 100, 0 }, /* accel = 1 */
- { 50, 1 }, /* accel = 2 */
- { 34, 2 }, /* accel = 3 */
- { 25, 3 }, /* accel = 4 */
- { 20, 4 }, /* accel = 5 */
- { 17, 5 }, /* accel = 6 */
- { 14, 6 }, /* accel = 7 */
- { 13, 7 }, /* accel = 8 */
- { 11, 8 }, /* accel = 9 */
- { 10, 9 }, /* accel = 10 */
-};
-
-
-/*-*************************************
-* Context
-***************************************/
-typedef struct {
- const BYTE *samples;
- size_t *offsets;
- const size_t *samplesSizes;
- size_t nbSamples;
- size_t nbTrainSamples;
- size_t nbTestSamples;
- size_t nbDmers;
- U32 *freqs;
- unsigned d;
- unsigned f;
- FASTCOVER_accel_t accelParams;
-} FASTCOVER_ctx_t;
-
-
-/*-*************************************
-* Helper functions
-***************************************/
-/**
- * Selects the best segment in an epoch.
- * Segments of are scored according to the function:
- *
- * Let F(d) be the frequency of all dmers with hash value d.
- * Let S_i be hash value of the dmer at position i of segment S which has length k.
- *
- * Score(S) = F(S_1) + F(S_2) + ... + F(S_{k-d+1})
- *
- * Once the dmer with hash value d is in the dictionary we set F(d) = 0.
- */
-static COVER_segment_t FASTCOVER_selectSegment(const FASTCOVER_ctx_t *ctx,
- U32 *freqs, U32 begin, U32 end,
- ZDICT_cover_params_t parameters,
- U16* segmentFreqs) {
- /* Constants */
- const U32 k = parameters.k;
- const U32 d = parameters.d;
- const U32 f = ctx->f;
- const U32 dmersInK = k - d + 1;
-
- /* Try each segment (activeSegment) and save the best (bestSegment) */
- COVER_segment_t bestSegment = {0, 0, 0};
- COVER_segment_t activeSegment;
-
- /* Reset the activeDmers in the segment */
- /* The activeSegment starts at the beginning of the epoch. */
- activeSegment.begin = begin;
- activeSegment.end = begin;
- activeSegment.score = 0;
-
- /* Slide the activeSegment through the whole epoch.
- * Save the best segment in bestSegment.
- */
- while (activeSegment.end < end) {
- /* Get hash value of current dmer */
- const size_t idx = FASTCOVER_hashPtrToIndex(ctx->samples + activeSegment.end, f, d);
-
- /* Add frequency of this index to score if this is the first occurrence of index in active segment */
- if (segmentFreqs[idx] == 0) {
- activeSegment.score += freqs[idx];
- }
- /* Increment end of segment and segmentFreqs*/
- activeSegment.end += 1;
- segmentFreqs[idx] += 1;
- /* If the window is now too large, drop the first position */
- if (activeSegment.end - activeSegment.begin == dmersInK + 1) {
- /* Get hash value of the dmer to be eliminated from active segment */
- const size_t delIndex = FASTCOVER_hashPtrToIndex(ctx->samples + activeSegment.begin, f, d);
- segmentFreqs[delIndex] -= 1;
- /* Subtract frequency of this index from score if this is the last occurrence of this index in active segment */
- if (segmentFreqs[delIndex] == 0) {
- activeSegment.score -= freqs[delIndex];
- }
- /* Increment start of segment */
- activeSegment.begin += 1;
- }
-
- /* If this segment is the best so far save it */
- if (activeSegment.score > bestSegment.score) {
- bestSegment = activeSegment;
- }
- }
-
- /* Zero out rest of segmentFreqs array */
- while (activeSegment.begin < end) {
- const size_t delIndex = FASTCOVER_hashPtrToIndex(ctx->samples + activeSegment.begin, f, d);
- segmentFreqs[delIndex] -= 1;
- activeSegment.begin += 1;
- }
-
- {
- /* Zero the frequency of hash value of each dmer covered by the chosen segment. */
- U32 pos;
- for (pos = bestSegment.begin; pos != bestSegment.end; ++pos) {
- const size_t i = FASTCOVER_hashPtrToIndex(ctx->samples + pos, f, d);
- freqs[i] = 0;
- }
- }
-
- return bestSegment;
-}
-
-
-static int FASTCOVER_checkParameters(ZDICT_cover_params_t parameters,
- size_t maxDictSize, unsigned f,
- unsigned accel) {
- /* k, d, and f are required parameters */
- if (parameters.d == 0 || parameters.k == 0) {
- return 0;
- }
- /* d has to be 6 or 8 */
- if (parameters.d != 6 && parameters.d != 8) {
- return 0;
- }
- /* k <= maxDictSize */
- if (parameters.k > maxDictSize) {
- return 0;
- }
- /* d <= k */
- if (parameters.d > parameters.k) {
- return 0;
- }
- /* 0 < f <= FASTCOVER_MAX_F*/
- if (f > FASTCOVER_MAX_F || f == 0) {
- return 0;
- }
- /* 0 < splitPoint <= 1 */
- if (parameters.splitPoint <= 0 || parameters.splitPoint > 1) {
- return 0;
- }
- /* 0 < accel <= 10 */
- if (accel > 10 || accel == 0) {
- return 0;
- }
- return 1;
-}
-
-
-/**
- * Clean up a context initialized with `FASTCOVER_ctx_init()`.
- */
-static void
-FASTCOVER_ctx_destroy(FASTCOVER_ctx_t* ctx)
-{
- if (!ctx) return;
-
- free(ctx->freqs);
- ctx->freqs = NULL;
-
- free(ctx->offsets);
- ctx->offsets = NULL;
-}
-
-
-/**
- * Calculate for frequency of hash value of each dmer in ctx->samples
- */
-static void
-FASTCOVER_computeFrequency(U32* freqs, const FASTCOVER_ctx_t* ctx)
-{
- const unsigned f = ctx->f;
- const unsigned d = ctx->d;
- const unsigned skip = ctx->accelParams.skip;
- const unsigned readLength = MAX(d, 8);
- size_t i;
- assert(ctx->nbTrainSamples >= 5);
- assert(ctx->nbTrainSamples <= ctx->nbSamples);
- for (i = 0; i < ctx->nbTrainSamples; i++) {
- size_t start = ctx->offsets[i]; /* start of current dmer */
- size_t const currSampleEnd = ctx->offsets[i+1];
- while (start + readLength <= currSampleEnd) {
- const size_t dmerIndex = FASTCOVER_hashPtrToIndex(ctx->samples + start, f, d);
- freqs[dmerIndex]++;
- start = start + skip + 1;
- }
- }
-}
-
-
-/**
- * Prepare a context for dictionary building.
- * The context is only dependent on the parameter `d` and can used multiple
- * times.
- * Returns 0 on success or error code on error.
- * The context must be destroyed with `FASTCOVER_ctx_destroy()`.
- */
-static size_t
-FASTCOVER_ctx_init(FASTCOVER_ctx_t* ctx,
- const void* samplesBuffer,
- const size_t* samplesSizes, unsigned nbSamples,
- unsigned d, double splitPoint, unsigned f,
- FASTCOVER_accel_t accelParams)
-{
- const BYTE* const samples = (const BYTE*)samplesBuffer;
- const size_t totalSamplesSize = COVER_sum(samplesSizes, nbSamples);
- /* Split samples into testing and training sets */
- const unsigned nbTrainSamples = splitPoint < 1.0 ? (unsigned)((double)nbSamples * splitPoint) : nbSamples;
- const unsigned nbTestSamples = splitPoint < 1.0 ? nbSamples - nbTrainSamples : nbSamples;
- const size_t trainingSamplesSize = splitPoint < 1.0 ? COVER_sum(samplesSizes, nbTrainSamples) : totalSamplesSize;
- const size_t testSamplesSize = splitPoint < 1.0 ? COVER_sum(samplesSizes + nbTrainSamples, nbTestSamples) : totalSamplesSize;
-
- /* Checks */
- if (totalSamplesSize < MAX(d, sizeof(U64)) ||
- totalSamplesSize >= (size_t)FASTCOVER_MAX_SAMPLES_SIZE) {
- DISPLAYLEVEL(1, "Total samples size is too large (%u MB), maximum size is %u MB\n",
- (unsigned)(totalSamplesSize >> 20), (FASTCOVER_MAX_SAMPLES_SIZE >> 20));
- return ERROR(srcSize_wrong);
- }
-
- /* Check if there are at least 5 training samples */
- if (nbTrainSamples < 5) {
- DISPLAYLEVEL(1, "Total number of training samples is %u and is invalid\n", nbTrainSamples);
- return ERROR(srcSize_wrong);
- }
-
- /* Check if there's testing sample */
- if (nbTestSamples < 1) {
- DISPLAYLEVEL(1, "Total number of testing samples is %u and is invalid.\n", nbTestSamples);
- return ERROR(srcSize_wrong);
- }
-
- /* Zero the context */
- memset(ctx, 0, sizeof(*ctx));
- DISPLAYLEVEL(2, "Training on %u samples of total size %u\n", nbTrainSamples,
- (unsigned)trainingSamplesSize);
- DISPLAYLEVEL(2, "Testing on %u samples of total size %u\n", nbTestSamples,
- (unsigned)testSamplesSize);
-
- ctx->samples = samples;
- ctx->samplesSizes = samplesSizes;
- ctx->nbSamples = nbSamples;
- ctx->nbTrainSamples = nbTrainSamples;
- ctx->nbTestSamples = nbTestSamples;
- ctx->nbDmers = trainingSamplesSize - MAX(d, sizeof(U64)) + 1;
- ctx->d = d;
- ctx->f = f;
- ctx->accelParams = accelParams;
-
- /* The offsets of each file */
- ctx->offsets = (size_t*)calloc((nbSamples + 1), sizeof(size_t));
- if (ctx->offsets == NULL) {
- DISPLAYLEVEL(1, "Failed to allocate scratch buffers \n");
- FASTCOVER_ctx_destroy(ctx);
- return ERROR(memory_allocation);
- }
-
- /* Fill offsets from the samplesSizes */
- { U32 i;
- ctx->offsets[0] = 0;
- assert(nbSamples >= 5);
- for (i = 1; i <= nbSamples; ++i) {
- ctx->offsets[i] = ctx->offsets[i - 1] + samplesSizes[i - 1];
- }
- }
-
- /* Initialize frequency array of size 2^f */
- ctx->freqs = (U32*)calloc(((U64)1 << f), sizeof(U32));
- if (ctx->freqs == NULL) {
- DISPLAYLEVEL(1, "Failed to allocate frequency table \n");
- FASTCOVER_ctx_destroy(ctx);
- return ERROR(memory_allocation);
- }
-
- DISPLAYLEVEL(2, "Computing frequencies\n");
- FASTCOVER_computeFrequency(ctx->freqs, ctx);
-
- return 0;
-}
-
-
-/**
- * Given the prepared context build the dictionary.
- */
-static size_t
-FASTCOVER_buildDictionary(const FASTCOVER_ctx_t* ctx,
- U32* freqs,
- void* dictBuffer, size_t dictBufferCapacity,
- ZDICT_cover_params_t parameters,
- U16* segmentFreqs)
-{
- BYTE *const dict = (BYTE *)dictBuffer;
- size_t tail = dictBufferCapacity;
- /* Divide the data into epochs. We will select one segment from each epoch. */
- const COVER_epoch_info_t epochs = COVER_computeEpochs(
- (U32)dictBufferCapacity, (U32)ctx->nbDmers, parameters.k, 1);
- const size_t maxZeroScoreRun = 10;
- size_t zeroScoreRun = 0;
- size_t epoch;
- DISPLAYLEVEL(2, "Breaking content into %u epochs of size %u\n",
- (U32)epochs.num, (U32)epochs.size);
- /* Loop through the epochs until there are no more segments or the dictionary
- * is full.
- */
- for (epoch = 0; tail > 0; epoch = (epoch + 1) % epochs.num) {
- const U32 epochBegin = (U32)(epoch * epochs.size);
- const U32 epochEnd = epochBegin + epochs.size;
- size_t segmentSize;
- /* Select a segment */
- COVER_segment_t segment = FASTCOVER_selectSegment(
- ctx, freqs, epochBegin, epochEnd, parameters, segmentFreqs);
-
- /* If the segment covers no dmers, then we are out of content.
- * There may be new content in other epochs, for continue for some time.
- */
- if (segment.score == 0) {
- if (++zeroScoreRun >= maxZeroScoreRun) {
- break;
- }
- continue;
- }
- zeroScoreRun = 0;
-
- /* Trim the segment if necessary and if it is too small then we are done */
- segmentSize = MIN(segment.end - segment.begin + parameters.d - 1, tail);
- if (segmentSize < parameters.d) {
- break;
- }
-
- /* We fill the dictionary from the back to allow the best segments to be
- * referenced with the smallest offsets.
- */
- tail -= segmentSize;
- memcpy(dict + tail, ctx->samples + segment.begin, segmentSize);
- DISPLAYUPDATE(
- 2, "\r%u%% ",
- (unsigned)(((dictBufferCapacity - tail) * 100) / dictBufferCapacity));
- }
- DISPLAYLEVEL(2, "\r%79s\r", "");
- return tail;
-}
-
-/**
- * Parameters for FASTCOVER_tryParameters().
- */
-typedef struct FASTCOVER_tryParameters_data_s {
- const FASTCOVER_ctx_t* ctx;
- COVER_best_t* best;
- size_t dictBufferCapacity;
- ZDICT_cover_params_t parameters;
-} FASTCOVER_tryParameters_data_t;
-
-
-/**
- * Tries a set of parameters and updates the COVER_best_t with the results.
- * This function is thread safe if zstd is compiled with multithreaded support.
- * It takes its parameters as an *OWNING* opaque pointer to support threading.
- */
-static void FASTCOVER_tryParameters(void *opaque)
-{
- /* Save parameters as local variables */
- FASTCOVER_tryParameters_data_t *const data = (FASTCOVER_tryParameters_data_t *)opaque;
- const FASTCOVER_ctx_t *const ctx = data->ctx;
- const ZDICT_cover_params_t parameters = data->parameters;
- size_t dictBufferCapacity = data->dictBufferCapacity;
- size_t totalCompressedSize = ERROR(GENERIC);
- /* Initialize array to keep track of frequency of dmer within activeSegment */
- U16* segmentFreqs = (U16 *)calloc(((U64)1 << ctx->f), sizeof(U16));
- /* Allocate space for hash table, dict, and freqs */
- BYTE *const dict = (BYTE * const)malloc(dictBufferCapacity);
- COVER_dictSelection_t selection = COVER_dictSelectionError(ERROR(GENERIC));
- U32 *freqs = (U32*) malloc(((U64)1 << ctx->f) * sizeof(U32));
- if (!segmentFreqs || !dict || !freqs) {
- DISPLAYLEVEL(1, "Failed to allocate buffers: out of memory\n");
- goto _cleanup;
- }
- /* Copy the frequencies because we need to modify them */
- memcpy(freqs, ctx->freqs, ((U64)1 << ctx->f) * sizeof(U32));
- /* Build the dictionary */
- { const size_t tail = FASTCOVER_buildDictionary(ctx, freqs, dict, dictBufferCapacity,
- parameters, segmentFreqs);
-
- const unsigned nbFinalizeSamples = (unsigned)(ctx->nbTrainSamples * ctx->accelParams.finalize / 100);
- selection = COVER_selectDict(dict + tail, dictBufferCapacity - tail,
- ctx->samples, ctx->samplesSizes, nbFinalizeSamples, ctx->nbTrainSamples, ctx->nbSamples, parameters, ctx->offsets,
- totalCompressedSize);
-
- if (COVER_dictSelectionIsError(selection)) {
- DISPLAYLEVEL(1, "Failed to select dictionary\n");
- goto _cleanup;
- }
- }
-_cleanup:
- free(dict);
- COVER_best_finish(data->best, parameters, selection);
- free(data);
- free(segmentFreqs);
- COVER_dictSelectionFree(selection);
- free(freqs);
-}
-
-
-static void
-FASTCOVER_convertToCoverParams(ZDICT_fastCover_params_t fastCoverParams,
- ZDICT_cover_params_t* coverParams)
-{
- coverParams->k = fastCoverParams.k;
- coverParams->d = fastCoverParams.d;
- coverParams->steps = fastCoverParams.steps;
- coverParams->nbThreads = fastCoverParams.nbThreads;
- coverParams->splitPoint = fastCoverParams.splitPoint;
- coverParams->zParams = fastCoverParams.zParams;
- coverParams->shrinkDict = fastCoverParams.shrinkDict;
-}
-
-
-static void
-FASTCOVER_convertToFastCoverParams(ZDICT_cover_params_t coverParams,
- ZDICT_fastCover_params_t* fastCoverParams,
- unsigned f, unsigned accel)
-{
- fastCoverParams->k = coverParams.k;
- fastCoverParams->d = coverParams.d;
- fastCoverParams->steps = coverParams.steps;
- fastCoverParams->nbThreads = coverParams.nbThreads;
- fastCoverParams->splitPoint = coverParams.splitPoint;
- fastCoverParams->f = f;
- fastCoverParams->accel = accel;
- fastCoverParams->zParams = coverParams.zParams;
- fastCoverParams->shrinkDict = coverParams.shrinkDict;
-}
-
-
-ZDICTLIB_API size_t
-ZDICT_trainFromBuffer_fastCover(void* dictBuffer, size_t dictBufferCapacity,
- const void* samplesBuffer,
- const size_t* samplesSizes, unsigned nbSamples,
- ZDICT_fastCover_params_t parameters)
-{
- BYTE* const dict = (BYTE*)dictBuffer;
- FASTCOVER_ctx_t ctx;
- ZDICT_cover_params_t coverParams;
- FASTCOVER_accel_t accelParams;
- /* Initialize global data */
- g_displayLevel = parameters.zParams.notificationLevel;
- /* Assign splitPoint and f if not provided */
- parameters.splitPoint = 1.0;
- parameters.f = parameters.f == 0 ? DEFAULT_F : parameters.f;
- parameters.accel = parameters.accel == 0 ? DEFAULT_ACCEL : parameters.accel;
- /* Convert to cover parameter */
- memset(&coverParams, 0 , sizeof(coverParams));
- FASTCOVER_convertToCoverParams(parameters, &coverParams);
- /* Checks */
- if (!FASTCOVER_checkParameters(coverParams, dictBufferCapacity, parameters.f,
- parameters.accel)) {
- DISPLAYLEVEL(1, "FASTCOVER parameters incorrect\n");
- return ERROR(parameter_outOfBound);
- }
- if (nbSamples == 0) {
- DISPLAYLEVEL(1, "FASTCOVER must have at least one input file\n");
- return ERROR(srcSize_wrong);
- }
- if (dictBufferCapacity < ZDICT_DICTSIZE_MIN) {
- DISPLAYLEVEL(1, "dictBufferCapacity must be at least %u\n",
- ZDICT_DICTSIZE_MIN);
- return ERROR(dstSize_tooSmall);
- }
- /* Assign corresponding FASTCOVER_accel_t to accelParams*/
- accelParams = FASTCOVER_defaultAccelParameters[parameters.accel];
- /* Initialize context */
- {
- size_t const initVal = FASTCOVER_ctx_init(&ctx, samplesBuffer, samplesSizes, nbSamples,
- coverParams.d, parameters.splitPoint, parameters.f,
- accelParams);
- if (ZSTD_isError(initVal)) {
- DISPLAYLEVEL(1, "Failed to initialize context\n");
- return initVal;
- }
- }
- COVER_warnOnSmallCorpus(dictBufferCapacity, ctx.nbDmers, g_displayLevel);
- /* Build the dictionary */
- DISPLAYLEVEL(2, "Building dictionary\n");
- {
- /* Initialize array to keep track of frequency of dmer within activeSegment */
- U16* segmentFreqs = (U16 *)calloc(((U64)1 << parameters.f), sizeof(U16));
- const size_t tail = FASTCOVER_buildDictionary(&ctx, ctx.freqs, dictBuffer,
- dictBufferCapacity, coverParams, segmentFreqs);
- const unsigned nbFinalizeSamples = (unsigned)(ctx.nbTrainSamples * ctx.accelParams.finalize / 100);
- const size_t dictionarySize = ZDICT_finalizeDictionary(
- dict, dictBufferCapacity, dict + tail, dictBufferCapacity - tail,
- samplesBuffer, samplesSizes, nbFinalizeSamples, coverParams.zParams);
- if (!ZSTD_isError(dictionarySize)) {
- DISPLAYLEVEL(2, "Constructed dictionary of size %u\n",
- (unsigned)dictionarySize);
- }
- FASTCOVER_ctx_destroy(&ctx);
- free(segmentFreqs);
- return dictionarySize;
- }
-}
-
-
-ZDICTLIB_API size_t
-ZDICT_optimizeTrainFromBuffer_fastCover(
- void* dictBuffer, size_t dictBufferCapacity,
- const void* samplesBuffer,
- const size_t* samplesSizes, unsigned nbSamples,
- ZDICT_fastCover_params_t* parameters)
-{
- ZDICT_cover_params_t coverParams;
- FASTCOVER_accel_t accelParams;
- /* constants */
- const unsigned nbThreads = parameters->nbThreads;
- const double splitPoint =
- parameters->splitPoint <= 0.0 ? DEFAULT_SPLITPOINT : parameters->splitPoint;
- const unsigned kMinD = parameters->d == 0 ? 6 : parameters->d;
- const unsigned kMaxD = parameters->d == 0 ? 8 : parameters->d;
- const unsigned kMinK = parameters->k == 0 ? 50 : parameters->k;
- const unsigned kMaxK = parameters->k == 0 ? 2000 : parameters->k;
- const unsigned kSteps = parameters->steps == 0 ? 40 : parameters->steps;
- const unsigned kStepSize = MAX((kMaxK - kMinK) / kSteps, 1);
- const unsigned kIterations =
- (1 + (kMaxD - kMinD) / 2) * (1 + (kMaxK - kMinK) / kStepSize);
- const unsigned f = parameters->f == 0 ? DEFAULT_F : parameters->f;
- const unsigned accel = parameters->accel == 0 ? DEFAULT_ACCEL : parameters->accel;
- const unsigned shrinkDict = 0;
- /* Local variables */
- const int displayLevel = parameters->zParams.notificationLevel;
- unsigned iteration = 1;
- unsigned d;
- unsigned k;
- COVER_best_t best;
- POOL_ctx *pool = NULL;
- int warned = 0;
- /* Checks */
- if (splitPoint <= 0 || splitPoint > 1) {
- LOCALDISPLAYLEVEL(displayLevel, 1, "Incorrect splitPoint\n");
- return ERROR(parameter_outOfBound);
- }
- if (accel == 0 || accel > FASTCOVER_MAX_ACCEL) {
- LOCALDISPLAYLEVEL(displayLevel, 1, "Incorrect accel\n");
- return ERROR(parameter_outOfBound);
- }
- if (kMinK < kMaxD || kMaxK < kMinK) {
- LOCALDISPLAYLEVEL(displayLevel, 1, "Incorrect k\n");
- return ERROR(parameter_outOfBound);
- }
- if (nbSamples == 0) {
- LOCALDISPLAYLEVEL(displayLevel, 1, "FASTCOVER must have at least one input file\n");
- return ERROR(srcSize_wrong);
- }
- if (dictBufferCapacity < ZDICT_DICTSIZE_MIN) {
- LOCALDISPLAYLEVEL(displayLevel, 1, "dictBufferCapacity must be at least %u\n",
- ZDICT_DICTSIZE_MIN);
- return ERROR(dstSize_tooSmall);
- }
- if (nbThreads > 1) {
- pool = POOL_create(nbThreads, 1);
- if (!pool) {
- return ERROR(memory_allocation);
- }
- }
- /* Initialization */
- COVER_best_init(&best);
- memset(&coverParams, 0 , sizeof(coverParams));
- FASTCOVER_convertToCoverParams(*parameters, &coverParams);
- accelParams = FASTCOVER_defaultAccelParameters[accel];
- /* Turn down global display level to clean up display at level 2 and below */
- g_displayLevel = displayLevel == 0 ? 0 : displayLevel - 1;
- /* Loop through d first because each new value needs a new context */
- LOCALDISPLAYLEVEL(displayLevel, 2, "Trying %u different sets of parameters\n",
- kIterations);
- for (d = kMinD; d <= kMaxD; d += 2) {
- /* Initialize the context for this value of d */
- FASTCOVER_ctx_t ctx;
- LOCALDISPLAYLEVEL(displayLevel, 3, "d=%u\n", d);
- {
- size_t const initVal = FASTCOVER_ctx_init(&ctx, samplesBuffer, samplesSizes, nbSamples, d, splitPoint, f, accelParams);
- if (ZSTD_isError(initVal)) {
- LOCALDISPLAYLEVEL(displayLevel, 1, "Failed to initialize context\n");
- COVER_best_destroy(&best);
- POOL_free(pool);
- return initVal;
- }
- }
- if (!warned) {
- COVER_warnOnSmallCorpus(dictBufferCapacity, ctx.nbDmers, displayLevel);
- warned = 1;
- }
- /* Loop through k reusing the same context */
- for (k = kMinK; k <= kMaxK; k += kStepSize) {
- /* Prepare the arguments */
- FASTCOVER_tryParameters_data_t *data = (FASTCOVER_tryParameters_data_t *)malloc(
- sizeof(FASTCOVER_tryParameters_data_t));
- LOCALDISPLAYLEVEL(displayLevel, 3, "k=%u\n", k);
- if (!data) {
- LOCALDISPLAYLEVEL(displayLevel, 1, "Failed to allocate parameters\n");
- COVER_best_destroy(&best);
- FASTCOVER_ctx_destroy(&ctx);
- POOL_free(pool);
- return ERROR(memory_allocation);
- }
- data->ctx = &ctx;
- data->best = &best;
- data->dictBufferCapacity = dictBufferCapacity;
- data->parameters = coverParams;
- data->parameters.k = k;
- data->parameters.d = d;
- data->parameters.splitPoint = splitPoint;
- data->parameters.steps = kSteps;
- data->parameters.shrinkDict = shrinkDict;
- data->parameters.zParams.notificationLevel = g_displayLevel;
- /* Check the parameters */
- if (!FASTCOVER_checkParameters(data->parameters, dictBufferCapacity,
- data->ctx->f, accel)) {
- DISPLAYLEVEL(1, "FASTCOVER parameters incorrect\n");
- free(data);
- continue;
- }
- /* Call the function and pass ownership of data to it */
- COVER_best_start(&best);
- if (pool) {
- POOL_add(pool, &FASTCOVER_tryParameters, data);
- } else {
- FASTCOVER_tryParameters(data);
- }
- /* Print status */
- LOCALDISPLAYUPDATE(displayLevel, 2, "\r%u%% ",
- (unsigned)((iteration * 100) / kIterations));
- ++iteration;
- }
- COVER_best_wait(&best);
- FASTCOVER_ctx_destroy(&ctx);
- }
- LOCALDISPLAYLEVEL(displayLevel, 2, "\r%79s\r", "");
- /* Fill the output buffer and parameters with output of the best parameters */
- {
- const size_t dictSize = best.dictSize;
- if (ZSTD_isError(best.compressedSize)) {
- const size_t compressedSize = best.compressedSize;
- COVER_best_destroy(&best);
- POOL_free(pool);
- return compressedSize;
- }
- FASTCOVER_convertToFastCoverParams(best.parameters, parameters, f, accel);
- memcpy(dictBuffer, best.dict, dictSize);
- COVER_best_destroy(&best);
- POOL_free(pool);
- return dictSize;
- }
-
-}
diff --git a/vendor/github.com/DataDog/zstd/fse.h b/vendor/github.com/DataDog/zstd/fse.h
deleted file mode 100644
index 811c670..0000000
--- a/vendor/github.com/DataDog/zstd/fse.h
+++ /dev/null
@@ -1,708 +0,0 @@
-/* ******************************************************************
- FSE : Finite State Entropy codec
- Public Prototypes declaration
- Copyright (C) 2013-2016, Yann Collet.
-
- BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
- * Redistributions of source code must retain the above copyright
- notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
- copyright notice, this list of conditions and the following disclaimer
- in the documentation and/or other materials provided with the
- distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
- "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
- LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
- A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
- OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
- OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
- You can contact the author at :
- - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#ifndef FSE_H
-#define FSE_H
-
-
-/*-*****************************************
-* Dependencies
-******************************************/
-#include <stddef.h> /* size_t, ptrdiff_t */
-
-
-/*-*****************************************
-* FSE_PUBLIC_API : control library symbols visibility
-******************************************/
-#if defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) && defined(__GNUC__) && (__GNUC__ >= 4)
-# define FSE_PUBLIC_API __attribute__ ((visibility ("default")))
-#elif defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) /* Visual expected */
-# define FSE_PUBLIC_API __declspec(dllexport)
-#elif defined(FSE_DLL_IMPORT) && (FSE_DLL_IMPORT==1)
-# define FSE_PUBLIC_API __declspec(dllimport) /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/
-#else
-# define FSE_PUBLIC_API
-#endif
-
-/*------ Version ------*/
-#define FSE_VERSION_MAJOR 0
-#define FSE_VERSION_MINOR 9
-#define FSE_VERSION_RELEASE 0
-
-#define FSE_LIB_VERSION FSE_VERSION_MAJOR.FSE_VERSION_MINOR.FSE_VERSION_RELEASE
-#define FSE_QUOTE(str) #str
-#define FSE_EXPAND_AND_QUOTE(str) FSE_QUOTE(str)
-#define FSE_VERSION_STRING FSE_EXPAND_AND_QUOTE(FSE_LIB_VERSION)
-
-#define FSE_VERSION_NUMBER (FSE_VERSION_MAJOR *100*100 + FSE_VERSION_MINOR *100 + FSE_VERSION_RELEASE)
-FSE_PUBLIC_API unsigned FSE_versionNumber(void); /**< library version number; to be used when checking dll version */
-
-
-/*-****************************************
-* FSE simple functions
-******************************************/
-/*! FSE_compress() :
- Compress content of buffer 'src', of size 'srcSize', into destination buffer 'dst'.
- 'dst' buffer must be already allocated. Compression runs faster is dstCapacity >= FSE_compressBound(srcSize).
- @return : size of compressed data (<= dstCapacity).
- Special values : if return == 0, srcData is not compressible => Nothing is stored within dst !!!
- if return == 1, srcData is a single byte symbol * srcSize times. Use RLE compression instead.
- if FSE_isError(return), compression failed (more details using FSE_getErrorName())
-*/
-FSE_PUBLIC_API size_t FSE_compress(void* dst, size_t dstCapacity,
- const void* src, size_t srcSize);
-
-/*! FSE_decompress():
- Decompress FSE data from buffer 'cSrc', of size 'cSrcSize',
- into already allocated destination buffer 'dst', of size 'dstCapacity'.
- @return : size of regenerated data (<= maxDstSize),
- or an error code, which can be tested using FSE_isError() .
-
- ** Important ** : FSE_decompress() does not decompress non-compressible nor RLE data !!!
- Why ? : making this distinction requires a header.
- Header management is intentionally delegated to the user layer, which can better manage special cases.
-*/
-FSE_PUBLIC_API size_t FSE_decompress(void* dst, size_t dstCapacity,
- const void* cSrc, size_t cSrcSize);
-
-
-/*-*****************************************
-* Tool functions
-******************************************/
-FSE_PUBLIC_API size_t FSE_compressBound(size_t size); /* maximum compressed size */
-
-/* Error Management */
-FSE_PUBLIC_API unsigned FSE_isError(size_t code); /* tells if a return value is an error code */
-FSE_PUBLIC_API const char* FSE_getErrorName(size_t code); /* provides error code string (useful for debugging) */
-
-
-/*-*****************************************
-* FSE advanced functions
-******************************************/
-/*! FSE_compress2() :
- Same as FSE_compress(), but allows the selection of 'maxSymbolValue' and 'tableLog'
- Both parameters can be defined as '0' to mean : use default value
- @return : size of compressed data
- Special values : if return == 0, srcData is not compressible => Nothing is stored within cSrc !!!
- if return == 1, srcData is a single byte symbol * srcSize times. Use RLE compression.
- if FSE_isError(return), it's an error code.
-*/
-FSE_PUBLIC_API size_t FSE_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog);
-
-
-/*-*****************************************
-* FSE detailed API
-******************************************/
-/*!
-FSE_compress() does the following:
-1. count symbol occurrence from source[] into table count[] (see hist.h)
-2. normalize counters so that sum(count[]) == Power_of_2 (2^tableLog)
-3. save normalized counters to memory buffer using writeNCount()
-4. build encoding table 'CTable' from normalized counters
-5. encode the data stream using encoding table 'CTable'
-
-FSE_decompress() does the following:
-1. read normalized counters with readNCount()
-2. build decoding table 'DTable' from normalized counters
-3. decode the data stream using decoding table 'DTable'
-
-The following API allows targeting specific sub-functions for advanced tasks.
-For example, it's possible to compress several blocks using the same 'CTable',
-or to save and provide normalized distribution using external method.
-*/
-
-/* *** COMPRESSION *** */
-
-/*! FSE_optimalTableLog():
- dynamically downsize 'tableLog' when conditions are met.
- It saves CPU time, by using smaller tables, while preserving or even improving compression ratio.
- @return : recommended tableLog (necessarily <= 'maxTableLog') */
-FSE_PUBLIC_API unsigned FSE_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue);
-
-/*! FSE_normalizeCount():
- normalize counts so that sum(count[]) == Power_of_2 (2^tableLog)
- 'normalizedCounter' is a table of short, of minimum size (maxSymbolValue+1).
- @return : tableLog,
- or an errorCode, which can be tested using FSE_isError() */
-FSE_PUBLIC_API size_t FSE_normalizeCount(short* normalizedCounter, unsigned tableLog,
- const unsigned* count, size_t srcSize, unsigned maxSymbolValue);
-
-/*! FSE_NCountWriteBound():
- Provides the maximum possible size of an FSE normalized table, given 'maxSymbolValue' and 'tableLog'.
- Typically useful for allocation purpose. */
-FSE_PUBLIC_API size_t FSE_NCountWriteBound(unsigned maxSymbolValue, unsigned tableLog);
-
-/*! FSE_writeNCount():
- Compactly save 'normalizedCounter' into 'buffer'.
- @return : size of the compressed table,
- or an errorCode, which can be tested using FSE_isError(). */
-FSE_PUBLIC_API size_t FSE_writeNCount (void* buffer, size_t bufferSize,
- const short* normalizedCounter,
- unsigned maxSymbolValue, unsigned tableLog);
-
-/*! Constructor and Destructor of FSE_CTable.
- Note that FSE_CTable size depends on 'tableLog' and 'maxSymbolValue' */
-typedef unsigned FSE_CTable; /* don't allocate that. It's only meant to be more restrictive than void* */
-FSE_PUBLIC_API FSE_CTable* FSE_createCTable (unsigned maxSymbolValue, unsigned tableLog);
-FSE_PUBLIC_API void FSE_freeCTable (FSE_CTable* ct);
-
-/*! FSE_buildCTable():
- Builds `ct`, which must be already allocated, using FSE_createCTable().
- @return : 0, or an errorCode, which can be tested using FSE_isError() */
-FSE_PUBLIC_API size_t FSE_buildCTable(FSE_CTable* ct, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
-
-/*! FSE_compress_usingCTable():
- Compress `src` using `ct` into `dst` which must be already allocated.
- @return : size of compressed data (<= `dstCapacity`),
- or 0 if compressed data could not fit into `dst`,
- or an errorCode, which can be tested using FSE_isError() */
-FSE_PUBLIC_API size_t FSE_compress_usingCTable (void* dst, size_t dstCapacity, const void* src, size_t srcSize, const FSE_CTable* ct);
-
-/*!
-Tutorial :
-----------
-The first step is to count all symbols. FSE_count() does this job very fast.
-Result will be saved into 'count', a table of unsigned int, which must be already allocated, and have 'maxSymbolValuePtr[0]+1' cells.
-'src' is a table of bytes of size 'srcSize'. All values within 'src' MUST be <= maxSymbolValuePtr[0]
-maxSymbolValuePtr[0] will be updated, with its real value (necessarily <= original value)
-FSE_count() will return the number of occurrence of the most frequent symbol.
-This can be used to know if there is a single symbol within 'src', and to quickly evaluate its compressibility.
-If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()).
-
-The next step is to normalize the frequencies.
-FSE_normalizeCount() will ensure that sum of frequencies is == 2 ^'tableLog'.
-It also guarantees a minimum of 1 to any Symbol with frequency >= 1.
-You can use 'tableLog'==0 to mean "use default tableLog value".
-If you are unsure of which tableLog value to use, you can ask FSE_optimalTableLog(),
-which will provide the optimal valid tableLog given sourceSize, maxSymbolValue, and a user-defined maximum (0 means "default").
-
-The result of FSE_normalizeCount() will be saved into a table,
-called 'normalizedCounter', which is a table of signed short.
-'normalizedCounter' must be already allocated, and have at least 'maxSymbolValue+1' cells.
-The return value is tableLog if everything proceeded as expected.
-It is 0 if there is a single symbol within distribution.
-If there is an error (ex: invalid tableLog value), the function will return an ErrorCode (which can be tested using FSE_isError()).
-
-'normalizedCounter' can be saved in a compact manner to a memory area using FSE_writeNCount().
-'buffer' must be already allocated.
-For guaranteed success, buffer size must be at least FSE_headerBound().
-The result of the function is the number of bytes written into 'buffer'.
-If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError(); ex : buffer size too small).
-
-'normalizedCounter' can then be used to create the compression table 'CTable'.
-The space required by 'CTable' must be already allocated, using FSE_createCTable().
-You can then use FSE_buildCTable() to fill 'CTable'.
-If there is an error, both functions will return an ErrorCode (which can be tested using FSE_isError()).
-
-'CTable' can then be used to compress 'src', with FSE_compress_usingCTable().
-Similar to FSE_count(), the convention is that 'src' is assumed to be a table of char of size 'srcSize'
-The function returns the size of compressed data (without header), necessarily <= `dstCapacity`.
-If it returns '0', compressed data could not fit into 'dst'.
-If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()).
-*/
-
-
-/* *** DECOMPRESSION *** */
-
-/*! FSE_readNCount():
- Read compactly saved 'normalizedCounter' from 'rBuffer'.
- @return : size read from 'rBuffer',
- or an errorCode, which can be tested using FSE_isError().
- maxSymbolValuePtr[0] and tableLogPtr[0] will also be updated with their respective values */
-FSE_PUBLIC_API size_t FSE_readNCount (short* normalizedCounter,
- unsigned* maxSymbolValuePtr, unsigned* tableLogPtr,
- const void* rBuffer, size_t rBuffSize);
-
-/*! Constructor and Destructor of FSE_DTable.
- Note that its size depends on 'tableLog' */
-typedef unsigned FSE_DTable; /* don't allocate that. It's just a way to be more restrictive than void* */
-FSE_PUBLIC_API FSE_DTable* FSE_createDTable(unsigned tableLog);
-FSE_PUBLIC_API void FSE_freeDTable(FSE_DTable* dt);
-
-/*! FSE_buildDTable():
- Builds 'dt', which must be already allocated, using FSE_createDTable().
- return : 0, or an errorCode, which can be tested using FSE_isError() */
-FSE_PUBLIC_API size_t FSE_buildDTable (FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
-
-/*! FSE_decompress_usingDTable():
- Decompress compressed source `cSrc` of size `cSrcSize` using `dt`
- into `dst` which must be already allocated.
- @return : size of regenerated data (necessarily <= `dstCapacity`),
- or an errorCode, which can be tested using FSE_isError() */
-FSE_PUBLIC_API size_t FSE_decompress_usingDTable(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, const FSE_DTable* dt);
-
-/*!
-Tutorial :
-----------
-(Note : these functions only decompress FSE-compressed blocks.
- If block is uncompressed, use memcpy() instead
- If block is a single repeated byte, use memset() instead )
-
-The first step is to obtain the normalized frequencies of symbols.
-This can be performed by FSE_readNCount() if it was saved using FSE_writeNCount().
-'normalizedCounter' must be already allocated, and have at least 'maxSymbolValuePtr[0]+1' cells of signed short.
-In practice, that means it's necessary to know 'maxSymbolValue' beforehand,
-or size the table to handle worst case situations (typically 256).
-FSE_readNCount() will provide 'tableLog' and 'maxSymbolValue'.
-The result of FSE_readNCount() is the number of bytes read from 'rBuffer'.
-Note that 'rBufferSize' must be at least 4 bytes, even if useful information is less than that.
-If there is an error, the function will return an error code, which can be tested using FSE_isError().
-
-The next step is to build the decompression tables 'FSE_DTable' from 'normalizedCounter'.
-This is performed by the function FSE_buildDTable().
-The space required by 'FSE_DTable' must be already allocated using FSE_createDTable().
-If there is an error, the function will return an error code, which can be tested using FSE_isError().
-
-`FSE_DTable` can then be used to decompress `cSrc`, with FSE_decompress_usingDTable().
-`cSrcSize` must be strictly correct, otherwise decompression will fail.
-FSE_decompress_usingDTable() result will tell how many bytes were regenerated (<=`dstCapacity`).
-If there is an error, the function will return an error code, which can be tested using FSE_isError(). (ex: dst buffer too small)
-*/
-
-#endif /* FSE_H */
-
-#if defined(FSE_STATIC_LINKING_ONLY) && !defined(FSE_H_FSE_STATIC_LINKING_ONLY)
-#define FSE_H_FSE_STATIC_LINKING_ONLY
-
-/* *** Dependency *** */
-#include "bitstream.h"
-
-
-/* *****************************************
-* Static allocation
-*******************************************/
-/* FSE buffer bounds */
-#define FSE_NCOUNTBOUND 512
-#define FSE_BLOCKBOUND(size) (size + (size>>7))
-#define FSE_COMPRESSBOUND(size) (FSE_NCOUNTBOUND + FSE_BLOCKBOUND(size)) /* Macro version, useful for static allocation */
-
-/* It is possible to statically allocate FSE CTable/DTable as a table of FSE_CTable/FSE_DTable using below macros */
-#define FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) (1 + (1<<(maxTableLog-1)) + ((maxSymbolValue+1)*2))
-#define FSE_DTABLE_SIZE_U32(maxTableLog) (1 + (1<<maxTableLog))
-
-/* or use the size to malloc() space directly. Pay attention to alignment restrictions though */
-#define FSE_CTABLE_SIZE(maxTableLog, maxSymbolValue) (FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) * sizeof(FSE_CTable))
-#define FSE_DTABLE_SIZE(maxTableLog) (FSE_DTABLE_SIZE_U32(maxTableLog) * sizeof(FSE_DTable))
-
-
-/* *****************************************
- * FSE advanced API
- ***************************************** */
-
-unsigned FSE_optimalTableLog_internal(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue, unsigned minus);
-/**< same as FSE_optimalTableLog(), which used `minus==2` */
-
-/* FSE_compress_wksp() :
- * Same as FSE_compress2(), but using an externally allocated scratch buffer (`workSpace`).
- * FSE_WKSP_SIZE_U32() provides the minimum size required for `workSpace` as a table of FSE_CTable.
- */
-#define FSE_WKSP_SIZE_U32(maxTableLog, maxSymbolValue) ( FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) + ((maxTableLog > 12) ? (1 << (maxTableLog - 2)) : 1024) )
-size_t FSE_compress_wksp (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize);
-
-size_t FSE_buildCTable_raw (FSE_CTable* ct, unsigned nbBits);
-/**< build a fake FSE_CTable, designed for a flat distribution, where each symbol uses nbBits */
-
-size_t FSE_buildCTable_rle (FSE_CTable* ct, unsigned char symbolValue);
-/**< build a fake FSE_CTable, designed to compress always the same symbolValue */
-
-/* FSE_buildCTable_wksp() :
- * Same as FSE_buildCTable(), but using an externally allocated scratch buffer (`workSpace`).
- * `wkspSize` must be >= `(1<<tableLog)`.
- */
-size_t FSE_buildCTable_wksp(FSE_CTable* ct, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize);
-
-size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits);
-/**< build a fake FSE_DTable, designed to read a flat distribution where each symbol uses nbBits */
-
-size_t FSE_buildDTable_rle (FSE_DTable* dt, unsigned char symbolValue);
-/**< build a fake FSE_DTable, designed to always generate the same symbolValue */
-
-size_t FSE_decompress_wksp(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, FSE_DTable* workSpace, unsigned maxLog);
-/**< same as FSE_decompress(), using an externally allocated `workSpace` produced with `FSE_DTABLE_SIZE_U32(maxLog)` */
-
-typedef enum {
- FSE_repeat_none, /**< Cannot use the previous table */
- FSE_repeat_check, /**< Can use the previous table but it must be checked */
- FSE_repeat_valid /**< Can use the previous table and it is assumed to be valid */
- } FSE_repeat;
-
-/* *****************************************
-* FSE symbol compression API
-*******************************************/
-/*!
- This API consists of small unitary functions, which highly benefit from being inlined.
- Hence their body are included in next section.
-*/
-typedef struct {
- ptrdiff_t value;
- const void* stateTable;
- const void* symbolTT;
- unsigned stateLog;
-} FSE_CState_t;
-
-static void FSE_initCState(FSE_CState_t* CStatePtr, const FSE_CTable* ct);
-
-static void FSE_encodeSymbol(BIT_CStream_t* bitC, FSE_CState_t* CStatePtr, unsigned symbol);
-
-static void FSE_flushCState(BIT_CStream_t* bitC, const FSE_CState_t* CStatePtr);
-
-/**<
-These functions are inner components of FSE_compress_usingCTable().
-They allow the creation of custom streams, mixing multiple tables and bit sources.
-
-A key property to keep in mind is that encoding and decoding are done **in reverse direction**.
-So the first symbol you will encode is the last you will decode, like a LIFO stack.
-
-You will need a few variables to track your CStream. They are :
-
-FSE_CTable ct; // Provided by FSE_buildCTable()
-BIT_CStream_t bitStream; // bitStream tracking structure
-FSE_CState_t state; // State tracking structure (can have several)
-
-
-The first thing to do is to init bitStream and state.
- size_t errorCode = BIT_initCStream(&bitStream, dstBuffer, maxDstSize);
- FSE_initCState(&state, ct);
-
-Note that BIT_initCStream() can produce an error code, so its result should be tested, using FSE_isError();
-You can then encode your input data, byte after byte.
-FSE_encodeSymbol() outputs a maximum of 'tableLog' bits at a time.
-Remember decoding will be done in reverse direction.
- FSE_encodeByte(&bitStream, &state, symbol);
-
-At any time, you can also add any bit sequence.
-Note : maximum allowed nbBits is 25, for compatibility with 32-bits decoders
- BIT_addBits(&bitStream, bitField, nbBits);
-
-The above methods don't commit data to memory, they just store it into local register, for speed.
-Local register size is 64-bits on 64-bits systems, 32-bits on 32-bits systems (size_t).
-Writing data to memory is a manual operation, performed by the flushBits function.
- BIT_flushBits(&bitStream);
-
-Your last FSE encoding operation shall be to flush your last state value(s).
- FSE_flushState(&bitStream, &state);
-
-Finally, you must close the bitStream.
-The function returns the size of CStream in bytes.
-If data couldn't fit into dstBuffer, it will return a 0 ( == not compressible)
-If there is an error, it returns an errorCode (which can be tested using FSE_isError()).
- size_t size = BIT_closeCStream(&bitStream);
-*/
-
-
-/* *****************************************
-* FSE symbol decompression API
-*******************************************/
-typedef struct {
- size_t state;
- const void* table; /* precise table may vary, depending on U16 */
-} FSE_DState_t;
-
-
-static void FSE_initDState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD, const FSE_DTable* dt);
-
-static unsigned char FSE_decodeSymbol(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD);
-
-static unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr);
-
-/**<
-Let's now decompose FSE_decompress_usingDTable() into its unitary components.
-You will decode FSE-encoded symbols from the bitStream,
-and also any other bitFields you put in, **in reverse order**.
-
-You will need a few variables to track your bitStream. They are :
-
-BIT_DStream_t DStream; // Stream context
-FSE_DState_t DState; // State context. Multiple ones are possible
-FSE_DTable* DTablePtr; // Decoding table, provided by FSE_buildDTable()
-
-The first thing to do is to init the bitStream.
- errorCode = BIT_initDStream(&DStream, srcBuffer, srcSize);
-
-You should then retrieve your initial state(s)
-(in reverse flushing order if you have several ones) :
- errorCode = FSE_initDState(&DState, &DStream, DTablePtr);
-
-You can then decode your data, symbol after symbol.
-For information the maximum number of bits read by FSE_decodeSymbol() is 'tableLog'.
-Keep in mind that symbols are decoded in reverse order, like a LIFO stack (last in, first out).
- unsigned char symbol = FSE_decodeSymbol(&DState, &DStream);
-
-You can retrieve any bitfield you eventually stored into the bitStream (in reverse order)
-Note : maximum allowed nbBits is 25, for 32-bits compatibility
- size_t bitField = BIT_readBits(&DStream, nbBits);
-
-All above operations only read from local register (which size depends on size_t).
-Refueling the register from memory is manually performed by the reload method.
- endSignal = FSE_reloadDStream(&DStream);
-
-BIT_reloadDStream() result tells if there is still some more data to read from DStream.
-BIT_DStream_unfinished : there is still some data left into the DStream.
-BIT_DStream_endOfBuffer : Dstream reached end of buffer. Its container may no longer be completely filled.
-BIT_DStream_completed : Dstream reached its exact end, corresponding in general to decompression completed.
-BIT_DStream_tooFar : Dstream went too far. Decompression result is corrupted.
-
-When reaching end of buffer (BIT_DStream_endOfBuffer), progress slowly, notably if you decode multiple symbols per loop,
-to properly detect the exact end of stream.
-After each decoded symbol, check if DStream is fully consumed using this simple test :
- BIT_reloadDStream(&DStream) >= BIT_DStream_completed
-
-When it's done, verify decompression is fully completed, by checking both DStream and the relevant states.
-Checking if DStream has reached its end is performed by :
- BIT_endOfDStream(&DStream);
-Check also the states. There might be some symbols left there, if some high probability ones (>50%) are possible.
- FSE_endOfDState(&DState);
-*/
-
-
-/* *****************************************
-* FSE unsafe API
-*******************************************/
-static unsigned char FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD);
-/* faster, but works only if nbBits is always >= 1 (otherwise, result will be corrupted) */
-
-
-/* *****************************************
-* Implementation of inlined functions
-*******************************************/
-typedef struct {
- int deltaFindState;
- U32 deltaNbBits;
-} FSE_symbolCompressionTransform; /* total 8 bytes */
-
-MEM_STATIC void FSE_initCState(FSE_CState_t* statePtr, const FSE_CTable* ct)
-{
- const void* ptr = ct;
- const U16* u16ptr = (const U16*) ptr;
- const U32 tableLog = MEM_read16(ptr);
- statePtr->value = (ptrdiff_t)1<<tableLog;
- statePtr->stateTable = u16ptr+2;
- statePtr->symbolTT = ct + 1 + (tableLog ? (1<<(tableLog-1)) : 1);
- statePtr->stateLog = tableLog;
-}
-
-
-/*! FSE_initCState2() :
-* Same as FSE_initCState(), but the first symbol to include (which will be the last to be read)
-* uses the smallest state value possible, saving the cost of this symbol */
-MEM_STATIC void FSE_initCState2(FSE_CState_t* statePtr, const FSE_CTable* ct, U32 symbol)
-{
- FSE_initCState(statePtr, ct);
- { const FSE_symbolCompressionTransform symbolTT = ((const FSE_symbolCompressionTransform*)(statePtr->symbolTT))[symbol];
- const U16* stateTable = (const U16*)(statePtr->stateTable);
- U32 nbBitsOut = (U32)((symbolTT.deltaNbBits + (1<<15)) >> 16);
- statePtr->value = (nbBitsOut << 16) - symbolTT.deltaNbBits;
- statePtr->value = stateTable[(statePtr->value >> nbBitsOut) + symbolTT.deltaFindState];
- }
-}
-
-MEM_STATIC void FSE_encodeSymbol(BIT_CStream_t* bitC, FSE_CState_t* statePtr, unsigned symbol)
-{
- FSE_symbolCompressionTransform const symbolTT = ((const FSE_symbolCompressionTransform*)(statePtr->symbolTT))[symbol];
- const U16* const stateTable = (const U16*)(statePtr->stateTable);
- U32 const nbBitsOut = (U32)((statePtr->value + symbolTT.deltaNbBits) >> 16);
- BIT_addBits(bitC, statePtr->value, nbBitsOut);
- statePtr->value = stateTable[ (statePtr->value >> nbBitsOut) + symbolTT.deltaFindState];
-}
-
-MEM_STATIC void FSE_flushCState(BIT_CStream_t* bitC, const FSE_CState_t* statePtr)
-{
- BIT_addBits(bitC, statePtr->value, statePtr->stateLog);
- BIT_flushBits(bitC);
-}
-
-
-/* FSE_getMaxNbBits() :
- * Approximate maximum cost of a symbol, in bits.
- * Fractional get rounded up (i.e : a symbol with a normalized frequency of 3 gives the same result as a frequency of 2)
- * note 1 : assume symbolValue is valid (<= maxSymbolValue)
- * note 2 : if freq[symbolValue]==0, @return a fake cost of tableLog+1 bits */
-MEM_STATIC U32 FSE_getMaxNbBits(const void* symbolTTPtr, U32 symbolValue)
-{
- const FSE_symbolCompressionTransform* symbolTT = (const FSE_symbolCompressionTransform*) symbolTTPtr;
- return (symbolTT[symbolValue].deltaNbBits + ((1<<16)-1)) >> 16;
-}
-
-/* FSE_bitCost() :
- * Approximate symbol cost, as fractional value, using fixed-point format (accuracyLog fractional bits)
- * note 1 : assume symbolValue is valid (<= maxSymbolValue)
- * note 2 : if freq[symbolValue]==0, @return a fake cost of tableLog+1 bits */
-MEM_STATIC U32 FSE_bitCost(const void* symbolTTPtr, U32 tableLog, U32 symbolValue, U32 accuracyLog)
-{
- const FSE_symbolCompressionTransform* symbolTT = (const FSE_symbolCompressionTransform*) symbolTTPtr;
- U32 const minNbBits = symbolTT[symbolValue].deltaNbBits >> 16;
- U32 const threshold = (minNbBits+1) << 16;
- assert(tableLog < 16);
- assert(accuracyLog < 31-tableLog); /* ensure enough room for renormalization double shift */
- { U32 const tableSize = 1 << tableLog;
- U32 const deltaFromThreshold = threshold - (symbolTT[symbolValue].deltaNbBits + tableSize);
- U32 const normalizedDeltaFromThreshold = (deltaFromThreshold << accuracyLog) >> tableLog; /* linear interpolation (very approximate) */
- U32 const bitMultiplier = 1 << accuracyLog;
- assert(symbolTT[symbolValue].deltaNbBits + tableSize <= threshold);
- assert(normalizedDeltaFromThreshold <= bitMultiplier);
- return (minNbBits+1)*bitMultiplier - normalizedDeltaFromThreshold;
- }
-}
-
-
-/* ====== Decompression ====== */
-
-typedef struct {
- U16 tableLog;
- U16 fastMode;
-} FSE_DTableHeader; /* sizeof U32 */
-
-typedef struct
-{
- unsigned short newState;
- unsigned char symbol;
- unsigned char nbBits;
-} FSE_decode_t; /* size == U32 */
-
-MEM_STATIC void FSE_initDState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD, const FSE_DTable* dt)
-{
- const void* ptr = dt;
- const FSE_DTableHeader* const DTableH = (const FSE_DTableHeader*)ptr;
- DStatePtr->state = BIT_readBits(bitD, DTableH->tableLog);
- BIT_reloadDStream(bitD);
- DStatePtr->table = dt + 1;
-}
-
-MEM_STATIC BYTE FSE_peekSymbol(const FSE_DState_t* DStatePtr)
-{
- FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
- return DInfo.symbol;
-}
-
-MEM_STATIC void FSE_updateState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD)
-{
- FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
- U32 const nbBits = DInfo.nbBits;
- size_t const lowBits = BIT_readBits(bitD, nbBits);
- DStatePtr->state = DInfo.newState + lowBits;
-}
-
-MEM_STATIC BYTE FSE_decodeSymbol(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD)
-{
- FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
- U32 const nbBits = DInfo.nbBits;
- BYTE const symbol = DInfo.symbol;
- size_t const lowBits = BIT_readBits(bitD, nbBits);
-
- DStatePtr->state = DInfo.newState + lowBits;
- return symbol;
-}
-
-/*! FSE_decodeSymbolFast() :
- unsafe, only works if no symbol has a probability > 50% */
-MEM_STATIC BYTE FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD)
-{
- FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
- U32 const nbBits = DInfo.nbBits;
- BYTE const symbol = DInfo.symbol;
- size_t const lowBits = BIT_readBitsFast(bitD, nbBits);
-
- DStatePtr->state = DInfo.newState + lowBits;
- return symbol;
-}
-
-MEM_STATIC unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr)
-{
- return DStatePtr->state == 0;
-}
-
-
-
-#ifndef FSE_COMMONDEFS_ONLY
-
-/* **************************************************************
-* Tuning parameters
-****************************************************************/
-/*!MEMORY_USAGE :
-* Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
-* Increasing memory usage improves compression ratio
-* Reduced memory usage can improve speed, due to cache effect
-* Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */
-#ifndef FSE_MAX_MEMORY_USAGE
-# define FSE_MAX_MEMORY_USAGE 14
-#endif
-#ifndef FSE_DEFAULT_MEMORY_USAGE
-# define FSE_DEFAULT_MEMORY_USAGE 13
-#endif
-
-/*!FSE_MAX_SYMBOL_VALUE :
-* Maximum symbol value authorized.
-* Required for proper stack allocation */
-#ifndef FSE_MAX_SYMBOL_VALUE
-# define FSE_MAX_SYMBOL_VALUE 255
-#endif
-
-/* **************************************************************
-* template functions type & suffix
-****************************************************************/
-#define FSE_FUNCTION_TYPE BYTE
-#define FSE_FUNCTION_EXTENSION
-#define FSE_DECODE_TYPE FSE_decode_t
-
-
-#endif /* !FSE_COMMONDEFS_ONLY */
-
-
-/* ***************************************************************
-* Constants
-*****************************************************************/
-#define FSE_MAX_TABLELOG (FSE_MAX_MEMORY_USAGE-2)
-#define FSE_MAX_TABLESIZE (1U<<FSE_MAX_TABLELOG)
-#define FSE_MAXTABLESIZE_MASK (FSE_MAX_TABLESIZE-1)
-#define FSE_DEFAULT_TABLELOG (FSE_DEFAULT_MEMORY_USAGE-2)
-#define FSE_MIN_TABLELOG 5
-
-#define FSE_TABLELOG_ABSOLUTE_MAX 15
-#if FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX
-# error "FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX is not supported"
-#endif
-
-#define FSE_TABLESTEP(tableSize) ((tableSize>>1) + (tableSize>>3) + 3)
-
-
-#endif /* FSE_STATIC_LINKING_ONLY */
-
-
-#if defined (__cplusplus)
-}
-#endif
diff --git a/vendor/github.com/DataDog/zstd/fse_compress.c b/vendor/github.com/DataDog/zstd/fse_compress.c
deleted file mode 100644
index 68b47e1..0000000
--- a/vendor/github.com/DataDog/zstd/fse_compress.c
+++ /dev/null
@@ -1,721 +0,0 @@
-/* ******************************************************************
- FSE : Finite State Entropy encoder
- Copyright (C) 2013-present, Yann Collet.
-
- BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
- * Redistributions of source code must retain the above copyright
- notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
- copyright notice, this list of conditions and the following disclaimer
- in the documentation and/or other materials provided with the
- distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
- "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
- LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
- A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
- OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
- OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
- You can contact the author at :
- - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
- - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/* **************************************************************
-* Includes
-****************************************************************/
-#include <stdlib.h> /* malloc, free, qsort */
-#include <string.h> /* memcpy, memset */
-#include "compiler.h"
-#include "mem.h" /* U32, U16, etc. */
-#include "debug.h" /* assert, DEBUGLOG */
-#include "hist.h" /* HIST_count_wksp */
-#include "bitstream.h"
-#define FSE_STATIC_LINKING_ONLY
-#include "fse.h"
-#include "error_private.h"
-
-
-/* **************************************************************
-* Error Management
-****************************************************************/
-#define FSE_isError ERR_isError
-
-
-/* **************************************************************
-* Templates
-****************************************************************/
-/*
- designed to be included
- for type-specific functions (template emulation in C)
- Objective is to write these functions only once, for improved maintenance
-*/
-
-/* safety checks */
-#ifndef FSE_FUNCTION_EXTENSION
-# error "FSE_FUNCTION_EXTENSION must be defined"
-#endif
-#ifndef FSE_FUNCTION_TYPE
-# error "FSE_FUNCTION_TYPE must be defined"
-#endif
-
-/* Function names */
-#define FSE_CAT(X,Y) X##Y
-#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y)
-#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y)
-
-
-/* Function templates */
-
-/* FSE_buildCTable_wksp() :
- * Same as FSE_buildCTable(), but using an externally allocated scratch buffer (`workSpace`).
- * wkspSize should be sized to handle worst case situation, which is `1<<max_tableLog * sizeof(FSE_FUNCTION_TYPE)`
- * workSpace must also be properly aligned with FSE_FUNCTION_TYPE requirements
- */
-size_t FSE_buildCTable_wksp(FSE_CTable* ct,
- const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog,
- void* workSpace, size_t wkspSize)
-{
- U32 const tableSize = 1 << tableLog;
- U32 const tableMask = tableSize - 1;
- void* const ptr = ct;
- U16* const tableU16 = ( (U16*) ptr) + 2;
- void* const FSCT = ((U32*)ptr) + 1 /* header */ + (tableLog ? tableSize>>1 : 1) ;
- FSE_symbolCompressionTransform* const symbolTT = (FSE_symbolCompressionTransform*) (FSCT);
- U32 const step = FSE_TABLESTEP(tableSize);
- U32 cumul[FSE_MAX_SYMBOL_VALUE+2];
-
- FSE_FUNCTION_TYPE* const tableSymbol = (FSE_FUNCTION_TYPE*)workSpace;
- U32 highThreshold = tableSize-1;
-
- /* CTable header */
- if (((size_t)1 << tableLog) * sizeof(FSE_FUNCTION_TYPE) > wkspSize) return ERROR(tableLog_tooLarge);
- tableU16[-2] = (U16) tableLog;
- tableU16[-1] = (U16) maxSymbolValue;
- assert(tableLog < 16); /* required for threshold strategy to work */
-
- /* For explanations on how to distribute symbol values over the table :
- * http://fastcompression.blogspot.fr/2014/02/fse-distributing-symbol-values.html */
-
- #ifdef __clang_analyzer__
- memset(tableSymbol, 0, sizeof(*tableSymbol) * tableSize); /* useless initialization, just to keep scan-build happy */
- #endif
-
- /* symbol start positions */
- { U32 u;
- cumul[0] = 0;
- for (u=1; u <= maxSymbolValue+1; u++) {
- if (normalizedCounter[u-1]==-1) { /* Low proba symbol */
- cumul[u] = cumul[u-1] + 1;
- tableSymbol[highThreshold--] = (FSE_FUNCTION_TYPE)(u-1);
- } else {
- cumul[u] = cumul[u-1] + normalizedCounter[u-1];
- } }
- cumul[maxSymbolValue+1] = tableSize+1;
- }
-
- /* Spread symbols */
- { U32 position = 0;
- U32 symbol;
- for (symbol=0; symbol<=maxSymbolValue; symbol++) {
- int nbOccurrences;
- int const freq = normalizedCounter[symbol];
- for (nbOccurrences=0; nbOccurrences<freq; nbOccurrences++) {
- tableSymbol[position] = (FSE_FUNCTION_TYPE)symbol;
- position = (position + step) & tableMask;
- while (position > highThreshold)
- position = (position + step) & tableMask; /* Low proba area */
- } }
-
- assert(position==0); /* Must have initialized all positions */
- }
-
- /* Build table */
- { U32 u; for (u=0; u<tableSize; u++) {
- FSE_FUNCTION_TYPE s = tableSymbol[u]; /* note : static analyzer may not understand tableSymbol is properly initialized */
- tableU16[cumul[s]++] = (U16) (tableSize+u); /* TableU16 : sorted by symbol order; gives next state value */
- } }
-
- /* Build Symbol Transformation Table */
- { unsigned total = 0;
- unsigned s;
- for (s=0; s<=maxSymbolValue; s++) {
- switch (normalizedCounter[s])
- {
- case 0:
- /* filling nonetheless, for compatibility with FSE_getMaxNbBits() */
- symbolTT[s].deltaNbBits = ((tableLog+1) << 16) - (1<<tableLog);
- break;
-
- case -1:
- case 1:
- symbolTT[s].deltaNbBits = (tableLog << 16) - (1<<tableLog);
- symbolTT[s].deltaFindState = total - 1;
- total ++;
- break;
- default :
- {
- U32 const maxBitsOut = tableLog - BIT_highbit32 (normalizedCounter[s]-1);
- U32 const minStatePlus = normalizedCounter[s] << maxBitsOut;
- symbolTT[s].deltaNbBits = (maxBitsOut << 16) - minStatePlus;
- symbolTT[s].deltaFindState = total - normalizedCounter[s];
- total += normalizedCounter[s];
- } } } }
-
-#if 0 /* debug : symbol costs */
- DEBUGLOG(5, "\n --- table statistics : ");
- { U32 symbol;
- for (symbol=0; symbol<=maxSymbolValue; symbol++) {
- DEBUGLOG(5, "%3u: w=%3i, maxBits=%u, fracBits=%.2f",
- symbol, normalizedCounter[symbol],
- FSE_getMaxNbBits(symbolTT, symbol),
- (double)FSE_bitCost(symbolTT, tableLog, symbol, 8) / 256);
- }
- }
-#endif
-
- return 0;
-}
-
-
-size_t FSE_buildCTable(FSE_CTable* ct, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
- FSE_FUNCTION_TYPE tableSymbol[FSE_MAX_TABLESIZE]; /* memset() is not necessary, even if static analyzer complain about it */
- return FSE_buildCTable_wksp(ct, normalizedCounter, maxSymbolValue, tableLog, tableSymbol, sizeof(tableSymbol));
-}
-
-
-
-#ifndef FSE_COMMONDEFS_ONLY
-
-
-/*-**************************************************************
-* FSE NCount encoding
-****************************************************************/
-size_t FSE_NCountWriteBound(unsigned maxSymbolValue, unsigned tableLog)
-{
- size_t const maxHeaderSize = (((maxSymbolValue+1) * tableLog) >> 3) + 3;
- return maxSymbolValue ? maxHeaderSize : FSE_NCOUNTBOUND; /* maxSymbolValue==0 ? use default */
-}
-
-static size_t
-FSE_writeNCount_generic (void* header, size_t headerBufferSize,
- const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog,
- unsigned writeIsSafe)
-{
- BYTE* const ostart = (BYTE*) header;
- BYTE* out = ostart;
- BYTE* const oend = ostart + headerBufferSize;
- int nbBits;
- const int tableSize = 1 << tableLog;
- int remaining;
- int threshold;
- U32 bitStream = 0;
- int bitCount = 0;
- unsigned symbol = 0;
- unsigned const alphabetSize = maxSymbolValue + 1;
- int previousIs0 = 0;
-
- /* Table Size */
- bitStream += (tableLog-FSE_MIN_TABLELOG) << bitCount;
- bitCount += 4;
-
- /* Init */
- remaining = tableSize+1; /* +1 for extra accuracy */
- threshold = tableSize;
- nbBits = tableLog+1;
-
- while ((symbol < alphabetSize) && (remaining>1)) { /* stops at 1 */
- if (previousIs0) {
- unsigned start = symbol;
- while ((symbol < alphabetSize) && !normalizedCounter[symbol]) symbol++;
- if (symbol == alphabetSize) break; /* incorrect distribution */
- while (symbol >= start+24) {
- start+=24;
- bitStream += 0xFFFFU << bitCount;
- if ((!writeIsSafe) && (out > oend-2))
- return ERROR(dstSize_tooSmall); /* Buffer overflow */
- out[0] = (BYTE) bitStream;
- out[1] = (BYTE)(bitStream>>8);
- out+=2;
- bitStream>>=16;
- }
- while (symbol >= start+3) {
- start+=3;
- bitStream += 3 << bitCount;
- bitCount += 2;
- }
- bitStream += (symbol-start) << bitCount;
- bitCount += 2;
- if (bitCount>16) {
- if ((!writeIsSafe) && (out > oend - 2))
- return ERROR(dstSize_tooSmall); /* Buffer overflow */
- out[0] = (BYTE)bitStream;
- out[1] = (BYTE)(bitStream>>8);
- out += 2;
- bitStream >>= 16;
- bitCount -= 16;
- } }
- { int count = normalizedCounter[symbol++];
- int const max = (2*threshold-1) - remaining;
- remaining -= count < 0 ? -count : count;
- count++; /* +1 for extra accuracy */
- if (count>=threshold)
- count += max; /* [0..max[ [max..threshold[ (...) [threshold+max 2*threshold[ */
- bitStream += count << bitCount;
- bitCount += nbBits;
- bitCount -= (count<max);
- previousIs0 = (count==1);
- if (remaining<1) return ERROR(GENERIC);
- while (remaining<threshold) { nbBits--; threshold>>=1; }
- }
- if (bitCount>16) {
- if ((!writeIsSafe) && (out > oend - 2))
- return ERROR(dstSize_tooSmall); /* Buffer overflow */
- out[0] = (BYTE)bitStream;
- out[1] = (BYTE)(bitStream>>8);
- out += 2;
- bitStream >>= 16;
- bitCount -= 16;
- } }
-
- if (remaining != 1)
- return ERROR(GENERIC); /* incorrect normalized distribution */
- assert(symbol <= alphabetSize);
-
- /* flush remaining bitStream */
- if ((!writeIsSafe) && (out > oend - 2))
- return ERROR(dstSize_tooSmall); /* Buffer overflow */
- out[0] = (BYTE)bitStream;
- out[1] = (BYTE)(bitStream>>8);
- out+= (bitCount+7) /8;
-
- return (out-ostart);
-}
-
-
-size_t FSE_writeNCount (void* buffer, size_t bufferSize,
- const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
- if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge); /* Unsupported */
- if (tableLog < FSE_MIN_TABLELOG) return ERROR(GENERIC); /* Unsupported */
-
- if (bufferSize < FSE_NCountWriteBound(maxSymbolValue, tableLog))
- return FSE_writeNCount_generic(buffer, bufferSize, normalizedCounter, maxSymbolValue, tableLog, 0);
-
- return FSE_writeNCount_generic(buffer, bufferSize, normalizedCounter, maxSymbolValue, tableLog, 1 /* write in buffer is safe */);
-}
-
-
-/*-**************************************************************
-* FSE Compression Code
-****************************************************************/
-
-FSE_CTable* FSE_createCTable (unsigned maxSymbolValue, unsigned tableLog)
-{
- size_t size;
- if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX;
- size = FSE_CTABLE_SIZE_U32 (tableLog, maxSymbolValue) * sizeof(U32);
- return (FSE_CTable*)malloc(size);
-}
-
-void FSE_freeCTable (FSE_CTable* ct) { free(ct); }
-
-/* provides the minimum logSize to safely represent a distribution */
-static unsigned FSE_minTableLog(size_t srcSize, unsigned maxSymbolValue)
-{
- U32 minBitsSrc = BIT_highbit32((U32)(srcSize)) + 1;
- U32 minBitsSymbols = BIT_highbit32(maxSymbolValue) + 2;
- U32 minBits = minBitsSrc < minBitsSymbols ? minBitsSrc : minBitsSymbols;
- assert(srcSize > 1); /* Not supported, RLE should be used instead */
- return minBits;
-}
-
-unsigned FSE_optimalTableLog_internal(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue, unsigned minus)
-{
- U32 maxBitsSrc = BIT_highbit32((U32)(srcSize - 1)) - minus;
- U32 tableLog = maxTableLog;
- U32 minBits = FSE_minTableLog(srcSize, maxSymbolValue);
- assert(srcSize > 1); /* Not supported, RLE should be used instead */
- if (tableLog==0) tableLog = FSE_DEFAULT_TABLELOG;
- if (maxBitsSrc < tableLog) tableLog = maxBitsSrc; /* Accuracy can be reduced */
- if (minBits > tableLog) tableLog = minBits; /* Need a minimum to safely represent all symbol values */
- if (tableLog < FSE_MIN_TABLELOG) tableLog = FSE_MIN_TABLELOG;
- if (tableLog > FSE_MAX_TABLELOG) tableLog = FSE_MAX_TABLELOG;
- return tableLog;
-}
-
-unsigned FSE_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue)
-{
- return FSE_optimalTableLog_internal(maxTableLog, srcSize, maxSymbolValue, 2);
-}
-
-
-/* Secondary normalization method.
- To be used when primary method fails. */
-
-static size_t FSE_normalizeM2(short* norm, U32 tableLog, const unsigned* count, size_t total, U32 maxSymbolValue)
-{
- short const NOT_YET_ASSIGNED = -2;
- U32 s;
- U32 distributed = 0;
- U32 ToDistribute;
-
- /* Init */
- U32 const lowThreshold = (U32)(total >> tableLog);
- U32 lowOne = (U32)((total * 3) >> (tableLog + 1));
-
- for (s=0; s<=maxSymbolValue; s++) {
- if (count[s] == 0) {
- norm[s]=0;
- continue;
- }
- if (count[s] <= lowThreshold) {
- norm[s] = -1;
- distributed++;
- total -= count[s];
- continue;
- }
- if (count[s] <= lowOne) {
- norm[s] = 1;
- distributed++;
- total -= count[s];
- continue;
- }
-
- norm[s]=NOT_YET_ASSIGNED;
- }
- ToDistribute = (1 << tableLog) - distributed;
-
- if (ToDistribute == 0)
- return 0;
-
- if ((total / ToDistribute) > lowOne) {
- /* risk of rounding to zero */
- lowOne = (U32)((total * 3) / (ToDistribute * 2));
- for (s=0; s<=maxSymbolValue; s++) {
- if ((norm[s] == NOT_YET_ASSIGNED) && (count[s] <= lowOne)) {
- norm[s] = 1;
- distributed++;
- total -= count[s];
- continue;
- } }
- ToDistribute = (1 << tableLog) - distributed;
- }
-
- if (distributed == maxSymbolValue+1) {
- /* all values are pretty poor;
- probably incompressible data (should have already been detected);
- find max, then give all remaining points to max */
- U32 maxV = 0, maxC = 0;
- for (s=0; s<=maxSymbolValue; s++)
- if (count[s] > maxC) { maxV=s; maxC=count[s]; }
- norm[maxV] += (short)ToDistribute;
- return 0;
- }
-
- if (total == 0) {
- /* all of the symbols were low enough for the lowOne or lowThreshold */
- for (s=0; ToDistribute > 0; s = (s+1)%(maxSymbolValue+1))
- if (norm[s] > 0) { ToDistribute--; norm[s]++; }
- return 0;
- }
-
- { U64 const vStepLog = 62 - tableLog;
- U64 const mid = (1ULL << (vStepLog-1)) - 1;
- U64 const rStep = ((((U64)1<<vStepLog) * ToDistribute) + mid) / total; /* scale on remaining */
- U64 tmpTotal = mid;
- for (s=0; s<=maxSymbolValue; s++) {
- if (norm[s]==NOT_YET_ASSIGNED) {
- U64 const end = tmpTotal + (count[s] * rStep);
- U32 const sStart = (U32)(tmpTotal >> vStepLog);
- U32 const sEnd = (U32)(end >> vStepLog);
- U32 const weight = sEnd - sStart;
- if (weight < 1)
- return ERROR(GENERIC);
- norm[s] = (short)weight;
- tmpTotal = end;
- } } }
-
- return 0;
-}
-
-
-size_t FSE_normalizeCount (short* normalizedCounter, unsigned tableLog,
- const unsigned* count, size_t total,
- unsigned maxSymbolValue)
-{
- /* Sanity checks */
- if (tableLog==0) tableLog = FSE_DEFAULT_TABLELOG;
- if (tableLog < FSE_MIN_TABLELOG) return ERROR(GENERIC); /* Unsupported size */
- if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge); /* Unsupported size */
- if (tableLog < FSE_minTableLog(total, maxSymbolValue)) return ERROR(GENERIC); /* Too small tableLog, compression potentially impossible */
-
- { static U32 const rtbTable[] = { 0, 473195, 504333, 520860, 550000, 700000, 750000, 830000 };
- U64 const scale = 62 - tableLog;
- U64 const step = ((U64)1<<62) / total; /* <== here, one division ! */
- U64 const vStep = 1ULL<<(scale-20);
- int stillToDistribute = 1<<tableLog;
- unsigned s;
- unsigned largest=0;
- short largestP=0;
- U32 lowThreshold = (U32)(total >> tableLog);
-
- for (s=0; s<=maxSymbolValue; s++) {
- if (count[s] == total) return 0; /* rle special case */
- if (count[s] == 0) { normalizedCounter[s]=0; continue; }
- if (count[s] <= lowThreshold) {
- normalizedCounter[s] = -1;
- stillToDistribute--;
- } else {
- short proba = (short)((count[s]*step) >> scale);
- if (proba<8) {
- U64 restToBeat = vStep * rtbTable[proba];
- proba += (count[s]*step) - ((U64)proba<<scale) > restToBeat;
- }
- if (proba > largestP) { largestP=proba; largest=s; }
- normalizedCounter[s] = proba;
- stillToDistribute -= proba;
- } }
- if (-stillToDistribute >= (normalizedCounter[largest] >> 1)) {
- /* corner case, need another normalization method */
- size_t const errorCode = FSE_normalizeM2(normalizedCounter, tableLog, count, total, maxSymbolValue);
- if (FSE_isError(errorCode)) return errorCode;
- }
- else normalizedCounter[largest] += (short)stillToDistribute;
- }
-
-#if 0
- { /* Print Table (debug) */
- U32 s;
- U32 nTotal = 0;
- for (s=0; s<=maxSymbolValue; s++)
- RAWLOG(2, "%3i: %4i \n", s, normalizedCounter[s]);
- for (s=0; s<=maxSymbolValue; s++)
- nTotal += abs(normalizedCounter[s]);
- if (nTotal != (1U<<tableLog))
- RAWLOG(2, "Warning !!! Total == %u != %u !!!", nTotal, 1U<<tableLog);
- getchar();
- }
-#endif
-
- return tableLog;
-}
-
-
-/* fake FSE_CTable, for raw (uncompressed) input */
-size_t FSE_buildCTable_raw (FSE_CTable* ct, unsigned nbBits)
-{
- const unsigned tableSize = 1 << nbBits;
- const unsigned tableMask = tableSize - 1;
- const unsigned maxSymbolValue = tableMask;
- void* const ptr = ct;
- U16* const tableU16 = ( (U16*) ptr) + 2;
- void* const FSCT = ((U32*)ptr) + 1 /* header */ + (tableSize>>1); /* assumption : tableLog >= 1 */
- FSE_symbolCompressionTransform* const symbolTT = (FSE_symbolCompressionTransform*) (FSCT);
- unsigned s;
-
- /* Sanity checks */
- if (nbBits < 1) return ERROR(GENERIC); /* min size */
-
- /* header */
- tableU16[-2] = (U16) nbBits;
- tableU16[-1] = (U16) maxSymbolValue;
-
- /* Build table */
- for (s=0; s<tableSize; s++)
- tableU16[s] = (U16)(tableSize + s);
-
- /* Build Symbol Transformation Table */
- { const U32 deltaNbBits = (nbBits << 16) - (1 << nbBits);
- for (s=0; s<=maxSymbolValue; s++) {
- symbolTT[s].deltaNbBits = deltaNbBits;
- symbolTT[s].deltaFindState = s-1;
- } }
-
- return 0;
-}
-
-/* fake FSE_CTable, for rle input (always same symbol) */
-size_t FSE_buildCTable_rle (FSE_CTable* ct, BYTE symbolValue)
-{
- void* ptr = ct;
- U16* tableU16 = ( (U16*) ptr) + 2;
- void* FSCTptr = (U32*)ptr + 2;
- FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) FSCTptr;
-
- /* header */
- tableU16[-2] = (U16) 0;
- tableU16[-1] = (U16) symbolValue;
-
- /* Build table */
- tableU16[0] = 0;
- tableU16[1] = 0; /* just in case */
-
- /* Build Symbol Transformation Table */
- symbolTT[symbolValue].deltaNbBits = 0;
- symbolTT[symbolValue].deltaFindState = 0;
-
- return 0;
-}
-
-
-static size_t FSE_compress_usingCTable_generic (void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- const FSE_CTable* ct, const unsigned fast)
-{
- const BYTE* const istart = (const BYTE*) src;
- const BYTE* const iend = istart + srcSize;
- const BYTE* ip=iend;
-
- BIT_CStream_t bitC;
- FSE_CState_t CState1, CState2;
-
- /* init */
- if (srcSize <= 2) return 0;
- { size_t const initError = BIT_initCStream(&bitC, dst, dstSize);
- if (FSE_isError(initError)) return 0; /* not enough space available to write a bitstream */ }
-
-#define FSE_FLUSHBITS(s) (fast ? BIT_flushBitsFast(s) : BIT_flushBits(s))
-
- if (srcSize & 1) {
- FSE_initCState2(&CState1, ct, *--ip);
- FSE_initCState2(&CState2, ct, *--ip);
- FSE_encodeSymbol(&bitC, &CState1, *--ip);
- FSE_FLUSHBITS(&bitC);
- } else {
- FSE_initCState2(&CState2, ct, *--ip);
- FSE_initCState2(&CState1, ct, *--ip);
- }
-
- /* join to mod 4 */
- srcSize -= 2;
- if ((sizeof(bitC.bitContainer)*8 > FSE_MAX_TABLELOG*4+7 ) && (srcSize & 2)) { /* test bit 2 */
- FSE_encodeSymbol(&bitC, &CState2, *--ip);
- FSE_encodeSymbol(&bitC, &CState1, *--ip);
- FSE_FLUSHBITS(&bitC);
- }
-
- /* 2 or 4 encoding per loop */
- while ( ip>istart ) {
-
- FSE_encodeSymbol(&bitC, &CState2, *--ip);
-
- if (sizeof(bitC.bitContainer)*8 < FSE_MAX_TABLELOG*2+7 ) /* this test must be static */
- FSE_FLUSHBITS(&bitC);
-
- FSE_encodeSymbol(&bitC, &CState1, *--ip);
-
- if (sizeof(bitC.bitContainer)*8 > FSE_MAX_TABLELOG*4+7 ) { /* this test must be static */
- FSE_encodeSymbol(&bitC, &CState2, *--ip);
- FSE_encodeSymbol(&bitC, &CState1, *--ip);
- }
-
- FSE_FLUSHBITS(&bitC);
- }
-
- FSE_flushCState(&bitC, &CState2);
- FSE_flushCState(&bitC, &CState1);
- return BIT_closeCStream(&bitC);
-}
-
-size_t FSE_compress_usingCTable (void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- const FSE_CTable* ct)
-{
- unsigned const fast = (dstSize >= FSE_BLOCKBOUND(srcSize));
-
- if (fast)
- return FSE_compress_usingCTable_generic(dst, dstSize, src, srcSize, ct, 1);
- else
- return FSE_compress_usingCTable_generic(dst, dstSize, src, srcSize, ct, 0);
-}
-
-
-size_t FSE_compressBound(size_t size) { return FSE_COMPRESSBOUND(size); }
-
-#define CHECK_V_F(e, f) size_t const e = f; if (ERR_isError(e)) return e
-#define CHECK_F(f) { CHECK_V_F(_var_err__, f); }
-
-/* FSE_compress_wksp() :
- * Same as FSE_compress2(), but using an externally allocated scratch buffer (`workSpace`).
- * `wkspSize` size must be `(1<<tableLog)`.
- */
-size_t FSE_compress_wksp (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize)
-{
- BYTE* const ostart = (BYTE*) dst;
- BYTE* op = ostart;
- BYTE* const oend = ostart + dstSize;
-
- unsigned count[FSE_MAX_SYMBOL_VALUE+1];
- S16 norm[FSE_MAX_SYMBOL_VALUE+1];
- FSE_CTable* CTable = (FSE_CTable*)workSpace;
- size_t const CTableSize = FSE_CTABLE_SIZE_U32(tableLog, maxSymbolValue);
- void* scratchBuffer = (void*)(CTable + CTableSize);
- size_t const scratchBufferSize = wkspSize - (CTableSize * sizeof(FSE_CTable));
-
- /* init conditions */
- if (wkspSize < FSE_WKSP_SIZE_U32(tableLog, maxSymbolValue)) return ERROR(tableLog_tooLarge);
- if (srcSize <= 1) return 0; /* Not compressible */
- if (!maxSymbolValue) maxSymbolValue = FSE_MAX_SYMBOL_VALUE;
- if (!tableLog) tableLog = FSE_DEFAULT_TABLELOG;
-
- /* Scan input and build symbol stats */
- { CHECK_V_F(maxCount, HIST_count_wksp(count, &maxSymbolValue, src, srcSize, scratchBuffer, scratchBufferSize) );
- if (maxCount == srcSize) return 1; /* only a single symbol in src : rle */
- if (maxCount == 1) return 0; /* each symbol present maximum once => not compressible */
- if (maxCount < (srcSize >> 7)) return 0; /* Heuristic : not compressible enough */
- }
-
- tableLog = FSE_optimalTableLog(tableLog, srcSize, maxSymbolValue);
- CHECK_F( FSE_normalizeCount(norm, tableLog, count, srcSize, maxSymbolValue) );
-
- /* Write table description header */
- { CHECK_V_F(nc_err, FSE_writeNCount(op, oend-op, norm, maxSymbolValue, tableLog) );
- op += nc_err;
- }
-
- /* Compress */
- CHECK_F( FSE_buildCTable_wksp(CTable, norm, maxSymbolValue, tableLog, scratchBuffer, scratchBufferSize) );
- { CHECK_V_F(cSize, FSE_compress_usingCTable(op, oend - op, src, srcSize, CTable) );
- if (cSize == 0) return 0; /* not enough space for compressed data */
- op += cSize;
- }
-
- /* check compressibility */
- if ( (size_t)(op-ostart) >= srcSize-1 ) return 0;
-
- return op-ostart;
-}
-
-typedef struct {
- FSE_CTable CTable_max[FSE_CTABLE_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE)];
- BYTE scratchBuffer[1 << FSE_MAX_TABLELOG];
-} fseWkspMax_t;
-
-size_t FSE_compress2 (void* dst, size_t dstCapacity, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog)
-{
- fseWkspMax_t scratchBuffer;
- DEBUG_STATIC_ASSERT(sizeof(scratchBuffer) >= FSE_WKSP_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE)); /* compilation failures here means scratchBuffer is not large enough */
- if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge);
- return FSE_compress_wksp(dst, dstCapacity, src, srcSize, maxSymbolValue, tableLog, &scratchBuffer, sizeof(scratchBuffer));
-}
-
-size_t FSE_compress (void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
- return FSE_compress2(dst, dstCapacity, src, srcSize, FSE_MAX_SYMBOL_VALUE, FSE_DEFAULT_TABLELOG);
-}
-
-
-#endif /* FSE_COMMONDEFS_ONLY */
diff --git a/vendor/github.com/DataDog/zstd/fse_decompress.c b/vendor/github.com/DataDog/zstd/fse_decompress.c
deleted file mode 100644
index 72bbead..0000000
--- a/vendor/github.com/DataDog/zstd/fse_decompress.c
+++ /dev/null
@@ -1,309 +0,0 @@
-/* ******************************************************************
- FSE : Finite State Entropy decoder
- Copyright (C) 2013-2015, Yann Collet.
-
- BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
- * Redistributions of source code must retain the above copyright
- notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
- copyright notice, this list of conditions and the following disclaimer
- in the documentation and/or other materials provided with the
- distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
- "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
- LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
- A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
- OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
- OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
- You can contact the author at :
- - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
- - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-
-/* **************************************************************
-* Includes
-****************************************************************/
-#include <stdlib.h> /* malloc, free, qsort */
-#include <string.h> /* memcpy, memset */
-#include "bitstream.h"
-#include "compiler.h"
-#define FSE_STATIC_LINKING_ONLY
-#include "fse.h"
-#include "error_private.h"
-
-
-/* **************************************************************
-* Error Management
-****************************************************************/
-#define FSE_isError ERR_isError
-#define FSE_STATIC_ASSERT(c) DEBUG_STATIC_ASSERT(c) /* use only *after* variable declarations */
-
-/* check and forward error code */
-#define CHECK_F(f) { size_t const e = f; if (FSE_isError(e)) return e; }
-
-
-/* **************************************************************
-* Templates
-****************************************************************/
-/*
- designed to be included
- for type-specific functions (template emulation in C)
- Objective is to write these functions only once, for improved maintenance
-*/
-
-/* safety checks */
-#ifndef FSE_FUNCTION_EXTENSION
-# error "FSE_FUNCTION_EXTENSION must be defined"
-#endif
-#ifndef FSE_FUNCTION_TYPE
-# error "FSE_FUNCTION_TYPE must be defined"
-#endif
-
-/* Function names */
-#define FSE_CAT(X,Y) X##Y
-#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y)
-#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y)
-
-
-/* Function templates */
-FSE_DTable* FSE_createDTable (unsigned tableLog)
-{
- if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX;
- return (FSE_DTable*)malloc( FSE_DTABLE_SIZE_U32(tableLog) * sizeof (U32) );
-}
-
-void FSE_freeDTable (FSE_DTable* dt)
-{
- free(dt);
-}
-
-size_t FSE_buildDTable(FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
- void* const tdPtr = dt+1; /* because *dt is unsigned, 32-bits aligned on 32-bits */
- FSE_DECODE_TYPE* const tableDecode = (FSE_DECODE_TYPE*) (tdPtr);
- U16 symbolNext[FSE_MAX_SYMBOL_VALUE+1];
-
- U32 const maxSV1 = maxSymbolValue + 1;
- U32 const tableSize = 1 << tableLog;
- U32 highThreshold = tableSize-1;
-
- /* Sanity Checks */
- if (maxSymbolValue > FSE_MAX_SYMBOL_VALUE) return ERROR(maxSymbolValue_tooLarge);
- if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge);
-
- /* Init, lay down lowprob symbols */
- { FSE_DTableHeader DTableH;
- DTableH.tableLog = (U16)tableLog;
- DTableH.fastMode = 1;
- { S16 const largeLimit= (S16)(1 << (tableLog-1));
- U32 s;
- for (s=0; s<maxSV1; s++) {
- if (normalizedCounter[s]==-1) {
- tableDecode[highThreshold--].symbol = (FSE_FUNCTION_TYPE)s;
- symbolNext[s] = 1;
- } else {
- if (normalizedCounter[s] >= largeLimit) DTableH.fastMode=0;
- symbolNext[s] = normalizedCounter[s];
- } } }
- memcpy(dt, &DTableH, sizeof(DTableH));
- }
-
- /* Spread symbols */
- { U32 const tableMask = tableSize-1;
- U32 const step = FSE_TABLESTEP(tableSize);
- U32 s, position = 0;
- for (s=0; s<maxSV1; s++) {
- int i;
- for (i=0; i<normalizedCounter[s]; i++) {
- tableDecode[position].symbol = (FSE_FUNCTION_TYPE)s;
- position = (position + step) & tableMask;
- while (position > highThreshold) position = (position + step) & tableMask; /* lowprob area */
- } }
- if (position!=0) return ERROR(GENERIC); /* position must reach all cells once, otherwise normalizedCounter is incorrect */
- }
-
- /* Build Decoding table */
- { U32 u;
- for (u=0; u<tableSize; u++) {
- FSE_FUNCTION_TYPE const symbol = (FSE_FUNCTION_TYPE)(tableDecode[u].symbol);
- U32 const nextState = symbolNext[symbol]++;
- tableDecode[u].nbBits = (BYTE) (tableLog - BIT_highbit32(nextState) );
- tableDecode[u].newState = (U16) ( (nextState << tableDecode[u].nbBits) - tableSize);
- } }
-
- return 0;
-}
-
-
-#ifndef FSE_COMMONDEFS_ONLY
-
-/*-*******************************************************
-* Decompression (Byte symbols)
-*********************************************************/
-size_t FSE_buildDTable_rle (FSE_DTable* dt, BYTE symbolValue)
-{
- void* ptr = dt;
- FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
- void* dPtr = dt + 1;
- FSE_decode_t* const cell = (FSE_decode_t*)dPtr;
-
- DTableH->tableLog = 0;
- DTableH->fastMode = 0;
-
- cell->newState = 0;
- cell->symbol = symbolValue;
- cell->nbBits = 0;
-
- return 0;
-}
-
-
-size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits)
-{
- void* ptr = dt;
- FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
- void* dPtr = dt + 1;
- FSE_decode_t* const dinfo = (FSE_decode_t*)dPtr;
- const unsigned tableSize = 1 << nbBits;
- const unsigned tableMask = tableSize - 1;
- const unsigned maxSV1 = tableMask+1;
- unsigned s;
-
- /* Sanity checks */
- if (nbBits < 1) return ERROR(GENERIC); /* min size */
-
- /* Build Decoding Table */
- DTableH->tableLog = (U16)nbBits;
- DTableH->fastMode = 1;
- for (s=0; s<maxSV1; s++) {
- dinfo[s].newState = 0;
- dinfo[s].symbol = (BYTE)s;
- dinfo[s].nbBits = (BYTE)nbBits;
- }
-
- return 0;
-}
-
-FORCE_INLINE_TEMPLATE size_t FSE_decompress_usingDTable_generic(
- void* dst, size_t maxDstSize,
- const void* cSrc, size_t cSrcSize,
- const FSE_DTable* dt, const unsigned fast)
-{
- BYTE* const ostart = (BYTE*) dst;
- BYTE* op = ostart;
- BYTE* const omax = op + maxDstSize;
- BYTE* const olimit = omax-3;
-
- BIT_DStream_t bitD;
- FSE_DState_t state1;
- FSE_DState_t state2;
-
- /* Init */
- CHECK_F(BIT_initDStream(&bitD, cSrc, cSrcSize));
-
- FSE_initDState(&state1, &bitD, dt);
- FSE_initDState(&state2, &bitD, dt);
-
-#define FSE_GETSYMBOL(statePtr) fast ? FSE_decodeSymbolFast(statePtr, &bitD) : FSE_decodeSymbol(statePtr, &bitD)
-
- /* 4 symbols per loop */
- for ( ; (BIT_reloadDStream(&bitD)==BIT_DStream_unfinished) & (op<olimit) ; op+=4) {
- op[0] = FSE_GETSYMBOL(&state1);
-
- if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8) /* This test must be static */
- BIT_reloadDStream(&bitD);
-
- op[1] = FSE_GETSYMBOL(&state2);
-
- if (FSE_MAX_TABLELOG*4+7 > sizeof(bitD.bitContainer)*8) /* This test must be static */
- { if (BIT_reloadDStream(&bitD) > BIT_DStream_unfinished) { op+=2; break; } }
-
- op[2] = FSE_GETSYMBOL(&state1);
-
- if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8) /* This test must be static */
- BIT_reloadDStream(&bitD);
-
- op[3] = FSE_GETSYMBOL(&state2);
- }
-
- /* tail */
- /* note : BIT_reloadDStream(&bitD) >= FSE_DStream_partiallyFilled; Ends at exactly BIT_DStream_completed */
- while (1) {
- if (op>(omax-2)) return ERROR(dstSize_tooSmall);
- *op++ = FSE_GETSYMBOL(&state1);
- if (BIT_reloadDStream(&bitD)==BIT_DStream_overflow) {
- *op++ = FSE_GETSYMBOL(&state2);
- break;
- }
-
- if (op>(omax-2)) return ERROR(dstSize_tooSmall);
- *op++ = FSE_GETSYMBOL(&state2);
- if (BIT_reloadDStream(&bitD)==BIT_DStream_overflow) {
- *op++ = FSE_GETSYMBOL(&state1);
- break;
- } }
-
- return op-ostart;
-}
-
-
-size_t FSE_decompress_usingDTable(void* dst, size_t originalSize,
- const void* cSrc, size_t cSrcSize,
- const FSE_DTable* dt)
-{
- const void* ptr = dt;
- const FSE_DTableHeader* DTableH = (const FSE_DTableHeader*)ptr;
- const U32 fastMode = DTableH->fastMode;
-
- /* select fast mode (static) */
- if (fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1);
- return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 0);
-}
-
-
-size_t FSE_decompress_wksp(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, FSE_DTable* workSpace, unsigned maxLog)
-{
- const BYTE* const istart = (const BYTE*)cSrc;
- const BYTE* ip = istart;
- short counting[FSE_MAX_SYMBOL_VALUE+1];
- unsigned tableLog;
- unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE;
-
- /* normal FSE decoding mode */
- size_t const NCountLength = FSE_readNCount (counting, &maxSymbolValue, &tableLog, istart, cSrcSize);
- if (FSE_isError(NCountLength)) return NCountLength;
- //if (NCountLength >= cSrcSize) return ERROR(srcSize_wrong); /* too small input size; supposed to be already checked in NCountLength, only remaining case : NCountLength==cSrcSize */
- if (tableLog > maxLog) return ERROR(tableLog_tooLarge);
- ip += NCountLength;
- cSrcSize -= NCountLength;
-
- CHECK_F( FSE_buildDTable (workSpace, counting, maxSymbolValue, tableLog) );
-
- return FSE_decompress_usingDTable (dst, dstCapacity, ip, cSrcSize, workSpace); /* always return, even if it is an error code */
-}
-
-
-typedef FSE_DTable DTable_max_t[FSE_DTABLE_SIZE_U32(FSE_MAX_TABLELOG)];
-
-size_t FSE_decompress(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize)
-{
- DTable_max_t dt; /* Static analyzer seems unable to understand this table will be properly initialized later */
- return FSE_decompress_wksp(dst, dstCapacity, cSrc, cSrcSize, dt, FSE_MAX_TABLELOG);
-}
-
-
-
-#endif /* FSE_COMMONDEFS_ONLY */
diff --git a/vendor/github.com/DataDog/zstd/hist.c b/vendor/github.com/DataDog/zstd/hist.c
deleted file mode 100644
index 45b7bab..0000000
--- a/vendor/github.com/DataDog/zstd/hist.c
+++ /dev/null
@@ -1,203 +0,0 @@
-/* ******************************************************************
- hist : Histogram functions
- part of Finite State Entropy project
- Copyright (C) 2013-present, Yann Collet.
-
- BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
- * Redistributions of source code must retain the above copyright
- notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
- copyright notice, this list of conditions and the following disclaimer
- in the documentation and/or other materials provided with the
- distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
- "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
- LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
- A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
- OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
- OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
- You can contact the author at :
- - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
- - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/* --- dependencies --- */
-#include "mem.h" /* U32, BYTE, etc. */
-#include "debug.h" /* assert, DEBUGLOG */
-#include "error_private.h" /* ERROR */
-#include "hist.h"
-
-
-/* --- Error management --- */
-unsigned HIST_isError(size_t code) { return ERR_isError(code); }
-
-/*-**************************************************************
- * Histogram functions
- ****************************************************************/
-unsigned HIST_count_simple(unsigned* count, unsigned* maxSymbolValuePtr,
- const void* src, size_t srcSize)
-{
- const BYTE* ip = (const BYTE*)src;
- const BYTE* const end = ip + srcSize;
- unsigned maxSymbolValue = *maxSymbolValuePtr;
- unsigned largestCount=0;
-
- memset(count, 0, (maxSymbolValue+1) * sizeof(*count));
- if (srcSize==0) { *maxSymbolValuePtr = 0; return 0; }
-
- while (ip<end) {
- assert(*ip <= maxSymbolValue);
- count[*ip++]++;
- }
-
- while (!count[maxSymbolValue]) maxSymbolValue--;
- *maxSymbolValuePtr = maxSymbolValue;
-
- { U32 s;
- for (s=0; s<=maxSymbolValue; s++)
- if (count[s] > largestCount) largestCount = count[s];
- }
-
- return largestCount;
-}
-
-typedef enum { trustInput, checkMaxSymbolValue } HIST_checkInput_e;
-
-/* HIST_count_parallel_wksp() :
- * store histogram into 4 intermediate tables, recombined at the end.
- * this design makes better use of OoO cpus,
- * and is noticeably faster when some values are heavily repeated.
- * But it needs some additional workspace for intermediate tables.
- * `workSpace` size must be a table of size >= HIST_WKSP_SIZE_U32.
- * @return : largest histogram frequency,
- * or an error code (notably when histogram would be larger than *maxSymbolValuePtr). */
-static size_t HIST_count_parallel_wksp(
- unsigned* count, unsigned* maxSymbolValuePtr,
- const void* source, size_t sourceSize,
- HIST_checkInput_e check,
- U32* const workSpace)
-{
- const BYTE* ip = (const BYTE*)source;
- const BYTE* const iend = ip+sourceSize;
- unsigned maxSymbolValue = *maxSymbolValuePtr;
- unsigned max=0;
- U32* const Counting1 = workSpace;
- U32* const Counting2 = Counting1 + 256;
- U32* const Counting3 = Counting2 + 256;
- U32* const Counting4 = Counting3 + 256;
-
- memset(workSpace, 0, 4*256*sizeof(unsigned));
-
- /* safety checks */
- if (!sourceSize) {
- memset(count, 0, maxSymbolValue + 1);
- *maxSymbolValuePtr = 0;
- return 0;
- }
- if (!maxSymbolValue) maxSymbolValue = 255; /* 0 == default */
-
- /* by stripes of 16 bytes */
- { U32 cached = MEM_read32(ip); ip += 4;
- while (ip < iend-15) {
- U32 c = cached; cached = MEM_read32(ip); ip += 4;
- Counting1[(BYTE) c ]++;
- Counting2[(BYTE)(c>>8) ]++;
- Counting3[(BYTE)(c>>16)]++;
- Counting4[ c>>24 ]++;
- c = cached; cached = MEM_read32(ip); ip += 4;
- Counting1[(BYTE) c ]++;
- Counting2[(BYTE)(c>>8) ]++;
- Counting3[(BYTE)(c>>16)]++;
- Counting4[ c>>24 ]++;
- c = cached; cached = MEM_read32(ip); ip += 4;
- Counting1[(BYTE) c ]++;
- Counting2[(BYTE)(c>>8) ]++;
- Counting3[(BYTE)(c>>16)]++;
- Counting4[ c>>24 ]++;
- c = cached; cached = MEM_read32(ip); ip += 4;
- Counting1[(BYTE) c ]++;
- Counting2[(BYTE)(c>>8) ]++;
- Counting3[(BYTE)(c>>16)]++;
- Counting4[ c>>24 ]++;
- }
- ip-=4;
- }
-
- /* finish last symbols */
- while (ip<iend) Counting1[*ip++]++;
-
- if (check) { /* verify stats will fit into destination table */
- U32 s; for (s=255; s>maxSymbolValue; s--) {
- Counting1[s] += Counting2[s] + Counting3[s] + Counting4[s];
- if (Counting1[s]) return ERROR(maxSymbolValue_tooSmall);
- } }
-
- { U32 s;
- if (maxSymbolValue > 255) maxSymbolValue = 255;
- for (s=0; s<=maxSymbolValue; s++) {
- count[s] = Counting1[s] + Counting2[s] + Counting3[s] + Counting4[s];
- if (count[s] > max) max = count[s];
- } }
-
- while (!count[maxSymbolValue]) maxSymbolValue--;
- *maxSymbolValuePtr = maxSymbolValue;
- return (size_t)max;
-}
-
-/* HIST_countFast_wksp() :
- * Same as HIST_countFast(), but using an externally provided scratch buffer.
- * `workSpace` is a writable buffer which must be 4-bytes aligned,
- * `workSpaceSize` must be >= HIST_WKSP_SIZE
- */
-size_t HIST_countFast_wksp(unsigned* count, unsigned* maxSymbolValuePtr,
- const void* source, size_t sourceSize,
- void* workSpace, size_t workSpaceSize)
-{
- if (sourceSize < 1500) /* heuristic threshold */
- return HIST_count_simple(count, maxSymbolValuePtr, source, sourceSize);
- if ((size_t)workSpace & 3) return ERROR(GENERIC); /* must be aligned on 4-bytes boundaries */
- if (workSpaceSize < HIST_WKSP_SIZE) return ERROR(workSpace_tooSmall);
- return HIST_count_parallel_wksp(count, maxSymbolValuePtr, source, sourceSize, trustInput, (U32*)workSpace);
-}
-
-/* fast variant (unsafe : won't check if src contains values beyond count[] limit) */
-size_t HIST_countFast(unsigned* count, unsigned* maxSymbolValuePtr,
- const void* source, size_t sourceSize)
-{
- unsigned tmpCounters[HIST_WKSP_SIZE_U32];
- return HIST_countFast_wksp(count, maxSymbolValuePtr, source, sourceSize, tmpCounters, sizeof(tmpCounters));
-}
-
-/* HIST_count_wksp() :
- * Same as HIST_count(), but using an externally provided scratch buffer.
- * `workSpace` size must be table of >= HIST_WKSP_SIZE_U32 unsigned */
-size_t HIST_count_wksp(unsigned* count, unsigned* maxSymbolValuePtr,
- const void* source, size_t sourceSize,
- void* workSpace, size_t workSpaceSize)
-{
- if ((size_t)workSpace & 3) return ERROR(GENERIC); /* must be aligned on 4-bytes boundaries */
- if (workSpaceSize < HIST_WKSP_SIZE) return ERROR(workSpace_tooSmall);
- if (*maxSymbolValuePtr < 255)
- return HIST_count_parallel_wksp(count, maxSymbolValuePtr, source, sourceSize, checkMaxSymbolValue, (U32*)workSpace);
- *maxSymbolValuePtr = 255;
- return HIST_countFast_wksp(count, maxSymbolValuePtr, source, sourceSize, workSpace, workSpaceSize);
-}
-
-size_t HIST_count(unsigned* count, unsigned* maxSymbolValuePtr,
- const void* src, size_t srcSize)
-{
- unsigned tmpCounters[HIST_WKSP_SIZE_U32];
- return HIST_count_wksp(count, maxSymbolValuePtr, src, srcSize, tmpCounters, sizeof(tmpCounters));
-}
diff --git a/vendor/github.com/DataDog/zstd/hist.h b/vendor/github.com/DataDog/zstd/hist.h
deleted file mode 100644
index 8b38935..0000000
--- a/vendor/github.com/DataDog/zstd/hist.h
+++ /dev/null
@@ -1,95 +0,0 @@
-/* ******************************************************************
- hist : Histogram functions
- part of Finite State Entropy project
- Copyright (C) 2013-present, Yann Collet.
-
- BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
- * Redistributions of source code must retain the above copyright
- notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
- copyright notice, this list of conditions and the following disclaimer
- in the documentation and/or other materials provided with the
- distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
- "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
- LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
- A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
- OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
- OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
- You can contact the author at :
- - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
- - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/* --- dependencies --- */
-#include <stddef.h> /* size_t */
-
-
-/* --- simple histogram functions --- */
-
-/*! HIST_count():
- * Provides the precise count of each byte within a table 'count'.
- * 'count' is a table of unsigned int, of minimum size (*maxSymbolValuePtr+1).
- * Updates *maxSymbolValuePtr with actual largest symbol value detected.
- * @return : count of the most frequent symbol (which isn't identified).
- * or an error code, which can be tested using HIST_isError().
- * note : if return == srcSize, there is only one symbol.
- */
-size_t HIST_count(unsigned* count, unsigned* maxSymbolValuePtr,
- const void* src, size_t srcSize);
-
-unsigned HIST_isError(size_t code); /**< tells if a return value is an error code */
-
-
-/* --- advanced histogram functions --- */
-
-#define HIST_WKSP_SIZE_U32 1024
-#define HIST_WKSP_SIZE (HIST_WKSP_SIZE_U32 * sizeof(unsigned))
-/** HIST_count_wksp() :
- * Same as HIST_count(), but using an externally provided scratch buffer.
- * Benefit is this function will use very little stack space.
- * `workSpace` is a writable buffer which must be 4-bytes aligned,
- * `workSpaceSize` must be >= HIST_WKSP_SIZE
- */
-size_t HIST_count_wksp(unsigned* count, unsigned* maxSymbolValuePtr,
- const void* src, size_t srcSize,
- void* workSpace, size_t workSpaceSize);
-
-/** HIST_countFast() :
- * same as HIST_count(), but blindly trusts that all byte values within src are <= *maxSymbolValuePtr.
- * This function is unsafe, and will segfault if any value within `src` is `> *maxSymbolValuePtr`
- */
-size_t HIST_countFast(unsigned* count, unsigned* maxSymbolValuePtr,
- const void* src, size_t srcSize);
-
-/** HIST_countFast_wksp() :
- * Same as HIST_countFast(), but using an externally provided scratch buffer.
- * `workSpace` is a writable buffer which must be 4-bytes aligned,
- * `workSpaceSize` must be >= HIST_WKSP_SIZE
- */
-size_t HIST_countFast_wksp(unsigned* count, unsigned* maxSymbolValuePtr,
- const void* src, size_t srcSize,
- void* workSpace, size_t workSpaceSize);
-
-/*! HIST_count_simple() :
- * Same as HIST_countFast(), this function is unsafe,
- * and will segfault if any value within `src` is `> *maxSymbolValuePtr`.
- * It is also a bit slower for large inputs.
- * However, it does not need any additional memory (not even on stack).
- * @return : count of the most frequent symbol.
- * Note this function doesn't produce any error (i.e. it must succeed).
- */
-unsigned HIST_count_simple(unsigned* count, unsigned* maxSymbolValuePtr,
- const void* src, size_t srcSize);
diff --git a/vendor/github.com/DataDog/zstd/huf.h b/vendor/github.com/DataDog/zstd/huf.h
deleted file mode 100644
index 6b572c4..0000000
--- a/vendor/github.com/DataDog/zstd/huf.h
+++ /dev/null
@@ -1,358 +0,0 @@
-/* ******************************************************************
- huff0 huffman codec,
- part of Finite State Entropy library
- Copyright (C) 2013-present, Yann Collet.
-
- BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
- * Redistributions of source code must retain the above copyright
- notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
- copyright notice, this list of conditions and the following disclaimer
- in the documentation and/or other materials provided with the
- distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
- "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
- LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
- A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
- OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
- OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
- You can contact the author at :
- - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#ifndef HUF_H_298734234
-#define HUF_H_298734234
-
-/* *** Dependencies *** */
-#include <stddef.h> /* size_t */
-
-
-/* *** library symbols visibility *** */
-/* Note : when linking with -fvisibility=hidden on gcc, or by default on Visual,
- * HUF symbols remain "private" (internal symbols for library only).
- * Set macro FSE_DLL_EXPORT to 1 if you want HUF symbols visible on DLL interface */
-#if defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) && defined(__GNUC__) && (__GNUC__ >= 4)
-# define HUF_PUBLIC_API __attribute__ ((visibility ("default")))
-#elif defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) /* Visual expected */
-# define HUF_PUBLIC_API __declspec(dllexport)
-#elif defined(FSE_DLL_IMPORT) && (FSE_DLL_IMPORT==1)
-# define HUF_PUBLIC_API __declspec(dllimport) /* not required, just to generate faster code (saves a function pointer load from IAT and an indirect jump) */
-#else
-# define HUF_PUBLIC_API
-#endif
-
-
-/* ========================== */
-/* *** simple functions *** */
-/* ========================== */
-
-/** HUF_compress() :
- * Compress content from buffer 'src', of size 'srcSize', into buffer 'dst'.
- * 'dst' buffer must be already allocated.
- * Compression runs faster if `dstCapacity` >= HUF_compressBound(srcSize).
- * `srcSize` must be <= `HUF_BLOCKSIZE_MAX` == 128 KB.
- * @return : size of compressed data (<= `dstCapacity`).
- * Special values : if return == 0, srcData is not compressible => Nothing is stored within dst !!!
- * if HUF_isError(return), compression failed (more details using HUF_getErrorName())
- */
-HUF_PUBLIC_API size_t HUF_compress(void* dst, size_t dstCapacity,
- const void* src, size_t srcSize);
-
-/** HUF_decompress() :
- * Decompress HUF data from buffer 'cSrc', of size 'cSrcSize',
- * into already allocated buffer 'dst', of minimum size 'dstSize'.
- * `originalSize` : **must** be the ***exact*** size of original (uncompressed) data.
- * Note : in contrast with FSE, HUF_decompress can regenerate
- * RLE (cSrcSize==1) and uncompressed (cSrcSize==dstSize) data,
- * because it knows size to regenerate (originalSize).
- * @return : size of regenerated data (== originalSize),
- * or an error code, which can be tested using HUF_isError()
- */
-HUF_PUBLIC_API size_t HUF_decompress(void* dst, size_t originalSize,
- const void* cSrc, size_t cSrcSize);
-
-
-/* *** Tool functions *** */
-#define HUF_BLOCKSIZE_MAX (128 * 1024) /**< maximum input size for a single block compressed with HUF_compress */
-HUF_PUBLIC_API size_t HUF_compressBound(size_t size); /**< maximum compressed size (worst case) */
-
-/* Error Management */
-HUF_PUBLIC_API unsigned HUF_isError(size_t code); /**< tells if a return value is an error code */
-HUF_PUBLIC_API const char* HUF_getErrorName(size_t code); /**< provides error code string (useful for debugging) */
-
-
-/* *** Advanced function *** */
-
-/** HUF_compress2() :
- * Same as HUF_compress(), but offers control over `maxSymbolValue` and `tableLog`.
- * `maxSymbolValue` must be <= HUF_SYMBOLVALUE_MAX .
- * `tableLog` must be `<= HUF_TABLELOG_MAX` . */
-HUF_PUBLIC_API size_t HUF_compress2 (void* dst, size_t dstCapacity,
- const void* src, size_t srcSize,
- unsigned maxSymbolValue, unsigned tableLog);
-
-/** HUF_compress4X_wksp() :
- * Same as HUF_compress2(), but uses externally allocated `workSpace`.
- * `workspace` must have minimum alignment of 4, and be at least as large as HUF_WORKSPACE_SIZE */
-#define HUF_WORKSPACE_SIZE (6 << 10)
-#define HUF_WORKSPACE_SIZE_U32 (HUF_WORKSPACE_SIZE / sizeof(U32))
-HUF_PUBLIC_API size_t HUF_compress4X_wksp (void* dst, size_t dstCapacity,
- const void* src, size_t srcSize,
- unsigned maxSymbolValue, unsigned tableLog,
- void* workSpace, size_t wkspSize);
-
-#endif /* HUF_H_298734234 */
-
-/* ******************************************************************
- * WARNING !!
- * The following section contains advanced and experimental definitions
- * which shall never be used in the context of a dynamic library,
- * because they are not guaranteed to remain stable in the future.
- * Only consider them in association with static linking.
- * *****************************************************************/
-#if defined(HUF_STATIC_LINKING_ONLY) && !defined(HUF_H_HUF_STATIC_LINKING_ONLY)
-#define HUF_H_HUF_STATIC_LINKING_ONLY
-
-/* *** Dependencies *** */
-#include "mem.h" /* U32 */
-
-
-/* *** Constants *** */
-#define HUF_TABLELOG_MAX 12 /* max runtime value of tableLog (due to static allocation); can be modified up to HUF_ABSOLUTEMAX_TABLELOG */
-#define HUF_TABLELOG_DEFAULT 11 /* default tableLog value when none specified */
-#define HUF_SYMBOLVALUE_MAX 255
-
-#define HUF_TABLELOG_ABSOLUTEMAX 15 /* absolute limit of HUF_MAX_TABLELOG. Beyond that value, code does not work */
-#if (HUF_TABLELOG_MAX > HUF_TABLELOG_ABSOLUTEMAX)
-# error "HUF_TABLELOG_MAX is too large !"
-#endif
-
-
-/* ****************************************
-* Static allocation
-******************************************/
-/* HUF buffer bounds */
-#define HUF_CTABLEBOUND 129
-#define HUF_BLOCKBOUND(size) (size + (size>>8) + 8) /* only true when incompressible is pre-filtered with fast heuristic */
-#define HUF_COMPRESSBOUND(size) (HUF_CTABLEBOUND + HUF_BLOCKBOUND(size)) /* Macro version, useful for static allocation */
-
-/* static allocation of HUF's Compression Table */
-#define HUF_CTABLE_SIZE_U32(maxSymbolValue) ((maxSymbolValue)+1) /* Use tables of U32, for proper alignment */
-#define HUF_CTABLE_SIZE(maxSymbolValue) (HUF_CTABLE_SIZE_U32(maxSymbolValue) * sizeof(U32))
-#define HUF_CREATE_STATIC_CTABLE(name, maxSymbolValue) \
- U32 name##hb[HUF_CTABLE_SIZE_U32(maxSymbolValue)]; \
- void* name##hv = &(name##hb); \
- HUF_CElt* name = (HUF_CElt*)(name##hv) /* no final ; */
-
-/* static allocation of HUF's DTable */
-typedef U32 HUF_DTable;
-#define HUF_DTABLE_SIZE(maxTableLog) (1 + (1<<(maxTableLog)))
-#define HUF_CREATE_STATIC_DTABLEX1(DTable, maxTableLog) \
- HUF_DTable DTable[HUF_DTABLE_SIZE((maxTableLog)-1)] = { ((U32)((maxTableLog)-1) * 0x01000001) }
-#define HUF_CREATE_STATIC_DTABLEX2(DTable, maxTableLog) \
- HUF_DTable DTable[HUF_DTABLE_SIZE(maxTableLog)] = { ((U32)(maxTableLog) * 0x01000001) }
-
-
-/* ****************************************
-* Advanced decompression functions
-******************************************/
-size_t HUF_decompress4X1 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< single-symbol decoder */
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< double-symbols decoder */
-#endif
-
-size_t HUF_decompress4X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< decodes RLE and uncompressed */
-size_t HUF_decompress4X_hufOnly(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< considers RLE and uncompressed as errors */
-size_t HUF_decompress4X_hufOnly_wksp(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize); /**< considers RLE and uncompressed as errors */
-size_t HUF_decompress4X1_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< single-symbol decoder */
-size_t HUF_decompress4X1_DCtx_wksp(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize); /**< single-symbol decoder */
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_decompress4X2_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< double-symbols decoder */
-size_t HUF_decompress4X2_DCtx_wksp(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize); /**< double-symbols decoder */
-#endif
-
-
-/* ****************************************
- * HUF detailed API
- * ****************************************/
-
-/*! HUF_compress() does the following:
- * 1. count symbol occurrence from source[] into table count[] using FSE_count() (exposed within "fse.h")
- * 2. (optional) refine tableLog using HUF_optimalTableLog()
- * 3. build Huffman table from count using HUF_buildCTable()
- * 4. save Huffman table to memory buffer using HUF_writeCTable()
- * 5. encode the data stream using HUF_compress4X_usingCTable()
- *
- * The following API allows targeting specific sub-functions for advanced tasks.
- * For example, it's possible to compress several blocks using the same 'CTable',
- * or to save and regenerate 'CTable' using external methods.
- */
-unsigned HUF_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue);
-typedef struct HUF_CElt_s HUF_CElt; /* incomplete type */
-size_t HUF_buildCTable (HUF_CElt* CTable, const unsigned* count, unsigned maxSymbolValue, unsigned maxNbBits); /* @return : maxNbBits; CTable and count can overlap. In which case, CTable will overwrite count content */
-size_t HUF_writeCTable (void* dst, size_t maxDstSize, const HUF_CElt* CTable, unsigned maxSymbolValue, unsigned huffLog);
-size_t HUF_compress4X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable);
-
-typedef enum {
- HUF_repeat_none, /**< Cannot use the previous table */
- HUF_repeat_check, /**< Can use the previous table but it must be checked. Note : The previous table must have been constructed by HUF_compress{1, 4}X_repeat */
- HUF_repeat_valid /**< Can use the previous table and it is assumed to be valid */
- } HUF_repeat;
-/** HUF_compress4X_repeat() :
- * Same as HUF_compress4X_wksp(), but considers using hufTable if *repeat != HUF_repeat_none.
- * If it uses hufTable it does not modify hufTable or repeat.
- * If it doesn't, it sets *repeat = HUF_repeat_none, and it sets hufTable to the table used.
- * If preferRepeat then the old table will always be used if valid. */
-size_t HUF_compress4X_repeat(void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- unsigned maxSymbolValue, unsigned tableLog,
- void* workSpace, size_t wkspSize, /**< `workSpace` must be aligned on 4-bytes boundaries, `wkspSize` must be >= HUF_WORKSPACE_SIZE */
- HUF_CElt* hufTable, HUF_repeat* repeat, int preferRepeat, int bmi2);
-
-/** HUF_buildCTable_wksp() :
- * Same as HUF_buildCTable(), but using externally allocated scratch buffer.
- * `workSpace` must be aligned on 4-bytes boundaries, and its size must be >= HUF_CTABLE_WORKSPACE_SIZE.
- */
-#define HUF_CTABLE_WORKSPACE_SIZE_U32 (2*HUF_SYMBOLVALUE_MAX +1 +1)
-#define HUF_CTABLE_WORKSPACE_SIZE (HUF_CTABLE_WORKSPACE_SIZE_U32 * sizeof(unsigned))
-size_t HUF_buildCTable_wksp (HUF_CElt* tree,
- const unsigned* count, U32 maxSymbolValue, U32 maxNbBits,
- void* workSpace, size_t wkspSize);
-
-/*! HUF_readStats() :
- * Read compact Huffman tree, saved by HUF_writeCTable().
- * `huffWeight` is destination buffer.
- * @return : size read from `src` , or an error Code .
- * Note : Needed by HUF_readCTable() and HUF_readDTableXn() . */
-size_t HUF_readStats(BYTE* huffWeight, size_t hwSize,
- U32* rankStats, U32* nbSymbolsPtr, U32* tableLogPtr,
- const void* src, size_t srcSize);
-
-/** HUF_readCTable() :
- * Loading a CTable saved with HUF_writeCTable() */
-size_t HUF_readCTable (HUF_CElt* CTable, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize);
-
-/** HUF_getNbBits() :
- * Read nbBits from CTable symbolTable, for symbol `symbolValue` presumed <= HUF_SYMBOLVALUE_MAX
- * Note 1 : is not inlined, as HUF_CElt definition is private
- * Note 2 : const void* used, so that it can provide a statically allocated table as argument (which uses type U32) */
-U32 HUF_getNbBits(const void* symbolTable, U32 symbolValue);
-
-/*
- * HUF_decompress() does the following:
- * 1. select the decompression algorithm (X1, X2) based on pre-computed heuristics
- * 2. build Huffman table from save, using HUF_readDTableX?()
- * 3. decode 1 or 4 segments in parallel using HUF_decompress?X?_usingDTable()
- */
-
-/** HUF_selectDecoder() :
- * Tells which decoder is likely to decode faster,
- * based on a set of pre-computed metrics.
- * @return : 0==HUF_decompress4X1, 1==HUF_decompress4X2 .
- * Assumption : 0 < dstSize <= 128 KB */
-U32 HUF_selectDecoder (size_t dstSize, size_t cSrcSize);
-
-/**
- * The minimum workspace size for the `workSpace` used in
- * HUF_readDTableX1_wksp() and HUF_readDTableX2_wksp().
- *
- * The space used depends on HUF_TABLELOG_MAX, ranging from ~1500 bytes when
- * HUF_TABLE_LOG_MAX=12 to ~1850 bytes when HUF_TABLE_LOG_MAX=15.
- * Buffer overflow errors may potentially occur if code modifications result in
- * a required workspace size greater than that specified in the following
- * macro.
- */
-#define HUF_DECOMPRESS_WORKSPACE_SIZE (2 << 10)
-#define HUF_DECOMPRESS_WORKSPACE_SIZE_U32 (HUF_DECOMPRESS_WORKSPACE_SIZE / sizeof(U32))
-
-#ifndef HUF_FORCE_DECOMPRESS_X2
-size_t HUF_readDTableX1 (HUF_DTable* DTable, const void* src, size_t srcSize);
-size_t HUF_readDTableX1_wksp (HUF_DTable* DTable, const void* src, size_t srcSize, void* workSpace, size_t wkspSize);
-#endif
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_readDTableX2 (HUF_DTable* DTable, const void* src, size_t srcSize);
-size_t HUF_readDTableX2_wksp (HUF_DTable* DTable, const void* src, size_t srcSize, void* workSpace, size_t wkspSize);
-#endif
-
-size_t HUF_decompress4X_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable);
-#ifndef HUF_FORCE_DECOMPRESS_X2
-size_t HUF_decompress4X1_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable);
-#endif
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_decompress4X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable);
-#endif
-
-
-/* ====================== */
-/* single stream variants */
-/* ====================== */
-
-size_t HUF_compress1X (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog);
-size_t HUF_compress1X_wksp (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize); /**< `workSpace` must be a table of at least HUF_WORKSPACE_SIZE_U32 unsigned */
-size_t HUF_compress1X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable);
-/** HUF_compress1X_repeat() :
- * Same as HUF_compress1X_wksp(), but considers using hufTable if *repeat != HUF_repeat_none.
- * If it uses hufTable it does not modify hufTable or repeat.
- * If it doesn't, it sets *repeat = HUF_repeat_none, and it sets hufTable to the table used.
- * If preferRepeat then the old table will always be used if valid. */
-size_t HUF_compress1X_repeat(void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- unsigned maxSymbolValue, unsigned tableLog,
- void* workSpace, size_t wkspSize, /**< `workSpace` must be aligned on 4-bytes boundaries, `wkspSize` must be >= HUF_WORKSPACE_SIZE */
- HUF_CElt* hufTable, HUF_repeat* repeat, int preferRepeat, int bmi2);
-
-size_t HUF_decompress1X1 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /* single-symbol decoder */
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /* double-symbol decoder */
-#endif
-
-size_t HUF_decompress1X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);
-size_t HUF_decompress1X_DCtx_wksp (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize);
-#ifndef HUF_FORCE_DECOMPRESS_X2
-size_t HUF_decompress1X1_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< single-symbol decoder */
-size_t HUF_decompress1X1_DCtx_wksp(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize); /**< single-symbol decoder */
-#endif
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_decompress1X2_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< double-symbols decoder */
-size_t HUF_decompress1X2_DCtx_wksp(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize); /**< double-symbols decoder */
-#endif
-
-size_t HUF_decompress1X_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); /**< automatic selection of sing or double symbol decoder, based on DTable */
-#ifndef HUF_FORCE_DECOMPRESS_X2
-size_t HUF_decompress1X1_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable);
-#endif
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_decompress1X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable);
-#endif
-
-/* BMI2 variants.
- * If the CPU has BMI2 support, pass bmi2=1, otherwise pass bmi2=0.
- */
-size_t HUF_decompress1X_usingDTable_bmi2(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable, int bmi2);
-#ifndef HUF_FORCE_DECOMPRESS_X2
-size_t HUF_decompress1X1_DCtx_wksp_bmi2(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize, int bmi2);
-#endif
-size_t HUF_decompress4X_usingDTable_bmi2(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable, int bmi2);
-size_t HUF_decompress4X_hufOnly_wksp_bmi2(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize, int bmi2);
-
-#endif /* HUF_STATIC_LINKING_ONLY */
-
-#if defined (__cplusplus)
-}
-#endif
diff --git a/vendor/github.com/DataDog/zstd/huf_compress.c b/vendor/github.com/DataDog/zstd/huf_compress.c
deleted file mode 100644
index f074f1e..0000000
--- a/vendor/github.com/DataDog/zstd/huf_compress.c
+++ /dev/null
@@ -1,798 +0,0 @@
-/* ******************************************************************
- Huffman encoder, part of New Generation Entropy library
- Copyright (C) 2013-2016, Yann Collet.
-
- BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
- * Redistributions of source code must retain the above copyright
- notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
- copyright notice, this list of conditions and the following disclaimer
- in the documentation and/or other materials provided with the
- distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
- "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
- LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
- A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
- OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
- OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
- You can contact the author at :
- - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy
- - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/* **************************************************************
-* Compiler specifics
-****************************************************************/
-#ifdef _MSC_VER /* Visual Studio */
-# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */
-#endif
-
-
-/* **************************************************************
-* Includes
-****************************************************************/
-#include <string.h> /* memcpy, memset */
-#include <stdio.h> /* printf (debug) */
-#include "compiler.h"
-#include "bitstream.h"
-#include "hist.h"
-#define FSE_STATIC_LINKING_ONLY /* FSE_optimalTableLog_internal */
-#include "fse.h" /* header compression */
-#define HUF_STATIC_LINKING_ONLY
-#include "huf.h"
-#include "error_private.h"
-
-
-/* **************************************************************
-* Error Management
-****************************************************************/
-#define HUF_isError ERR_isError
-#define HUF_STATIC_ASSERT(c) DEBUG_STATIC_ASSERT(c) /* use only *after* variable declarations */
-#define CHECK_V_F(e, f) size_t const e = f; if (ERR_isError(e)) return e
-#define CHECK_F(f) { CHECK_V_F(_var_err__, f); }
-
-
-/* **************************************************************
-* Utils
-****************************************************************/
-unsigned HUF_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue)
-{
- return FSE_optimalTableLog_internal(maxTableLog, srcSize, maxSymbolValue, 1);
-}
-
-
-/* *******************************************************
-* HUF : Huffman block compression
-*********************************************************/
-/* HUF_compressWeights() :
- * Same as FSE_compress(), but dedicated to huff0's weights compression.
- * The use case needs much less stack memory.
- * Note : all elements within weightTable are supposed to be <= HUF_TABLELOG_MAX.
- */
-#define MAX_FSE_TABLELOG_FOR_HUFF_HEADER 6
-static size_t HUF_compressWeights (void* dst, size_t dstSize, const void* weightTable, size_t wtSize)
-{
- BYTE* const ostart = (BYTE*) dst;
- BYTE* op = ostart;
- BYTE* const oend = ostart + dstSize;
-
- unsigned maxSymbolValue = HUF_TABLELOG_MAX;
- U32 tableLog = MAX_FSE_TABLELOG_FOR_HUFF_HEADER;
-
- FSE_CTable CTable[FSE_CTABLE_SIZE_U32(MAX_FSE_TABLELOG_FOR_HUFF_HEADER, HUF_TABLELOG_MAX)];
- BYTE scratchBuffer[1<<MAX_FSE_TABLELOG_FOR_HUFF_HEADER];
-
- unsigned count[HUF_TABLELOG_MAX+1];
- S16 norm[HUF_TABLELOG_MAX+1];
-
- /* init conditions */
- if (wtSize <= 1) return 0; /* Not compressible */
-
- /* Scan input and build symbol stats */
- { unsigned const maxCount = HIST_count_simple(count, &maxSymbolValue, weightTable, wtSize); /* never fails */
- if (maxCount == wtSize) return 1; /* only a single symbol in src : rle */
- if (maxCount == 1) return 0; /* each symbol present maximum once => not compressible */
- }
-
- tableLog = FSE_optimalTableLog(tableLog, wtSize, maxSymbolValue);
- CHECK_F( FSE_normalizeCount(norm, tableLog, count, wtSize, maxSymbolValue) );
-
- /* Write table description header */
- { CHECK_V_F(hSize, FSE_writeNCount(op, oend-op, norm, maxSymbolValue, tableLog) );
- op += hSize;
- }
-
- /* Compress */
- CHECK_F( FSE_buildCTable_wksp(CTable, norm, maxSymbolValue, tableLog, scratchBuffer, sizeof(scratchBuffer)) );
- { CHECK_V_F(cSize, FSE_compress_usingCTable(op, oend - op, weightTable, wtSize, CTable) );
- if (cSize == 0) return 0; /* not enough space for compressed data */
- op += cSize;
- }
-
- return op-ostart;
-}
-
-
-struct HUF_CElt_s {
- U16 val;
- BYTE nbBits;
-}; /* typedef'd to HUF_CElt within "huf.h" */
-
-/*! HUF_writeCTable() :
- `CTable` : Huffman tree to save, using huf representation.
- @return : size of saved CTable */
-size_t HUF_writeCTable (void* dst, size_t maxDstSize,
- const HUF_CElt* CTable, unsigned maxSymbolValue, unsigned huffLog)
-{
- BYTE bitsToWeight[HUF_TABLELOG_MAX + 1]; /* precomputed conversion table */
- BYTE huffWeight[HUF_SYMBOLVALUE_MAX];
- BYTE* op = (BYTE*)dst;
- U32 n;
-
- /* check conditions */
- if (maxSymbolValue > HUF_SYMBOLVALUE_MAX) return ERROR(maxSymbolValue_tooLarge);
-
- /* convert to weight */
- bitsToWeight[0] = 0;
- for (n=1; n<huffLog+1; n++)
- bitsToWeight[n] = (BYTE)(huffLog + 1 - n);
- for (n=0; n<maxSymbolValue; n++)
- huffWeight[n] = bitsToWeight[CTable[n].nbBits];
-
- /* attempt weights compression by FSE */
- { CHECK_V_F(hSize, HUF_compressWeights(op+1, maxDstSize-1, huffWeight, maxSymbolValue) );
- if ((hSize>1) & (hSize < maxSymbolValue/2)) { /* FSE compressed */
- op[0] = (BYTE)hSize;
- return hSize+1;
- } }
-
- /* write raw values as 4-bits (max : 15) */
- if (maxSymbolValue > (256-128)) return ERROR(GENERIC); /* should not happen : likely means source cannot be compressed */
- if (((maxSymbolValue+1)/2) + 1 > maxDstSize) return ERROR(dstSize_tooSmall); /* not enough space within dst buffer */
- op[0] = (BYTE)(128 /*special case*/ + (maxSymbolValue-1));
- huffWeight[maxSymbolValue] = 0; /* to be sure it doesn't cause msan issue in final combination */
- for (n=0; n<maxSymbolValue; n+=2)
- op[(n/2)+1] = (BYTE)((huffWeight[n] << 4) + huffWeight[n+1]);
- return ((maxSymbolValue+1)/2) + 1;
-}
-
-
-size_t HUF_readCTable (HUF_CElt* CTable, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize)
-{
- BYTE huffWeight[HUF_SYMBOLVALUE_MAX + 1]; /* init not required, even though some static analyzer may complain */
- U32 rankVal[HUF_TABLELOG_ABSOLUTEMAX + 1]; /* large enough for values from 0 to 16 */
- U32 tableLog = 0;
- U32 nbSymbols = 0;
-
- /* get symbol weights */
- CHECK_V_F(readSize, HUF_readStats(huffWeight, HUF_SYMBOLVALUE_MAX+1, rankVal, &nbSymbols, &tableLog, src, srcSize));
-
- /* check result */
- if (tableLog > HUF_TABLELOG_MAX) return ERROR(tableLog_tooLarge);
- if (nbSymbols > *maxSymbolValuePtr+1) return ERROR(maxSymbolValue_tooSmall);
-
- /* Prepare base value per rank */
- { U32 n, nextRankStart = 0;
- for (n=1; n<=tableLog; n++) {
- U32 current = nextRankStart;
- nextRankStart += (rankVal[n] << (n-1));
- rankVal[n] = current;
- } }
-
- /* fill nbBits */
- { U32 n; for (n=0; n<nbSymbols; n++) {
- const U32 w = huffWeight[n];
- CTable[n].nbBits = (BYTE)(tableLog + 1 - w);
- } }
-
- /* fill val */
- { U16 nbPerRank[HUF_TABLELOG_MAX+2] = {0}; /* support w=0=>n=tableLog+1 */
- U16 valPerRank[HUF_TABLELOG_MAX+2] = {0};
- { U32 n; for (n=0; n<nbSymbols; n++) nbPerRank[CTable[n].nbBits]++; }
- /* determine stating value per rank */
- valPerRank[tableLog+1] = 0; /* for w==0 */
- { U16 min = 0;
- U32 n; for (n=tableLog; n>0; n--) { /* start at n=tablelog <-> w=1 */
- valPerRank[n] = min; /* get starting value within each rank */
- min += nbPerRank[n];
- min >>= 1;
- } }
- /* assign value within rank, symbol order */
- { U32 n; for (n=0; n<nbSymbols; n++) CTable[n].val = valPerRank[CTable[n].nbBits]++; }
- }
-
- *maxSymbolValuePtr = nbSymbols - 1;
- return readSize;
-}
-
-U32 HUF_getNbBits(const void* symbolTable, U32 symbolValue)
-{
- const HUF_CElt* table = (const HUF_CElt*)symbolTable;
- assert(symbolValue <= HUF_SYMBOLVALUE_MAX);
- return table[symbolValue].nbBits;
-}
-
-
-typedef struct nodeElt_s {
- U32 count;
- U16 parent;
- BYTE byte;
- BYTE nbBits;
-} nodeElt;
-
-static U32 HUF_setMaxHeight(nodeElt* huffNode, U32 lastNonNull, U32 maxNbBits)
-{
- const U32 largestBits = huffNode[lastNonNull].nbBits;
- if (largestBits <= maxNbBits) return largestBits; /* early exit : no elt > maxNbBits */
-
- /* there are several too large elements (at least >= 2) */
- { int totalCost = 0;
- const U32 baseCost = 1 << (largestBits - maxNbBits);
- U32 n = lastNonNull;
-
- while (huffNode[n].nbBits > maxNbBits) {
- totalCost += baseCost - (1 << (largestBits - huffNode[n].nbBits));
- huffNode[n].nbBits = (BYTE)maxNbBits;
- n --;
- } /* n stops at huffNode[n].nbBits <= maxNbBits */
- while (huffNode[n].nbBits == maxNbBits) n--; /* n end at index of smallest symbol using < maxNbBits */
-
- /* renorm totalCost */
- totalCost >>= (largestBits - maxNbBits); /* note : totalCost is necessarily a multiple of baseCost */
-
- /* repay normalized cost */
- { U32 const noSymbol = 0xF0F0F0F0;
- U32 rankLast[HUF_TABLELOG_MAX+2];
- int pos;
-
- /* Get pos of last (smallest) symbol per rank */
- memset(rankLast, 0xF0, sizeof(rankLast));
- { U32 currentNbBits = maxNbBits;
- for (pos=n ; pos >= 0; pos--) {
- if (huffNode[pos].nbBits >= currentNbBits) continue;
- currentNbBits = huffNode[pos].nbBits; /* < maxNbBits */
- rankLast[maxNbBits-currentNbBits] = pos;
- } }
-
- while (totalCost > 0) {
- U32 nBitsToDecrease = BIT_highbit32(totalCost) + 1;
- for ( ; nBitsToDecrease > 1; nBitsToDecrease--) {
- U32 highPos = rankLast[nBitsToDecrease];
- U32 lowPos = rankLast[nBitsToDecrease-1];
- if (highPos == noSymbol) continue;
- if (lowPos == noSymbol) break;
- { U32 const highTotal = huffNode[highPos].count;
- U32 const lowTotal = 2 * huffNode[lowPos].count;
- if (highTotal <= lowTotal) break;
- } }
- /* only triggered when no more rank 1 symbol left => find closest one (note : there is necessarily at least one !) */
- /* HUF_MAX_TABLELOG test just to please gcc 5+; but it should not be necessary */
- while ((nBitsToDecrease<=HUF_TABLELOG_MAX) && (rankLast[nBitsToDecrease] == noSymbol))
- nBitsToDecrease ++;
- totalCost -= 1 << (nBitsToDecrease-1);
- if (rankLast[nBitsToDecrease-1] == noSymbol)
- rankLast[nBitsToDecrease-1] = rankLast[nBitsToDecrease]; /* this rank is no longer empty */
- huffNode[rankLast[nBitsToDecrease]].nbBits ++;
- if (rankLast[nBitsToDecrease] == 0) /* special case, reached largest symbol */
- rankLast[nBitsToDecrease] = noSymbol;
- else {
- rankLast[nBitsToDecrease]--;
- if (huffNode[rankLast[nBitsToDecrease]].nbBits != maxNbBits-nBitsToDecrease)
- rankLast[nBitsToDecrease] = noSymbol; /* this rank is now empty */
- } } /* while (totalCost > 0) */
-
- while (totalCost < 0) { /* Sometimes, cost correction overshoot */
- if (rankLast[1] == noSymbol) { /* special case : no rank 1 symbol (using maxNbBits-1); let's create one from largest rank 0 (using maxNbBits) */
- while (huffNode[n].nbBits == maxNbBits) n--;
- huffNode[n+1].nbBits--;
- rankLast[1] = n+1;
- totalCost++;
- continue;
- }
- huffNode[ rankLast[1] + 1 ].nbBits--;
- rankLast[1]++;
- totalCost ++;
- } } } /* there are several too large elements (at least >= 2) */
-
- return maxNbBits;
-}
-
-
-typedef struct {
- U32 base;
- U32 current;
-} rankPos;
-
-static void HUF_sort(nodeElt* huffNode, const unsigned* count, U32 maxSymbolValue)
-{
- rankPos rank[32];
- U32 n;
-
- memset(rank, 0, sizeof(rank));
- for (n=0; n<=maxSymbolValue; n++) {
- U32 r = BIT_highbit32(count[n] + 1);
- rank[r].base ++;
- }
- for (n=30; n>0; n--) rank[n-1].base += rank[n].base;
- for (n=0; n<32; n++) rank[n].current = rank[n].base;
- for (n=0; n<=maxSymbolValue; n++) {
- U32 const c = count[n];
- U32 const r = BIT_highbit32(c+1) + 1;
- U32 pos = rank[r].current++;
- while ((pos > rank[r].base) && (c > huffNode[pos-1].count)) {
- huffNode[pos] = huffNode[pos-1];
- pos--;
- }
- huffNode[pos].count = c;
- huffNode[pos].byte = (BYTE)n;
- }
-}
-
-
-/** HUF_buildCTable_wksp() :
- * Same as HUF_buildCTable(), but using externally allocated scratch buffer.
- * `workSpace` must be aligned on 4-bytes boundaries, and be at least as large as a table of HUF_CTABLE_WORKSPACE_SIZE_U32 unsigned.
- */
-#define STARTNODE (HUF_SYMBOLVALUE_MAX+1)
-typedef nodeElt huffNodeTable[HUF_CTABLE_WORKSPACE_SIZE_U32];
-size_t HUF_buildCTable_wksp (HUF_CElt* tree, const unsigned* count, U32 maxSymbolValue, U32 maxNbBits, void* workSpace, size_t wkspSize)
-{
- nodeElt* const huffNode0 = (nodeElt*)workSpace;
- nodeElt* const huffNode = huffNode0+1;
- U32 n, nonNullRank;
- int lowS, lowN;
- U16 nodeNb = STARTNODE;
- U32 nodeRoot;
-
- /* safety checks */
- if (((size_t)workSpace & 3) != 0) return ERROR(GENERIC); /* must be aligned on 4-bytes boundaries */
- if (wkspSize < sizeof(huffNodeTable)) return ERROR(workSpace_tooSmall);
- if (maxNbBits == 0) maxNbBits = HUF_TABLELOG_DEFAULT;
- if (maxSymbolValue > HUF_SYMBOLVALUE_MAX) return ERROR(maxSymbolValue_tooLarge);
- memset(huffNode0, 0, sizeof(huffNodeTable));
-
- /* sort, decreasing order */
- HUF_sort(huffNode, count, maxSymbolValue);
-
- /* init for parents */
- nonNullRank = maxSymbolValue;
- while(huffNode[nonNullRank].count == 0) nonNullRank--;
- lowS = nonNullRank; nodeRoot = nodeNb + lowS - 1; lowN = nodeNb;
- huffNode[nodeNb].count = huffNode[lowS].count + huffNode[lowS-1].count;
- huffNode[lowS].parent = huffNode[lowS-1].parent = nodeNb;
- nodeNb++; lowS-=2;
- for (n=nodeNb; n<=nodeRoot; n++) huffNode[n].count = (U32)(1U<<30);
- huffNode0[0].count = (U32)(1U<<31); /* fake entry, strong barrier */
-
- /* create parents */
- while (nodeNb <= nodeRoot) {
- U32 n1 = (huffNode[lowS].count < huffNode[lowN].count) ? lowS-- : lowN++;
- U32 n2 = (huffNode[lowS].count < huffNode[lowN].count) ? lowS-- : lowN++;
- huffNode[nodeNb].count = huffNode[n1].count + huffNode[n2].count;
- huffNode[n1].parent = huffNode[n2].parent = nodeNb;
- nodeNb++;
- }
-
- /* distribute weights (unlimited tree height) */
- huffNode[nodeRoot].nbBits = 0;
- for (n=nodeRoot-1; n>=STARTNODE; n--)
- huffNode[n].nbBits = huffNode[ huffNode[n].parent ].nbBits + 1;
- for (n=0; n<=nonNullRank; n++)
- huffNode[n].nbBits = huffNode[ huffNode[n].parent ].nbBits + 1;
-
- /* enforce maxTableLog */
- maxNbBits = HUF_setMaxHeight(huffNode, nonNullRank, maxNbBits);
-
- /* fill result into tree (val, nbBits) */
- { U16 nbPerRank[HUF_TABLELOG_MAX+1] = {0};
- U16 valPerRank[HUF_TABLELOG_MAX+1] = {0};
- if (maxNbBits > HUF_TABLELOG_MAX) return ERROR(GENERIC); /* check fit into table */
- for (n=0; n<=nonNullRank; n++)
- nbPerRank[huffNode[n].nbBits]++;
- /* determine stating value per rank */
- { U16 min = 0;
- for (n=maxNbBits; n>0; n--) {
- valPerRank[n] = min; /* get starting value within each rank */
- min += nbPerRank[n];
- min >>= 1;
- } }
- for (n=0; n<=maxSymbolValue; n++)
- tree[huffNode[n].byte].nbBits = huffNode[n].nbBits; /* push nbBits per symbol, symbol order */
- for (n=0; n<=maxSymbolValue; n++)
- tree[n].val = valPerRank[tree[n].nbBits]++; /* assign value within rank, symbol order */
- }
-
- return maxNbBits;
-}
-
-/** HUF_buildCTable() :
- * @return : maxNbBits
- * Note : count is used before tree is written, so they can safely overlap
- */
-size_t HUF_buildCTable (HUF_CElt* tree, const unsigned* count, unsigned maxSymbolValue, unsigned maxNbBits)
-{
- huffNodeTable nodeTable;
- return HUF_buildCTable_wksp(tree, count, maxSymbolValue, maxNbBits, nodeTable, sizeof(nodeTable));
-}
-
-static size_t HUF_estimateCompressedSize(HUF_CElt* CTable, const unsigned* count, unsigned maxSymbolValue)
-{
- size_t nbBits = 0;
- int s;
- for (s = 0; s <= (int)maxSymbolValue; ++s) {
- nbBits += CTable[s].nbBits * count[s];
- }
- return nbBits >> 3;
-}
-
-static int HUF_validateCTable(const HUF_CElt* CTable, const unsigned* count, unsigned maxSymbolValue) {
- int bad = 0;
- int s;
- for (s = 0; s <= (int)maxSymbolValue; ++s) {
- bad |= (count[s] != 0) & (CTable[s].nbBits == 0);
- }
- return !bad;
-}
-
-size_t HUF_compressBound(size_t size) { return HUF_COMPRESSBOUND(size); }
-
-FORCE_INLINE_TEMPLATE void
-HUF_encodeSymbol(BIT_CStream_t* bitCPtr, U32 symbol, const HUF_CElt* CTable)
-{
- BIT_addBitsFast(bitCPtr, CTable[symbol].val, CTable[symbol].nbBits);
-}
-
-#define HUF_FLUSHBITS(s) BIT_flushBits(s)
-
-#define HUF_FLUSHBITS_1(stream) \
- if (sizeof((stream)->bitContainer)*8 < HUF_TABLELOG_MAX*2+7) HUF_FLUSHBITS(stream)
-
-#define HUF_FLUSHBITS_2(stream) \
- if (sizeof((stream)->bitContainer)*8 < HUF_TABLELOG_MAX*4+7) HUF_FLUSHBITS(stream)
-
-FORCE_INLINE_TEMPLATE size_t
-HUF_compress1X_usingCTable_internal_body(void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- const HUF_CElt* CTable)
-{
- const BYTE* ip = (const BYTE*) src;
- BYTE* const ostart = (BYTE*)dst;
- BYTE* const oend = ostart + dstSize;
- BYTE* op = ostart;
- size_t n;
- BIT_CStream_t bitC;
-
- /* init */
- if (dstSize < 8) return 0; /* not enough space to compress */
- { size_t const initErr = BIT_initCStream(&bitC, op, oend-op);
- if (HUF_isError(initErr)) return 0; }
-
- n = srcSize & ~3; /* join to mod 4 */
- switch (srcSize & 3)
- {
- case 3 : HUF_encodeSymbol(&bitC, ip[n+ 2], CTable);
- HUF_FLUSHBITS_2(&bitC);
- /* fall-through */
- case 2 : HUF_encodeSymbol(&bitC, ip[n+ 1], CTable);
- HUF_FLUSHBITS_1(&bitC);
- /* fall-through */
- case 1 : HUF_encodeSymbol(&bitC, ip[n+ 0], CTable);
- HUF_FLUSHBITS(&bitC);
- /* fall-through */
- case 0 : /* fall-through */
- default: break;
- }
-
- for (; n>0; n-=4) { /* note : n&3==0 at this stage */
- HUF_encodeSymbol(&bitC, ip[n- 1], CTable);
- HUF_FLUSHBITS_1(&bitC);
- HUF_encodeSymbol(&bitC, ip[n- 2], CTable);
- HUF_FLUSHBITS_2(&bitC);
- HUF_encodeSymbol(&bitC, ip[n- 3], CTable);
- HUF_FLUSHBITS_1(&bitC);
- HUF_encodeSymbol(&bitC, ip[n- 4], CTable);
- HUF_FLUSHBITS(&bitC);
- }
-
- return BIT_closeCStream(&bitC);
-}
-
-#if DYNAMIC_BMI2
-
-static TARGET_ATTRIBUTE("bmi2") size_t
-HUF_compress1X_usingCTable_internal_bmi2(void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- const HUF_CElt* CTable)
-{
- return HUF_compress1X_usingCTable_internal_body(dst, dstSize, src, srcSize, CTable);
-}
-
-static size_t
-HUF_compress1X_usingCTable_internal_default(void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- const HUF_CElt* CTable)
-{
- return HUF_compress1X_usingCTable_internal_body(dst, dstSize, src, srcSize, CTable);
-}
-
-static size_t
-HUF_compress1X_usingCTable_internal(void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- const HUF_CElt* CTable, const int bmi2)
-{
- if (bmi2) {
- return HUF_compress1X_usingCTable_internal_bmi2(dst, dstSize, src, srcSize, CTable);
- }
- return HUF_compress1X_usingCTable_internal_default(dst, dstSize, src, srcSize, CTable);
-}
-
-#else
-
-static size_t
-HUF_compress1X_usingCTable_internal(void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- const HUF_CElt* CTable, const int bmi2)
-{
- (void)bmi2;
- return HUF_compress1X_usingCTable_internal_body(dst, dstSize, src, srcSize, CTable);
-}
-
-#endif
-
-size_t HUF_compress1X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable)
-{
- return HUF_compress1X_usingCTable_internal(dst, dstSize, src, srcSize, CTable, /* bmi2 */ 0);
-}
-
-
-static size_t
-HUF_compress4X_usingCTable_internal(void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- const HUF_CElt* CTable, int bmi2)
-{
- size_t const segmentSize = (srcSize+3)/4; /* first 3 segments */
- const BYTE* ip = (const BYTE*) src;
- const BYTE* const iend = ip + srcSize;
- BYTE* const ostart = (BYTE*) dst;
- BYTE* const oend = ostart + dstSize;
- BYTE* op = ostart;
-
- if (dstSize < 6 + 1 + 1 + 1 + 8) return 0; /* minimum space to compress successfully */
- if (srcSize < 12) return 0; /* no saving possible : too small input */
- op += 6; /* jumpTable */
-
- { CHECK_V_F(cSize, HUF_compress1X_usingCTable_internal(op, oend-op, ip, segmentSize, CTable, bmi2) );
- if (cSize==0) return 0;
- assert(cSize <= 65535);
- MEM_writeLE16(ostart, (U16)cSize);
- op += cSize;
- }
-
- ip += segmentSize;
- { CHECK_V_F(cSize, HUF_compress1X_usingCTable_internal(op, oend-op, ip, segmentSize, CTable, bmi2) );
- if (cSize==0) return 0;
- assert(cSize <= 65535);
- MEM_writeLE16(ostart+2, (U16)cSize);
- op += cSize;
- }
-
- ip += segmentSize;
- { CHECK_V_F(cSize, HUF_compress1X_usingCTable_internal(op, oend-op, ip, segmentSize, CTable, bmi2) );
- if (cSize==0) return 0;
- assert(cSize <= 65535);
- MEM_writeLE16(ostart+4, (U16)cSize);
- op += cSize;
- }
-
- ip += segmentSize;
- { CHECK_V_F(cSize, HUF_compress1X_usingCTable_internal(op, oend-op, ip, iend-ip, CTable, bmi2) );
- if (cSize==0) return 0;
- op += cSize;
- }
-
- return op-ostart;
-}
-
-size_t HUF_compress4X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable)
-{
- return HUF_compress4X_usingCTable_internal(dst, dstSize, src, srcSize, CTable, /* bmi2 */ 0);
-}
-
-typedef enum { HUF_singleStream, HUF_fourStreams } HUF_nbStreams_e;
-
-static size_t HUF_compressCTable_internal(
- BYTE* const ostart, BYTE* op, BYTE* const oend,
- const void* src, size_t srcSize,
- HUF_nbStreams_e nbStreams, const HUF_CElt* CTable, const int bmi2)
-{
- size_t const cSize = (nbStreams==HUF_singleStream) ?
- HUF_compress1X_usingCTable_internal(op, oend - op, src, srcSize, CTable, bmi2) :
- HUF_compress4X_usingCTable_internal(op, oend - op, src, srcSize, CTable, bmi2);
- if (HUF_isError(cSize)) { return cSize; }
- if (cSize==0) { return 0; } /* uncompressible */
- op += cSize;
- /* check compressibility */
- if ((size_t)(op-ostart) >= srcSize-1) { return 0; }
- return op-ostart;
-}
-
-typedef struct {
- unsigned count[HUF_SYMBOLVALUE_MAX + 1];
- HUF_CElt CTable[HUF_SYMBOLVALUE_MAX + 1];
- huffNodeTable nodeTable;
-} HUF_compress_tables_t;
-
-/* HUF_compress_internal() :
- * `workSpace` must a table of at least HUF_WORKSPACE_SIZE_U32 unsigned */
-static size_t
-HUF_compress_internal (void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- unsigned maxSymbolValue, unsigned huffLog,
- HUF_nbStreams_e nbStreams,
- void* workSpace, size_t wkspSize,
- HUF_CElt* oldHufTable, HUF_repeat* repeat, int preferRepeat,
- const int bmi2)
-{
- HUF_compress_tables_t* const table = (HUF_compress_tables_t*)workSpace;
- BYTE* const ostart = (BYTE*)dst;
- BYTE* const oend = ostart + dstSize;
- BYTE* op = ostart;
-
- /* checks & inits */
- if (((size_t)workSpace & 3) != 0) return ERROR(GENERIC); /* must be aligned on 4-bytes boundaries */
- if (wkspSize < HUF_WORKSPACE_SIZE) return ERROR(workSpace_tooSmall);
- if (!srcSize) return 0; /* Uncompressed */
- if (!dstSize) return 0; /* cannot fit anything within dst budget */
- if (srcSize > HUF_BLOCKSIZE_MAX) return ERROR(srcSize_wrong); /* current block size limit */
- if (huffLog > HUF_TABLELOG_MAX) return ERROR(tableLog_tooLarge);
- if (maxSymbolValue > HUF_SYMBOLVALUE_MAX) return ERROR(maxSymbolValue_tooLarge);
- if (!maxSymbolValue) maxSymbolValue = HUF_SYMBOLVALUE_MAX;
- if (!huffLog) huffLog = HUF_TABLELOG_DEFAULT;
-
- /* Heuristic : If old table is valid, use it for small inputs */
- if (preferRepeat && repeat && *repeat == HUF_repeat_valid) {
- return HUF_compressCTable_internal(ostart, op, oend,
- src, srcSize,
- nbStreams, oldHufTable, bmi2);
- }
-
- /* Scan input and build symbol stats */
- { CHECK_V_F(largest, HIST_count_wksp (table->count, &maxSymbolValue, (const BYTE*)src, srcSize, workSpace, wkspSize) );
- if (largest == srcSize) { *ostart = ((const BYTE*)src)[0]; return 1; } /* single symbol, rle */
- if (largest <= (srcSize >> 7)+4) return 0; /* heuristic : probably not compressible enough */
- }
-
- /* Check validity of previous table */
- if ( repeat
- && *repeat == HUF_repeat_check
- && !HUF_validateCTable(oldHufTable, table->count, maxSymbolValue)) {
- *repeat = HUF_repeat_none;
- }
- /* Heuristic : use existing table for small inputs */
- if (preferRepeat && repeat && *repeat != HUF_repeat_none) {
- return HUF_compressCTable_internal(ostart, op, oend,
- src, srcSize,
- nbStreams, oldHufTable, bmi2);
- }
-
- /* Build Huffman Tree */
- huffLog = HUF_optimalTableLog(huffLog, srcSize, maxSymbolValue);
- { size_t const maxBits = HUF_buildCTable_wksp(table->CTable, table->count,
- maxSymbolValue, huffLog,
- table->nodeTable, sizeof(table->nodeTable));
- CHECK_F(maxBits);
- huffLog = (U32)maxBits;
- /* Zero unused symbols in CTable, so we can check it for validity */
- memset(table->CTable + (maxSymbolValue + 1), 0,
- sizeof(table->CTable) - ((maxSymbolValue + 1) * sizeof(HUF_CElt)));
- }
-
- /* Write table description header */
- { CHECK_V_F(hSize, HUF_writeCTable (op, dstSize, table->CTable, maxSymbolValue, huffLog) );
- /* Check if using previous huffman table is beneficial */
- if (repeat && *repeat != HUF_repeat_none) {
- size_t const oldSize = HUF_estimateCompressedSize(oldHufTable, table->count, maxSymbolValue);
- size_t const newSize = HUF_estimateCompressedSize(table->CTable, table->count, maxSymbolValue);
- if (oldSize <= hSize + newSize || hSize + 12 >= srcSize) {
- return HUF_compressCTable_internal(ostart, op, oend,
- src, srcSize,
- nbStreams, oldHufTable, bmi2);
- } }
-
- /* Use the new huffman table */
- if (hSize + 12ul >= srcSize) { return 0; }
- op += hSize;
- if (repeat) { *repeat = HUF_repeat_none; }
- if (oldHufTable)
- memcpy(oldHufTable, table->CTable, sizeof(table->CTable)); /* Save new table */
- }
- return HUF_compressCTable_internal(ostart, op, oend,
- src, srcSize,
- nbStreams, table->CTable, bmi2);
-}
-
-
-size_t HUF_compress1X_wksp (void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- unsigned maxSymbolValue, unsigned huffLog,
- void* workSpace, size_t wkspSize)
-{
- return HUF_compress_internal(dst, dstSize, src, srcSize,
- maxSymbolValue, huffLog, HUF_singleStream,
- workSpace, wkspSize,
- NULL, NULL, 0, 0 /*bmi2*/);
-}
-
-size_t HUF_compress1X_repeat (void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- unsigned maxSymbolValue, unsigned huffLog,
- void* workSpace, size_t wkspSize,
- HUF_CElt* hufTable, HUF_repeat* repeat, int preferRepeat, int bmi2)
-{
- return HUF_compress_internal(dst, dstSize, src, srcSize,
- maxSymbolValue, huffLog, HUF_singleStream,
- workSpace, wkspSize, hufTable,
- repeat, preferRepeat, bmi2);
-}
-
-size_t HUF_compress1X (void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- unsigned maxSymbolValue, unsigned huffLog)
-{
- unsigned workSpace[HUF_WORKSPACE_SIZE_U32];
- return HUF_compress1X_wksp(dst, dstSize, src, srcSize, maxSymbolValue, huffLog, workSpace, sizeof(workSpace));
-}
-
-/* HUF_compress4X_repeat():
- * compress input using 4 streams.
- * provide workspace to generate compression tables */
-size_t HUF_compress4X_wksp (void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- unsigned maxSymbolValue, unsigned huffLog,
- void* workSpace, size_t wkspSize)
-{
- return HUF_compress_internal(dst, dstSize, src, srcSize,
- maxSymbolValue, huffLog, HUF_fourStreams,
- workSpace, wkspSize,
- NULL, NULL, 0, 0 /*bmi2*/);
-}
-
-/* HUF_compress4X_repeat():
- * compress input using 4 streams.
- * re-use an existing huffman compression table */
-size_t HUF_compress4X_repeat (void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- unsigned maxSymbolValue, unsigned huffLog,
- void* workSpace, size_t wkspSize,
- HUF_CElt* hufTable, HUF_repeat* repeat, int preferRepeat, int bmi2)
-{
- return HUF_compress_internal(dst, dstSize, src, srcSize,
- maxSymbolValue, huffLog, HUF_fourStreams,
- workSpace, wkspSize,
- hufTable, repeat, preferRepeat, bmi2);
-}
-
-size_t HUF_compress2 (void* dst, size_t dstSize,
- const void* src, size_t srcSize,
- unsigned maxSymbolValue, unsigned huffLog)
-{
- unsigned workSpace[HUF_WORKSPACE_SIZE_U32];
- return HUF_compress4X_wksp(dst, dstSize, src, srcSize, maxSymbolValue, huffLog, workSpace, sizeof(workSpace));
-}
-
-size_t HUF_compress (void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
- return HUF_compress2(dst, maxDstSize, src, srcSize, 255, HUF_TABLELOG_DEFAULT);
-}
diff --git a/vendor/github.com/DataDog/zstd/huf_decompress.c b/vendor/github.com/DataDog/zstd/huf_decompress.c
deleted file mode 100644
index 3f8bd29..0000000
--- a/vendor/github.com/DataDog/zstd/huf_decompress.c
+++ /dev/null
@@ -1,1232 +0,0 @@
-/* ******************************************************************
- huff0 huffman decoder,
- part of Finite State Entropy library
- Copyright (C) 2013-present, Yann Collet.
-
- BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
- * Redistributions of source code must retain the above copyright
- notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
- copyright notice, this list of conditions and the following disclaimer
- in the documentation and/or other materials provided with the
- distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
- "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
- LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
- A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
- OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
- OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
- You can contact the author at :
- - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-
-/* **************************************************************
-* Dependencies
-****************************************************************/
-#include <string.h> /* memcpy, memset */
-#include "compiler.h"
-#include "bitstream.h" /* BIT_* */
-#include "fse.h" /* to compress headers */
-#define HUF_STATIC_LINKING_ONLY
-#include "huf.h"
-#include "error_private.h"
-
-/* **************************************************************
-* Macros
-****************************************************************/
-
-/* These two optional macros force the use one way or another of the two
- * Huffman decompression implementations. You can't force in both directions
- * at the same time.
- */
-#if defined(HUF_FORCE_DECOMPRESS_X1) && \
- defined(HUF_FORCE_DECOMPRESS_X2)
-#error "Cannot force the use of the X1 and X2 decoders at the same time!"
-#endif
-
-
-/* **************************************************************
-* Error Management
-****************************************************************/
-#define HUF_isError ERR_isError
-#define CHECK_F(f) { size_t const err_ = (f); if (HUF_isError(err_)) return err_; }
-
-
-/* **************************************************************
-* Byte alignment for workSpace management
-****************************************************************/
-#define HUF_ALIGN(x, a) HUF_ALIGN_MASK((x), (a) - 1)
-#define HUF_ALIGN_MASK(x, mask) (((x) + (mask)) & ~(mask))
-
-
-/* **************************************************************
-* BMI2 Variant Wrappers
-****************************************************************/
-#if DYNAMIC_BMI2
-
-#define HUF_DGEN(fn) \
- \
- static size_t fn##_default( \
- void* dst, size_t dstSize, \
- const void* cSrc, size_t cSrcSize, \
- const HUF_DTable* DTable) \
- { \
- return fn##_body(dst, dstSize, cSrc, cSrcSize, DTable); \
- } \
- \
- static TARGET_ATTRIBUTE("bmi2") size_t fn##_bmi2( \
- void* dst, size_t dstSize, \
- const void* cSrc, size_t cSrcSize, \
- const HUF_DTable* DTable) \
- { \
- return fn##_body(dst, dstSize, cSrc, cSrcSize, DTable); \
- } \
- \
- static size_t fn(void* dst, size_t dstSize, void const* cSrc, \
- size_t cSrcSize, HUF_DTable const* DTable, int bmi2) \
- { \
- if (bmi2) { \
- return fn##_bmi2(dst, dstSize, cSrc, cSrcSize, DTable); \
- } \
- return fn##_default(dst, dstSize, cSrc, cSrcSize, DTable); \
- }
-
-#else
-
-#define HUF_DGEN(fn) \
- static size_t fn(void* dst, size_t dstSize, void const* cSrc, \
- size_t cSrcSize, HUF_DTable const* DTable, int bmi2) \
- { \
- (void)bmi2; \
- return fn##_body(dst, dstSize, cSrc, cSrcSize, DTable); \
- }
-
-#endif
-
-
-/*-***************************/
-/* generic DTableDesc */
-/*-***************************/
-typedef struct { BYTE maxTableLog; BYTE tableType; BYTE tableLog; BYTE reserved; } DTableDesc;
-
-static DTableDesc HUF_getDTableDesc(const HUF_DTable* table)
-{
- DTableDesc dtd;
- memcpy(&dtd, table, sizeof(dtd));
- return dtd;
-}
-
-
-#ifndef HUF_FORCE_DECOMPRESS_X2
-
-/*-***************************/
-/* single-symbol decoding */
-/*-***************************/
-typedef struct { BYTE byte; BYTE nbBits; } HUF_DEltX1; /* single-symbol decoding */
-
-size_t HUF_readDTableX1_wksp(HUF_DTable* DTable, const void* src, size_t srcSize, void* workSpace, size_t wkspSize)
-{
- U32 tableLog = 0;
- U32 nbSymbols = 0;
- size_t iSize;
- void* const dtPtr = DTable + 1;
- HUF_DEltX1* const dt = (HUF_DEltX1*)dtPtr;
-
- U32* rankVal;
- BYTE* huffWeight;
- size_t spaceUsed32 = 0;
-
- rankVal = (U32 *)workSpace + spaceUsed32;
- spaceUsed32 += HUF_TABLELOG_ABSOLUTEMAX + 1;
- huffWeight = (BYTE *)((U32 *)workSpace + spaceUsed32);
- spaceUsed32 += HUF_ALIGN(HUF_SYMBOLVALUE_MAX + 1, sizeof(U32)) >> 2;
-
- if ((spaceUsed32 << 2) > wkspSize) return ERROR(tableLog_tooLarge);
-
- DEBUG_STATIC_ASSERT(sizeof(DTableDesc) == sizeof(HUF_DTable));
- /* memset(huffWeight, 0, sizeof(huffWeight)); */ /* is not necessary, even though some analyzer complain ... */
-
- iSize = HUF_readStats(huffWeight, HUF_SYMBOLVALUE_MAX + 1, rankVal, &nbSymbols, &tableLog, src, srcSize);
- if (HUF_isError(iSize)) return iSize;
-
- /* Table header */
- { DTableDesc dtd = HUF_getDTableDesc(DTable);
- if (tableLog > (U32)(dtd.maxTableLog+1)) return ERROR(tableLog_tooLarge); /* DTable too small, Huffman tree cannot fit in */
- dtd.tableType = 0;
- dtd.tableLog = (BYTE)tableLog;
- memcpy(DTable, &dtd, sizeof(dtd));
- }
-
- /* Calculate starting value for each rank */
- { U32 n, nextRankStart = 0;
- for (n=1; n<tableLog+1; n++) {
- U32 const current = nextRankStart;
- nextRankStart += (rankVal[n] << (n-1));
- rankVal[n] = current;
- } }
-
- /* fill DTable */
- { U32 n;
- for (n=0; n<nbSymbols; n++) {
- U32 const w = huffWeight[n];
- U32 const length = (1 << w) >> 1;
- U32 u;
- HUF_DEltX1 D;
- D.byte = (BYTE)n; D.nbBits = (BYTE)(tableLog + 1 - w);
- for (u = rankVal[w]; u < rankVal[w] + length; u++)
- dt[u] = D;
- rankVal[w] += length;
- } }
-
- return iSize;
-}
-
-size_t HUF_readDTableX1(HUF_DTable* DTable, const void* src, size_t srcSize)
-{
- U32 workSpace[HUF_DECOMPRESS_WORKSPACE_SIZE_U32];
- return HUF_readDTableX1_wksp(DTable, src, srcSize,
- workSpace, sizeof(workSpace));
-}
-
-FORCE_INLINE_TEMPLATE BYTE
-HUF_decodeSymbolX1(BIT_DStream_t* Dstream, const HUF_DEltX1* dt, const U32 dtLog)
-{
- size_t const val = BIT_lookBitsFast(Dstream, dtLog); /* note : dtLog >= 1 */
- BYTE const c = dt[val].byte;
- BIT_skipBits(Dstream, dt[val].nbBits);
- return c;
-}
-
-#define HUF_DECODE_SYMBOLX1_0(ptr, DStreamPtr) \
- *ptr++ = HUF_decodeSymbolX1(DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX1_1(ptr, DStreamPtr) \
- if (MEM_64bits() || (HUF_TABLELOG_MAX<=12)) \
- HUF_DECODE_SYMBOLX1_0(ptr, DStreamPtr)
-
-#define HUF_DECODE_SYMBOLX1_2(ptr, DStreamPtr) \
- if (MEM_64bits()) \
- HUF_DECODE_SYMBOLX1_0(ptr, DStreamPtr)
-
-HINT_INLINE size_t
-HUF_decodeStreamX1(BYTE* p, BIT_DStream_t* const bitDPtr, BYTE* const pEnd, const HUF_DEltX1* const dt, const U32 dtLog)
-{
- BYTE* const pStart = p;
-
- /* up to 4 symbols at a time */
- while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) & (p < pEnd-3)) {
- HUF_DECODE_SYMBOLX1_2(p, bitDPtr);
- HUF_DECODE_SYMBOLX1_1(p, bitDPtr);
- HUF_DECODE_SYMBOLX1_2(p, bitDPtr);
- HUF_DECODE_SYMBOLX1_0(p, bitDPtr);
- }
-
- /* [0-3] symbols remaining */
- if (MEM_32bits())
- while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) & (p < pEnd))
- HUF_DECODE_SYMBOLX1_0(p, bitDPtr);
-
- /* no more data to retrieve from bitstream, no need to reload */
- while (p < pEnd)
- HUF_DECODE_SYMBOLX1_0(p, bitDPtr);
-
- return pEnd-pStart;
-}
-
-FORCE_INLINE_TEMPLATE size_t
-HUF_decompress1X1_usingDTable_internal_body(
- void* dst, size_t dstSize,
- const void* cSrc, size_t cSrcSize,
- const HUF_DTable* DTable)
-{
- BYTE* op = (BYTE*)dst;
- BYTE* const oend = op + dstSize;
- const void* dtPtr = DTable + 1;
- const HUF_DEltX1* const dt = (const HUF_DEltX1*)dtPtr;
- BIT_DStream_t bitD;
- DTableDesc const dtd = HUF_getDTableDesc(DTable);
- U32 const dtLog = dtd.tableLog;
-
- CHECK_F( BIT_initDStream(&bitD, cSrc, cSrcSize) );
-
- HUF_decodeStreamX1(op, &bitD, oend, dt, dtLog);
-
- if (!BIT_endOfDStream(&bitD)) return ERROR(corruption_detected);
-
- return dstSize;
-}
-
-FORCE_INLINE_TEMPLATE size_t
-HUF_decompress4X1_usingDTable_internal_body(
- void* dst, size_t dstSize,
- const void* cSrc, size_t cSrcSize,
- const HUF_DTable* DTable)
-{
- /* Check */
- if (cSrcSize < 10) return ERROR(corruption_detected); /* strict minimum : jump table + 1 byte per stream */
-
- { const BYTE* const istart = (const BYTE*) cSrc;
- BYTE* const ostart = (BYTE*) dst;
- BYTE* const oend = ostart + dstSize;
- const void* const dtPtr = DTable + 1;
- const HUF_DEltX1* const dt = (const HUF_DEltX1*)dtPtr;
-
- /* Init */
- BIT_DStream_t bitD1;
- BIT_DStream_t bitD2;
- BIT_DStream_t bitD3;
- BIT_DStream_t bitD4;
- size_t const length1 = MEM_readLE16(istart);
- size_t const length2 = MEM_readLE16(istart+2);
- size_t const length3 = MEM_readLE16(istart+4);
- size_t const length4 = cSrcSize - (length1 + length2 + length3 + 6);
- const BYTE* const istart1 = istart + 6; /* jumpTable */
- const BYTE* const istart2 = istart1 + length1;
- const BYTE* const istart3 = istart2 + length2;
- const BYTE* const istart4 = istart3 + length3;
- const size_t segmentSize = (dstSize+3) / 4;
- BYTE* const opStart2 = ostart + segmentSize;
- BYTE* const opStart3 = opStart2 + segmentSize;
- BYTE* const opStart4 = opStart3 + segmentSize;
- BYTE* op1 = ostart;
- BYTE* op2 = opStart2;
- BYTE* op3 = opStart3;
- BYTE* op4 = opStart4;
- U32 endSignal = BIT_DStream_unfinished;
- DTableDesc const dtd = HUF_getDTableDesc(DTable);
- U32 const dtLog = dtd.tableLog;
-
- if (length4 > cSrcSize) return ERROR(corruption_detected); /* overflow */
- CHECK_F( BIT_initDStream(&bitD1, istart1, length1) );
- CHECK_F( BIT_initDStream(&bitD2, istart2, length2) );
- CHECK_F( BIT_initDStream(&bitD3, istart3, length3) );
- CHECK_F( BIT_initDStream(&bitD4, istart4, length4) );
-
- /* up to 16 symbols per loop (4 symbols per stream) in 64-bit mode */
- endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
- while ( (endSignal==BIT_DStream_unfinished) && (op4<(oend-3)) ) {
- HUF_DECODE_SYMBOLX1_2(op1, &bitD1);
- HUF_DECODE_SYMBOLX1_2(op2, &bitD2);
- HUF_DECODE_SYMBOLX1_2(op3, &bitD3);
- HUF_DECODE_SYMBOLX1_2(op4, &bitD4);
- HUF_DECODE_SYMBOLX1_1(op1, &bitD1);
- HUF_DECODE_SYMBOLX1_1(op2, &bitD2);
- HUF_DECODE_SYMBOLX1_1(op3, &bitD3);
- HUF_DECODE_SYMBOLX1_1(op4, &bitD4);
- HUF_DECODE_SYMBOLX1_2(op1, &bitD1);
- HUF_DECODE_SYMBOLX1_2(op2, &bitD2);
- HUF_DECODE_SYMBOLX1_2(op3, &bitD3);
- HUF_DECODE_SYMBOLX1_2(op4, &bitD4);
- HUF_DECODE_SYMBOLX1_0(op1, &bitD1);
- HUF_DECODE_SYMBOLX1_0(op2, &bitD2);
- HUF_DECODE_SYMBOLX1_0(op3, &bitD3);
- HUF_DECODE_SYMBOLX1_0(op4, &bitD4);
- BIT_relo